FrankYang0529 commented on code in PR #15908:
URL: https://github.com/apache/kafka/pull/15908#discussion_r1641165864


##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java:
##########
@@ -16,834 +16,986 @@
  */
 package org.apache.kafka.tools.consumer.group;
 
+import kafka.test.ClusterConfig;
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTemplate;
+import kafka.test.junit.ClusterTestExtensions;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.ConsumerGroupDescription;
+import org.apache.kafka.clients.admin.NewTopic;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.GroupProtocol;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.RangeAssignor;
 import org.apache.kafka.clients.consumer.RoundRobinAssignor;
 import org.apache.kafka.common.ConsumerGroupState;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.common.utils.AppInfoParser;
 import org.apache.kafka.common.utils.Exit;
 import org.apache.kafka.test.TestUtils;
 import org.apache.kafka.tools.ToolsTestUtils;
 
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.MethodSource;
-import org.junit.jupiter.params.provider.ValueSource;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
 import java.util.Optional;
-import java.util.Properties;
+import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 import static org.apache.kafka.test.TestUtils.RANDOM;
-import static 
org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertInstanceOf;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
 
-public class DescribeConsumerGroupTest extends ConsumerGroupCommandTest {
+@Tag("integration")
+@ExtendWith(value = ClusterTestExtensions.class)
+public class DescribeConsumerGroupTest {
+    private static final String TOPIC_PREFIX = "test.topic.";
+    private static final String GROUP_PREFIX = "test.group.";
     private static final List<List<String>> DESCRIBE_TYPE_OFFSETS = 
Arrays.asList(Collections.singletonList(""), 
Collections.singletonList("--offsets"));
     private static final List<List<String>> DESCRIBE_TYPE_MEMBERS = 
Arrays.asList(Collections.singletonList("--members"), 
Arrays.asList("--members", "--verbose"));
     private static final List<List<String>> DESCRIBE_TYPE_STATE = 
Collections.singletonList(Collections.singletonList("--state"));
-    private static final List<List<String>> DESCRIBE_TYPES;
+    private static final List<List<String>> DESCRIBE_TYPES = 
Stream.of(DESCRIBE_TYPE_OFFSETS, DESCRIBE_TYPE_MEMBERS, 
DESCRIBE_TYPE_STATE).flatMap(Collection::stream).collect(Collectors.toList());
+    private ClusterInstance clusterInstance;
 
-    static {
-        List<List<String>> describeTypes = new ArrayList<>();
-
-        describeTypes.addAll(DESCRIBE_TYPE_OFFSETS);
-        describeTypes.addAll(DESCRIBE_TYPE_MEMBERS);
-        describeTypes.addAll(DESCRIBE_TYPE_STATE);
-
-        DESCRIBE_TYPES = describeTypes;
+    private static List<ClusterConfig> generator() {
+        return ConsumerGroupCommandTestUtils.generator();
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeNonExistingGroup(String quorum, String 
groupProtocol) {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTemplate("generator")
+    public void testDescribeNonExistingGroup(ClusterInstance clusterInstance) {
         String missingGroup = "missing.group";
 
         for (List<String> describeType : DESCRIBE_TYPES) {
             // note the group to be queried is a different (non-existing) group
-            List<String> cgcArgs = new 
ArrayList<>(Arrays.asList("--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", missingGroup));
+            List<String> cgcArgs = new 
ArrayList<>(Arrays.asList("--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup));
             cgcArgs.addAll(describeType);
-            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs.toArray(new String[0]));
-
-            String output = 
ToolsTestUtils.grabConsoleOutput(describeGroups(service));
-            assertTrue(output.contains("Consumer group '" + missingGroup + "' 
does not exist."),
-                "Expected error was not detected for describe option '" + 
String.join(" ", describeType) + "'");
+            try (ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(cgcArgs.toArray(new String[0]))) {
+                String output = 
ToolsTestUtils.grabConsoleOutput(describeGroups(service));
+                assertTrue(output.contains("Consumer group '" + missingGroup + 
"' does not exist."),
+                        "Expected error was not detected for describe option 
'" + String.join(" ", describeType) + "'");
+            }
         }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDescribeWithMultipleSubActions(String quorum) {
-        AtomicInteger exitStatus = new AtomicInteger(0);
-        AtomicReference<String> exitMessage = new AtomicReference<>("");
-        Exit.setExitProcedure((status, err) -> {
-            exitStatus.set(status);
-            exitMessage.set(err);
-            throw new RuntimeException();
-        });
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP, "--members", 
"--state"};
-        try {
-            assertThrows(RuntimeException.class, () -> 
ConsumerGroupCommand.main(cgcArgs));
-        } finally {
-            Exit.resetExitProcedure();
+    @ClusterTemplate("generator")
+    public void testDescribeOffsetsOfNonExistingGroup(ClusterInstance 
clusterInstance) throws Exception {
+        this.clusterInstance = clusterInstance;
+        String missingGroup = "missing.group";
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic);
+
+            // run one consumer in the group consuming from a single-partition 
topic
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap());
+                 // note the group to be queried is a different (non-existing) 
group
+                 ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup})
+            ) {
+                Entry<Optional<ConsumerGroupState>, 
Optional<Collection<PartitionAssignmentState>>> res = 
service.collectGroupOffsets(missingGroup);
+                assertTrue(res.getKey().map(s -> 
s.equals(ConsumerGroupState.DEAD)).orElse(false) && 
res.getValue().map(Collection::isEmpty).orElse(false),
+                        "Expected the state to be 'Dead', with no members in 
the group '" + missingGroup + "'.");
+            }
         }
-        assertEquals(1, exitStatus.get());
-        assertTrue(exitMessage.get().contains("Option [describe] takes at most 
one of these options"));
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDescribeWithStateValue(String quorum) {
-        AtomicInteger exitStatus = new AtomicInteger(0);
-        AtomicReference<String> exitMessage = new AtomicReference<>("");
-        Exit.setExitProcedure((status, err) -> {
-            exitStatus.set(status);
-            exitMessage.set(err);
-            throw new RuntimeException();
-        });
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--all-groups", "--state", 
"Stable"};
-        try {
-            assertThrows(RuntimeException.class, () -> 
ConsumerGroupCommand.main(cgcArgs));
-        } finally {
-            Exit.resetExitProcedure();
+    @ClusterTemplate("generator")
+    public void testDescribeMembersOfNonExistingGroup(ClusterInstance 
clusterInstance) throws Exception {
+        this.clusterInstance = clusterInstance;
+        String missingGroup = "missing.group";
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic);
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap());
+                 // note the group to be queried is a different (non-existing) 
group
+                 ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup})
+            ) {
+                Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res = 
service.collectGroupMembers(missingGroup, false);
+                assertTrue(res.getKey().map(s -> 
s.equals(ConsumerGroupState.DEAD)).orElse(false) && 
res.getValue().map(Collection::isEmpty).orElse(false),
+                        "Expected the state to be 'Dead', with no members in 
the group '" + missingGroup + "'.");
+
+                Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res2 = 
service.collectGroupMembers(missingGroup, true);
+                assertTrue(res2.getKey().map(s -> 
s.equals(ConsumerGroupState.DEAD)).orElse(false) && 
res2.getValue().map(Collection::isEmpty).orElse(false),
+                        "Expected the state to be 'Dead', with no members in 
the group '" + missingGroup + "' (verbose option).");
+            }
         }
-        assertEquals(1, exitStatus.get());
-        assertTrue(exitMessage.get().contains("Option [describe] does not take 
a value for [state]"));
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testPrintVersion(String quorum) {
-        ToolsTestUtils.MockExitProcedure exitProcedure = new 
ToolsTestUtils.MockExitProcedure();
-        Exit.setExitProcedure(exitProcedure);
-        try {
-            String out = ToolsTestUtils.captureStandardOut(() -> 
ConsumerGroupCommandOptions.fromArgs(new String[]{"--version"}));
-            assertEquals(0, exitProcedure.statusCode());
-            assertEquals(AppInfoParser.getVersion(), out);
-        } finally {
-            Exit.resetExitProcedure();
+    @ClusterTemplate("generator")
+    public void testDescribeStateOfNonExistingGroup(ClusterInstance 
clusterInstance) throws Exception {
+        this.clusterInstance = clusterInstance;
+        String missingGroup = "missing.group";
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic);
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap());
+                 // note the group to be queried is a different (non-existing) 
group
+                 ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup})
+            ) {
+                GroupState state = service.collectGroupState(missingGroup);
+                assertTrue(Objects.equals(state.state, 
ConsumerGroupState.DEAD) && state.numMembers == 0 &&
+                                state.coordinator != null && 
clusterInstance.brokerIds().contains(state.coordinator.id()),
+                        "Expected the state to be 'Dead', with no members in 
the group '" + missingGroup + "'."
+                );
+            }
         }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeOffsetsOfNonExistingGroup(String quorum, String 
groupProtocol) throws Exception {
-        String group = "missing.group";
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group consuming from a single-partition 
topic
-        addConsumerGroupExecutor(1, groupProtocol);
-        // note the group to be queried is a different (non-existing) group
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", group};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        Entry<Optional<ConsumerGroupState>, 
Optional<Collection<PartitionAssignmentState>>> res = 
service.collectGroupOffsets(group);
-        assertTrue(res.getKey().map(s -> 
s.equals(ConsumerGroupState.DEAD)).orElse(false) && 
res.getValue().map(Collection::isEmpty).orElse(false),
-            "Expected the state to be 'Dead', with no members in the group '" 
+ group + "'.");
+    @ClusterTemplate("generator")
+    public void testDescribeExistingGroup(ClusterInstance clusterInstance) 
throws Exception {
+        this.clusterInstance = clusterInstance;
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            createTopic(topic);
+            for (List<String> describeType : DESCRIBE_TYPES) {
+                String protocolGroup = GROUP_PREFIX + groupProtocol.name() + 
"." + String.join("", describeType);
+                try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, protocolGroup, topic, 
Collections.emptyMap());
+                     ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", protocolGroup})
+                ) {
+                    TestUtils.waitForCondition(() -> {
+                        Entry<String, String> res = 
ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
+                        return res.getKey().trim().split("\n").length == 2 && 
res.getValue().isEmpty();
+                    }, "Expected a data row and no error in describe results 
with describe type " + String.join(" ", describeType) + ".");
+                }
+            }
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeMembersOfNonExistingGroup(String quorum, String 
groupProtocol) throws Exception {
-        String group = "missing.group";
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTemplate("generator")
+    public void testDescribeExistingGroups(ClusterInstance clusterInstance) 
throws Exception {
+        this.clusterInstance = clusterInstance;
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            createTopic(topic);
 
-        // run one consumer in the group consuming from a single-partition 
topic
-        addConsumerGroupExecutor(1, groupProtocol);
-        // note the group to be queried is a different (non-existing) group
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", group};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res = 
service.collectGroupMembers(group, false);
-        assertTrue(res.getKey().map(s -> 
s.equals(ConsumerGroupState.DEAD)).orElse(false) && 
res.getValue().map(Collection::isEmpty).orElse(false),
-            "Expected the state to be 'Dead', with no members in the group '" 
+ group + "'.");
-
-        Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res2 = 
service.collectGroupMembers(group, true);
-        assertTrue(res2.getKey().map(s -> 
s.equals(ConsumerGroupState.DEAD)).orElse(false) && 
res2.getValue().map(Collection::isEmpty).orElse(false),
-            "Expected the state to be 'Dead', with no members in the group '" 
+ group + "' (verbose option).");
+            // Create N single-threaded consumer groups from a 
single-partition topic
+            List<AutoCloseable> protocolConsumerGroupExecutors = new 
ArrayList<>();
+            try {
+                List<String> groups = new ArrayList<>();
+                for (List<String> describeType : DESCRIBE_TYPES) {
+                    String group = GROUP_PREFIX + groupProtocol.name() + "." + 
String.join("", describeType);
+                    groups.addAll(Arrays.asList("--group", group));
+                    
protocolConsumerGroupExecutors.add(consumerGroupClosable(groupProtocol, group, 
topic, Collections.emptyMap()));
+                }
+
+                int expectedNumLines = DESCRIBE_TYPES.size() * 2;
+
+                for (List<String> describeType : DESCRIBE_TYPES) {
+                    List<String> cgcArgs = new 
ArrayList<>(Arrays.asList("--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe"));
+                    cgcArgs.addAll(groups);
+                    cgcArgs.addAll(describeType);
+                    try (ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(cgcArgs.toArray(new String[0]))) {
+                        TestUtils.waitForCondition(() -> {
+                            Entry<String, String> res = 
ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
+                            long numLines = 
Arrays.stream(res.getKey().trim().split("\n")).filter(line -> 
!line.isEmpty()).count();
+                            return (numLines == expectedNumLines) && 
res.getValue().isEmpty();
+                        }, "Expected a data row and no error in describe 
results with describe type " + String.join(" ", describeType) + ".");
+                    }
+                }
+            } finally {
+                for (AutoCloseable protocolConsumerGroupExecutor : 
protocolConsumerGroupExecutors) {
+                    protocolConsumerGroupExecutor.close();
+                }
+            }
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeStateOfNonExistingGroup(String quorum, String 
groupProtocol) throws Exception {
-        String group = "missing.group";
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group consuming from a single-partition 
topic
-        addConsumerGroupExecutor(1, groupProtocol);
-        // note the group to be queried is a different (non-existing) group
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", group};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+    @ClusterTemplate("generator")
+    public void testDescribeAllExistingGroups(ClusterInstance clusterInstance) 
throws Exception {
+        this.clusterInstance = clusterInstance;
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            createTopic(topic);
 
-        GroupState state = service.collectGroupState(group);
-        assertTrue(Objects.equals(state.state, ConsumerGroupState.DEAD) && 
state.numMembers == 0 &&
-                state.coordinator != null && !brokers().filter(s -> 
s.config().brokerId() == state.coordinator.id()).isEmpty(),
-            "Expected the state to be 'Dead', with no members in the group '" 
+ group + "'."
-        );
+            // Create N single-threaded consumer groups from a 
single-partition topic
+            List<AutoCloseable> protocolConsumerGroupExecutors = new 
ArrayList<>();
+            List<String> groups = new ArrayList<>();
+            try {
+                for (List<String> describeType : DESCRIBE_TYPES) {
+                    String group = GROUP_PREFIX + groupProtocol.name() + "." + 
String.join("", describeType);
+                    groups.add(group);
+                    
protocolConsumerGroupExecutors.add(consumerGroupClosable(groupProtocol, group, 
topic, Collections.emptyMap()));
+                }
+                int expectedNumLines = DESCRIBE_TYPES.size() * 2;
+                for (List<String> describeType : DESCRIBE_TYPES) {
+                    try (ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--all-groups"})) {
+                        TestUtils.waitForCondition(() -> {
+                            Entry<String, String> res = 
ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
+                            long numLines = 
Arrays.stream(res.getKey().trim().split("\n")).filter(line -> 
!line.isEmpty()).count();
+                            return (numLines == expectedNumLines) && 
res.getValue().isEmpty();
+                        }, "Expected a data row and no error in describe 
results with describe type " + String.join(" ", describeType) + ".");
+                    }
+                }
+            } finally {
+                for (AutoCloseable protocolConsumerGroupExecutor : 
protocolConsumerGroupExecutors) {
+                    protocolConsumerGroupExecutor.close();
+                }
+                // remove previous consumer groups, so we can have a clean 
cluster for next consumer group protocol test.
+                deleteConsumerGroups(groups);
+                deleteTopic(topic);
+            }
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeExistingGroup(String quorum, String groupProtocol) 
throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTemplate("generator")
+    public void testDescribeOffsetsOfExistingGroup(ClusterInstance 
clusterInstance) throws Exception {
+        this.clusterInstance = clusterInstance;
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic);
 
-        for (List<String> describeType : DESCRIBE_TYPES) {
-            String group = GROUP + String.join("", describeType);
             // run one consumer in the group consuming from a single-partition 
topic
-            addConsumerGroupExecutor(1, TOPIC, group, groupProtocol);
-            List<String> cgcArgs = new 
ArrayList<>(Arrays.asList("--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", group));
-            cgcArgs.addAll(describeType);
-            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs.toArray(new String[0]));
-
-            TestUtils.waitForCondition(() -> {
-                Entry<String, String> res = 
ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
-                return res.getKey().trim().split("\n").length == 2 && 
res.getValue().isEmpty();
-            }, "Expected a data row and no error in describe results with 
describe type " + String.join(" ", describeType) + ".");
-        }
-    }
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap());
+                 ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group})
+            ) {
+                TestUtils.waitForCondition(() -> {
+                    Entry<Optional<ConsumerGroupState>, 
Optional<Collection<PartitionAssignmentState>>> groupOffsets = 
service.collectGroupOffsets(group);
+                    Optional<ConsumerGroupState> state = groupOffsets.getKey();
+                    Optional<Collection<PartitionAssignmentState>> assignments 
= groupOffsets.getValue();
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeExistingGroups(String quorum, String 
groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+                    Predicate<PartitionAssignmentState> isGrp = s -> 
Objects.equals(s.group, group);
 
-        // Create N single-threaded consumer groups from a single-partition 
topic
-        List<String> groups = new ArrayList<>();
+                    boolean res = state.map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false) &&
+                            assignments.isPresent() &&
+                            assignments.get().stream().filter(isGrp).count() 
== 1;
 
-        for (List<String> describeType : DESCRIBE_TYPES) {
-            String group = GROUP + String.join("", describeType);
-            addConsumerGroupExecutor(1, TOPIC, group, groupProtocol);
-            groups.addAll(Arrays.asList("--group", group));
-        }
+                    if (!res)
+                        return false;
 
-        int expectedNumLines = DESCRIBE_TYPES.size() * 2;
+                    Optional<PartitionAssignmentState> maybePartitionState = 
assignments.get().stream().filter(isGrp).findFirst();
+                    if (!maybePartitionState.isPresent())
+                        return false;
 
-        for (List<String> describeType : DESCRIBE_TYPES) {
-            List<String> cgcArgs = new 
ArrayList<>(Arrays.asList("--bootstrap-server", 
bootstrapServers(listenerName()), "--describe"));
-            cgcArgs.addAll(groups);
-            cgcArgs.addAll(describeType);
-            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs.toArray(new String[0]));
+                    PartitionAssignmentState partitionState = 
maybePartitionState.get();
 
-            TestUtils.waitForCondition(() -> {
-                Entry<String, String> res = 
ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
-                long numLines = 
Arrays.stream(res.getKey().trim().split("\n")).filter(line -> 
!line.isEmpty()).count();
-                return (numLines == expectedNumLines) && 
res.getValue().isEmpty();
-            }, "Expected a data row and no error in describe results with 
describe type " + String.join(" ", describeType) + ".");
+                    return !partitionState.consumerId.map(s0 -> 
s0.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) &&
+                            !partitionState.clientId.map(s0 -> 
s0.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) &&
+                            !partitionState.host.map(h -> 
h.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false);
+                }, "Expected a 'Stable' group status, rows and valid values 
for consumer id / client id / host columns in describe results for group " + 
group + ".");
+            }
         }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeAllExistingGroups(String quorum, String 
groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTemplate("generator")
+    public void testDescribeMembersOfExistingGroup(ClusterInstance 
clusterInstance) throws Exception {
+        this.clusterInstance = clusterInstance;
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic);
 
-        // Create N single-threaded consumer groups from a single-partition 
topic
-        for (List<String> describeType : DESCRIBE_TYPES) {
-            String group = GROUP + String.join("", describeType);
-            addConsumerGroupExecutor(1, TOPIC, group, groupProtocol);
+            // run one consumer in the group consuming from a single-partition 
topic
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap());
+                 ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group});
+                 Admin admin = 
Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
 clusterInstance.bootstrapServers()))
+            ) {
+                TestUtils.waitForCondition(() -> {
+                    ConsumerGroupDescription consumerGroupDescription = 
admin.describeConsumerGroups(Collections.singleton(group)).describedGroups().get(group).get();
+                    return consumerGroupDescription.members().size() == 1 && 
consumerGroupDescription.members().iterator().next().assignment().topicPartitions().size()
 == 1;
+                }, "Expected a 'Stable' group status, rows and valid member 
information for group " + group + ".");
+
+                Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res = 
service.collectGroupMembers(group, true);
+
+                assertTrue(res.getValue().isPresent());
+                assertTrue(res.getValue().get().size() == 1 && 
res.getValue().get().iterator().next().assignment.size() == 1,
+                        "Expected a topic partition assigned to the single 
group member for group " + group);
+            }
         }
+    }
 
-        int expectedNumLines = DESCRIBE_TYPES.size() * 2;
-
-        for (List<String> describeType : DESCRIBE_TYPES) {
-            List<String> cgcArgs = new 
ArrayList<>(Arrays.asList("--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--all-groups"));
-            cgcArgs.addAll(describeType);
-            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs.toArray(new String[0]));
+    @ClusterTemplate("generator")
+    public void testDescribeStateOfExistingGroup(ClusterInstance 
clusterInstance) throws Exception {
+        this.clusterInstance = clusterInstance;
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic);
 
-            TestUtils.waitForCondition(() -> {
-                Entry<String, String> res = 
ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
-                long numLines = 
Arrays.stream(res.getKey().trim().split("\n")).filter(s -> 
!s.isEmpty()).count();
-                return (numLines == expectedNumLines) && 
res.getValue().isEmpty();
-            }, "Expected a data row and no error in describe results with 
describe type " + String.join(" ", describeType) + ".");
+            // run one consumer in the group consuming from a single-partition 
topic
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, 
Collections.singletonMap(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "range"));
+                 ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group})
+            ) {
+                TestUtils.waitForCondition(() -> {
+                    GroupState state = service.collectGroupState(group);
+                    return Objects.equals(state.state, 
ConsumerGroupState.STABLE) &&
+                            state.numMembers == 1 &&
+                            state.coordinator != null &&
+                            
clusterInstance.brokerIds().contains(state.coordinator.id());
+                }, "Expected a 'Stable' group status, with one member for 
group " + group + ".");
+            }
         }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeOffsetsOfExistingGroup(String quorum, String 
groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group consuming from a single-partition 
topic
-        addConsumerGroupExecutor(1, groupProtocol);
-
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(() -> {
-            Entry<Optional<ConsumerGroupState>, 
Optional<Collection<PartitionAssignmentState>>> groupOffsets = 
service.collectGroupOffsets(GROUP);
-            Optional<ConsumerGroupState> state = groupOffsets.getKey();
-            Optional<Collection<PartitionAssignmentState>> assignments = 
groupOffsets.getValue();
-
-            Predicate<PartitionAssignmentState> isGrp = s -> 
Objects.equals(s.group, GROUP);
-
-            boolean res = state.map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false) &&
-                assignments.isPresent() &&
-                assignments.get().stream().filter(isGrp).count() == 1;
-
-            if (!res)
-                return false;
+    @ClusterTemplate("generator")
+    public void 
testDescribeStateOfExistingGroupWithNonDefaultAssignor(ClusterInstance 
clusterInstance) throws Exception {
+        this.clusterInstance = clusterInstance;
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic);
 
-            Optional<PartitionAssignmentState> maybePartitionState = 
assignments.get().stream().filter(isGrp).findFirst();
-            if (!maybePartitionState.isPresent())
-                return false;
-
-            PartitionAssignmentState partitionState = 
maybePartitionState.get();
-
-            return !partitionState.consumerId.map(s0 -> 
s0.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) &&
-                !partitionState.clientId.map(s0 -> 
s0.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) &&
-                !partitionState.host.map(h -> 
h.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false);
-        }, "Expected a 'Stable' group status, rows and valid values for 
consumer id / client id / host columns in describe results for group " + GROUP 
+ ".");
+            // run one consumer in the group consuming from a single-partition 
topic
+            AutoCloseable protocolConsumerGroupExecutor = null;
+            try {
+                String expectedName;
+                if (groupProtocol.equals(GroupProtocol.CONSUMER)) {
+                    protocolConsumerGroupExecutor = 
consumerGroupClosable(GroupProtocol.CONSUMER, group, topic, 
Collections.singletonMap(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "range"));
+                    expectedName = RangeAssignor.RANGE_ASSIGNOR_NAME;
+                } else {
+                    protocolConsumerGroupExecutor = 
consumerGroupClosable(GroupProtocol.CLASSIC, group, topic, 
Collections.singletonMap(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, 
RoundRobinAssignor.class.getName()));
+                    expectedName = RoundRobinAssignor.ROUNDROBIN_ASSIGNOR_NAME;
+                }
+
+                try (ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group})) {
+                    TestUtils.waitForCondition(() -> {
+                        GroupState state = service.collectGroupState(group);
+                        return Objects.equals(state.state, 
ConsumerGroupState.STABLE) &&
+                                state.numMembers == 1 &&
+                                Objects.equals(state.assignmentStrategy, 
expectedName) &&
+                                state.coordinator != null &&
+                                
clusterInstance.brokerIds().contains(state.coordinator.id());
+                    }, "Expected a 'Stable' group status, with one member and 
" + expectedName + " assignment strategy for group " + group + ".");
+                }
+            } finally {
+                if (protocolConsumerGroupExecutor != null) {
+                    protocolConsumerGroupExecutor.close();
+                }
+            }
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeMembersOfExistingGroup(String quorum, String 
groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group consuming from a single-partition 
topic
-        addConsumerGroupExecutor(1, groupProtocol);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(() -> {
-            Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> groupMembers = 
service.collectGroupMembers(GROUP, false);
-            Optional<ConsumerGroupState> state = groupMembers.getKey();
-            Optional<Collection<MemberAssignmentState>> assignments = 
groupMembers.getValue();
-
-            Predicate<MemberAssignmentState> isGrp = s -> 
Objects.equals(s.group, GROUP);
-
-            boolean res = state.map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false) &&
-                assignments.isPresent() &&
-                assignments.get().stream().filter(s -> Objects.equals(s.group, 
GROUP)).count() == 1;
-
-            if (!res)
-                return false;
-
-            Optional<MemberAssignmentState> maybeAssignmentState = 
assignments.get().stream().filter(isGrp).findFirst();
-            if (!maybeAssignmentState.isPresent())
-                return false;
-
-            MemberAssignmentState assignmentState = maybeAssignmentState.get();
-
-            return !Objects.equals(assignmentState.consumerId, 
ConsumerGroupCommand.MISSING_COLUMN_VALUE) &&
-                !Objects.equals(assignmentState.clientId, 
ConsumerGroupCommand.MISSING_COLUMN_VALUE) &&
-                !Objects.equals(assignmentState.host, 
ConsumerGroupCommand.MISSING_COLUMN_VALUE);
-        }, "Expected a 'Stable' group status, rows and valid member 
information for group " + GROUP + ".");
-
-        Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res = 
service.collectGroupMembers(GROUP, true);
-
-        if (res.getValue().isPresent()) {
-            assertTrue(res.getValue().get().size() == 1 && 
res.getValue().get().iterator().next().assignment.size() == 1,
-                "Expected a topic partition assigned to the single group 
member for group " + GROUP);
-        } else {
-            fail("Expected partition assignments for members of group " + 
GROUP);
+    @ClusterTemplate("generator")
+    public void testDescribeExistingGroupWithNoMembers(ClusterInstance 
clusterInstance) throws Exception {
+        this.clusterInstance = clusterInstance;
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            createTopic(topic);
+
+            for (List<String> describeType : DESCRIBE_TYPES) {
+                String group = GROUP_PREFIX + groupProtocol.name() + 
String.join("", describeType);
+                // run one consumer in the group consuming from a 
single-partition topic
+                try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap());
+                     ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group})
+                ) {
+                    TestUtils.waitForCondition(() -> {
+                        Entry<String, String> res = 
ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
+                        return res.getKey().trim().split("\n").length == 2 && 
res.getValue().isEmpty();
+                    }, "Expected describe group results with one data row for 
describe type '" + String.join(" ", describeType) + "'");
+
+                    Entry<Optional<ConsumerGroupState>, 
Optional<Collection<PartitionAssignmentState>>> res = 
service.collectGroupOffsets(group);

Review Comment:
   Removed it. 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-unsubscr...@kafka.apache.org

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


Reply via email to