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


##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java:
##########
@@ -16,833 +16,962 @@
  */
 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.admin.ConsumerGroupDescription;
 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.TopicPartition;
+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.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
 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 final ClusterInstance clusterInstance;
 
-    static {
-        List<List<String>> describeTypes = new ArrayList<>();
-
-        describeTypes.addAll(DESCRIBE_TYPE_OFFSETS);
-        describeTypes.addAll(DESCRIBE_TYPE_MEMBERS);
-        describeTypes.addAll(DESCRIBE_TYPE_STATE);
+    DescribeConsumerGroupTest(ClusterInstance clusterInstance) {
+        this.clusterInstance = clusterInstance;
+    }
 
-        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() {
         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() throws Exception {
+        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() throws Exception {
+        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() throws Exception {
+        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() throws Exception {
+        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() throws Exception {
+        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() throws Exception {
+        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() throws Exception {
+        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() throws Exception {
+        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() throws Exception {
+        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() 
throws Exception {
+        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() throws Exception {
+        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);
+                    protocolConsumerGroupExecutor.close();
+                    TestUtils.waitForCondition(
+                            () -> 
ToolsTestUtils.grabConsoleError(describeGroups(service)).contains("Consumer 
group '" + group + "' has no active members."),
+                            "Expected no active member in describe group 
results with describe type " + String.join(" ", describeType));
+                }
+            }
         }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeStateOfExistingGroup(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,
-            // This is only effective when new protocol is used.
-            Optional.of("range")
-        );
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(() -> {
-            GroupState state = service.collectGroupState(GROUP);
-            return Objects.equals(state.state, ConsumerGroupState.STABLE) &&
-                state.numMembers == 1 &&
-                Objects.equals(state.assignmentStrategy, "range") &&
-                state.coordinator != null &&
-                brokers().count(s -> s.config().brokerId() == 
state.coordinator.id()) > 0;
-        }, "Expected a 'Stable' group status, with one member and round robin 
assignment strategy for group " + GROUP + ".");
-    }
-
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeStateOfExistingGroupWithNonDefaultAssignor(String 
quorum, String groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group consuming from a single-partition 
topic
-        String expectedName;
-        if (groupProtocol.equals("consumer")) {
-            addConsumerGroupExecutor(1, groupProtocol, Optional.of("range"));
-            expectedName = "range";
-        } else {
-            addConsumerGroupExecutor(1, TOPIC, GROUP, 
RoundRobinAssignor.class.getName(), Optional.empty(), Optional.empty(), false, 
groupProtocol);
-            expectedName = "roundrobin";
-        }
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        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 &&
-                brokers().count(s -> s.config().brokerId() == 
state.coordinator.id()) > 0;
-        }, "Expected a 'Stable' group status, with one member and " + 
expectedName + " assignment strategy for group " + GROUP + ".");
-    }
-
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeExistingGroupWithNoMembers(String quorum, String 
groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTemplate("generator")
+    public void testDescribeOffsetsOfExistingGroupWithNoMembers() throws 
Exception {
+        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
-            ConsumerGroupExecutor executor = 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 describe group results with one data row for describe 
type '" + String.join(" ", describeType) + "'");
-
-            // stop the consumer so the group has no active member anymore
-            executor.shutdown();
-            TestUtils.waitForCondition(
-                () -> 
ToolsTestUtils.grabConsoleError(describeGroups(service)).contains("Consumer 
group '" + group + "' has no active members."),
-                "Expected no active member in describe group 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>>> res = 
service.collectGroupOffsets(group);
+                    return res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false)
+                            && res.getValue().map(c -> 
c.stream().anyMatch(assignment -> Objects.equals(assignment.group, group) && 
assignment.offset.isPresent())).orElse(false);
+                }, "Expected the group to initially become stable, and to find 
group in assignments after initial offset commit.");
+
+                // stop the consumer so the group has no active member anymore
+                protocolConsumerGroupExecutor.close();
+
+                TestUtils.waitForCondition(() -> {
+                    Entry<Optional<ConsumerGroupState>, 
Optional<Collection<PartitionAssignmentState>>> offsets = 
service.collectGroupOffsets(group);
+                    Optional<ConsumerGroupState> state = offsets.getKey();
+                    Optional<Collection<PartitionAssignmentState>> assignments 
= offsets.getValue();
+                    List<PartitionAssignmentState> testGroupAssignments = 
assignments.get().stream().filter(a -> Objects.equals(a.group, 
group)).collect(Collectors.toList());
+                    PartitionAssignmentState assignment = 
testGroupAssignments.get(0);
+                    return state.map(s -> 
s.equals(ConsumerGroupState.EMPTY)).orElse(false) &&
+                            testGroupAssignments.size() == 1 &&
+                            assignment.consumerId.map(c -> 
c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) && // 
the member should be gone
+                            assignment.clientId.map(c -> 
c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) &&
+                            assignment.host.map(c -> 
c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false);
+                }, "failed to collect group offsets");
+            }
         }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeOffsetsOfExistingGroupWithNoMembers(String quorum, 
String groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group consuming from a single-partition 
topic
-        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, TOPIC, 
GROUP, RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), true, 
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>>> res = 
service.collectGroupOffsets(GROUP);
-            return res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false)
-                && res.getValue().map(c -> c.stream().anyMatch(assignment -> 
Objects.equals(assignment.group, GROUP) && 
assignment.offset.isPresent())).orElse(false);
-        }, "Expected the group to initially become stable, and to find group 
in assignments after initial offset commit.");
+    @ClusterTemplate("generator")
+    public void testDescribeMembersOfExistingGroupWithNoMembers() throws 
Exception {
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic);
 
-        // stop the consumer so the group has no active member anymore
-        executor.shutdown();
-
-        TestUtils.waitForCondition(() -> {
-            Entry<Optional<ConsumerGroupState>, 
Optional<Collection<PartitionAssignmentState>>> offsets = 
service.collectGroupOffsets(GROUP);
-            Optional<ConsumerGroupState> state = offsets.getKey();
-            Optional<Collection<PartitionAssignmentState>> assignments = 
offsets.getValue();
-            List<PartitionAssignmentState> testGroupAssignments = 
assignments.get().stream().filter(a -> Objects.equals(a.group, 
GROUP)).collect(Collectors.toList());
-            PartitionAssignmentState assignment = testGroupAssignments.get(0);
-            return state.map(s -> 
s.equals(ConsumerGroupState.EMPTY)).orElse(false) &&
-                testGroupAssignments.size() == 1 &&
-                assignment.consumerId.map(c -> 
c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) && // 
the member should be gone
-                assignment.clientId.map(c -> 
c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) &&
-                assignment.host.map(c -> 
c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false);
-        }, "failed to collect group offsets");
+            // 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<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res = 
service.collectGroupMembers(group, false);
+                    return res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false)
+                            && res.getValue().map(c -> c.stream().anyMatch(m 
-> Objects.equals(m.group, group))).orElse(false);
+                }, "Expected the group to initially become stable, and to find 
group in assignments after initial offset commit.");
+
+                // stop the consumer so the group has no active member anymore
+                protocolConsumerGroupExecutor.close();
+
+                TestUtils.waitForCondition(() -> {
+                    Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res = 
service.collectGroupMembers(group, false);
+                    return res.getKey().map(s -> 
s.equals(ConsumerGroupState.EMPTY)).orElse(false) && res.getValue().isPresent() 
&& res.getValue().get().isEmpty();
+                }, "Expected no member in describe group members results for 
group '" + group + "'");
+            }
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeMembersOfExistingGroupWithNoMembers(String quorum, 
String groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTemplate("generator")
+    public void testDescribeStateOfExistingGroupWithNoMembers() throws 
Exception {
+        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
-        ConsumerGroupExecutor executor = 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>>> res = 
service.collectGroupMembers(GROUP, false);
-            return res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false)
-                && res.getValue().map(c -> c.stream().anyMatch(m -> 
Objects.equals(m.group, GROUP))).orElse(false);
-        }, "Expected the group to initially become stable, and to find group 
in assignments after initial offset commit.");
-
-        // stop the consumer so the group has no active member anymore
-        executor.shutdown();
-
-        TestUtils.waitForCondition(() -> {
-            Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res = 
service.collectGroupMembers(GROUP, false);
-            return res.getKey().map(s -> 
s.equals(ConsumerGroupState.EMPTY)).orElse(false) && res.getValue().isPresent() 
&& res.getValue().get().isEmpty();
-        }, "Expected no member in describe group members results for group '" 
+ GROUP + "'");
+            // 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(() -> {
+                    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 the group to initially become stable, and have a 
single member.");
+
+                // stop the consumer so the group has no active member anymore
+                protocolConsumerGroupExecutor.close();
+
+                TestUtils.waitForCondition(() -> {
+                    GroupState state = service.collectGroupState(group);
+                    return Objects.equals(state.state, 
ConsumerGroupState.EMPTY) && state.numMembers == 0;
+                }, "Expected the group to become empty after the only member 
leaving.");
+            }
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeStateOfExistingGroupWithNoMembers(String quorum, 
String groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group consuming from a single-partition 
topic
-        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1, 
groupProtocol);
-
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(() -> {
-            GroupState state = service.collectGroupState(GROUP);
-            return Objects.equals(state.state, ConsumerGroupState.STABLE) &&
-                state.numMembers == 1 &&
-                state.coordinator != null &&
-                brokers().count(s -> s.config().brokerId() == 
state.coordinator.id()) > 0;
-        }, "Expected the group '" + GROUP + "' to initially become stable, and 
have a single member.");
-
-        // stop the consumer so the group has no active member anymore
-        executor.shutdown();
-
-        TestUtils.waitForCondition(() -> {
-            GroupState state = service.collectGroupState(GROUP);
-            return Objects.equals(state.state, ConsumerGroupState.EMPTY) && 
state.numMembers == 0;
-        }, "Expected the group '" + GROUP + "' to become empty after the only 
member leaving.");
+    @ClusterTemplate("generator")
+    public void testDescribeWithConsumersWithoutAssignedPartitions() throws 
Exception {
+        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);
+                List<String> cgcArgs = new 
ArrayList<>(Arrays.asList("--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group));
+                cgcArgs.addAll(describeType);
+                // run two consumers in the group consuming from a 
single-partition topic
+                try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2);
+                     ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(cgcArgs.toArray(new String[0]));
+                ) {
+                    TestUtils.waitForCondition(() -> {
+                        Entry<String, String> res = 
ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
+                        int expectedNumRows = 
DESCRIBE_TYPE_MEMBERS.contains(describeType) ? 3 : 2;
+                        return res.getValue().isEmpty() && 
res.getKey().trim().split("\n").length == expectedNumRows;
+                    }, "Expected a single data row in describe group result 
with describe type '" + String.join(" ", describeType) + "'");
+                }
+            }
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeWithConsumersWithoutAssignedPartitions(String 
quorum, String groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTemplate("generator")
+    public void testDescribeOffsetsWithConsumersWithoutAssignedPartitions() 
throws Exception {
+        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 two consumers in the group consuming from a 
single-partition topic
-            addConsumerGroupExecutor(2, 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));
-                int expectedNumRows = 
DESCRIBE_TYPE_MEMBERS.contains(describeType) ? 3 : 2;
-                return res.getValue().isEmpty() && 
res.getKey().trim().split("\n").length == expectedNumRows;
-            }, "Expected a single data row in describe group result with 
describe type '" + String.join(" ", describeType) + "'");
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2);
+                 ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group});
+            ) {
+                TestUtils.waitForCondition(() -> {
+                    Entry<Optional<ConsumerGroupState>, 
Optional<Collection<PartitionAssignmentState>>> res = 
service.collectGroupOffsets(group);
+                    return res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).isPresent() &&
+                            res.getValue().isPresent() &&
+                            res.getValue().get().stream().filter(s -> 
Objects.equals(s.group, group)).count() == 1 &&
+                            res.getValue().get().stream().filter(x -> 
Objects.equals(x.group, group) && x.partition.isPresent()).count() == 1;
+                }, "Expected rows for consumers with no assigned partitions in 
describe group results");
+            }
         }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void 
testDescribeOffsetsWithConsumersWithoutAssignedPartitions(String quorum, String 
groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run two consumers in the group consuming from a single-partition 
topic
-        addConsumerGroupExecutor(2, groupProtocol);
+    @ClusterTemplate("generator")
+    public void testDescribeMembersWithConsumersWithoutAssignedPartitions() 
throws Exception {
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic);
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(() -> {
-            Entry<Optional<ConsumerGroupState>, 
Optional<Collection<PartitionAssignmentState>>> res = 
service.collectGroupOffsets(GROUP);
-            return res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).isPresent() &&
-                res.getValue().isPresent() &&
-                res.getValue().get().stream().filter(s -> 
Objects.equals(s.group, GROUP)).count() == 1 &&
-                res.getValue().get().stream().filter(x -> 
Objects.equals(x.group, GROUP) && x.partition.isPresent()).count() == 1;
-        }, "Expected rows for consumers with no assigned partitions in 
describe group results");
+            // run two consumers in the group consuming from a 
single-partition topic
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2);
+                 ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group});
+            ) {
+                TestUtils.waitForCondition(() -> {
+                    Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res = 
service.collectGroupMembers(group, false);
+                    return res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false) &&
+                            res.getValue().isPresent() &&
+                            res.getValue().get().stream().filter(s -> 
Objects.equals(s.group, group)).count() == 2 &&
+                            res.getValue().get().stream().filter(x -> 
Objects.equals(x.group, group) && x.numPartitions == 1).count() == 1 &&
+                            res.getValue().get().stream().filter(x -> 
Objects.equals(x.group, group) && x.numPartitions == 0).count() == 1 &&
+                            res.getValue().get().stream().allMatch(s -> 
s.assignment.isEmpty());
+                }, "Expected rows for consumers with no assigned partitions in 
describe group results");
+
+                Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res = 
service.collectGroupMembers(group, true);
+                assertTrue(res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false)
+                                && res.getValue().map(c -> 
c.stream().anyMatch(s -> !s.assignment.isEmpty())).orElse(false),
+                        "Expected additional columns in verbose version of 
describe members");
+            }
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void 
testDescribeMembersWithConsumersWithoutAssignedPartitions(String quorum, String 
groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTemplate("generator")
+    public void testDescribeStateWithConsumersWithoutAssignedPartitions() 
throws Exception {
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic);
 
-        // run two consumers in the group consuming from a single-partition 
topic
-        addConsumerGroupExecutor(2, 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>>> res = 
service.collectGroupMembers(GROUP, false);
-            return res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false) &&
-                res.getValue().isPresent() &&
-                res.getValue().get().stream().filter(s -> 
Objects.equals(s.group, GROUP)).count() == 2 &&
-                res.getValue().get().stream().filter(x -> 
Objects.equals(x.group, GROUP) && x.numPartitions == 1).count() == 1 &&
-                res.getValue().get().stream().filter(x -> 
Objects.equals(x.group, GROUP) && x.numPartitions == 0).count() == 1 &&
-                res.getValue().get().stream().allMatch(s -> 
s.assignment.isEmpty());
-        }, "Expected rows for consumers with no assigned partitions in 
describe group results");
+            // run two consumers in the group consuming from a 
single-partition topic
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2);
+                 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 == 2;
+                }, "Expected two consumers in describe group results");
+            }
+        }
+    }
 
-        Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res = 
service.collectGroupMembers(GROUP, true);
-        assertTrue(res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false)
-                && res.getValue().map(c -> c.stream().anyMatch(s -> 
!s.assignment.isEmpty())).orElse(false),
-            "Expected additional columns in verbose version of describe 
members");
+    @ClusterTemplate("generator")
+    public void testDescribeWithMultiPartitionTopicAndMultipleConsumers() 
throws Exception {
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            createTopic(topic, 2);
+
+            for (List<String> describeType : DESCRIBE_TYPES) {
+                String group = GROUP_PREFIX + groupProtocol.name() + 
String.join("", describeType);
+                List<String> cgcArgs = new 
ArrayList<>(Arrays.asList("--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group));
+                cgcArgs.addAll(describeType);
+                // run two consumers in the group consuming from a 
two-partition topic
+                try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2);
+                     ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(cgcArgs.toArray(new String[0]));
+                ) {
+                    TestUtils.waitForCondition(() -> {
+                        Entry<String, String> res = 
ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
+                        int expectedNumRows = 
DESCRIBE_TYPE_STATE.contains(describeType) ? 2 : 3;
+                        return res.getValue().isEmpty() && 
res.getKey().trim().split("\n").length == expectedNumRows;
+                    }, "Expected a single data row in describe group result 
with describe type '" + String.join(" ", describeType) + "'");
+                }
+            }
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeStateWithConsumersWithoutAssignedPartitions(String 
quorum, String groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTemplate("generator")
+    public void 
testDescribeOffsetsWithMultiPartitionTopicAndMultipleConsumers() throws 
Exception {
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic, 2);
 
-        // run two consumers in the group consuming from a single-partition 
topic
-        addConsumerGroupExecutor(2, groupProtocol);
+            // run two consumers in the group consuming from a two-partition 
topic
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2);
+                 ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group});
+            ) {
+                TestUtils.waitForCondition(() -> {
+                    Entry<Optional<ConsumerGroupState>, 
Optional<Collection<PartitionAssignmentState>>> res = 
service.collectGroupOffsets(group);
+                    return res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false) &&
+                            res.getValue().isPresent() &&
+                            res.getValue().get().stream().filter(s -> 
Objects.equals(s.group, group)).count() == 2 &&
+                            res.getValue().get().stream().filter(x -> 
Objects.equals(x.group, group) && x.partition.isPresent()).count() == 2 &&
+                            res.getValue().get().stream().noneMatch(x -> 
Objects.equals(x.group, group) && !x.partition.isPresent());
+                }, "Expected two rows (one row per consumer) in describe group 
results.");
+            }
+        }
+    }
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+    @ClusterTemplate("generator")
+    public void 
testDescribeMembersWithMultiPartitionTopicAndMultipleConsumers() throws 
Exception {
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic, 2);
 
-        TestUtils.waitForCondition(() -> {
-            GroupState state = service.collectGroupState(GROUP);
-            return Objects.equals(state.state, ConsumerGroupState.STABLE) && 
state.numMembers == 2;
-        }, "Expected two consumers in describe group results");
+            // run two consumers in the group consuming from a two-partition 
topic
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2);
+                 ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group});
+            ) {
+                TestUtils.waitForCondition(() -> {
+                    Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res = 
service.collectGroupMembers(group, false);
+                    return res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false) &&
+                            res.getValue().isPresent() &&
+                            res.getValue().get().stream().filter(s -> 
Objects.equals(s.group, group)).count() == 2 &&
+                            res.getValue().get().stream().filter(x -> 
Objects.equals(x.group, group) && x.numPartitions == 1).count() == 2 &&
+                            res.getValue().get().stream().noneMatch(x -> 
Objects.equals(x.group, group) && x.numPartitions == 0);
+                }, "Expected two rows (one row per consumer) in describe group 
members results.");
+
+                Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res = 
service.collectGroupMembers(group, true);
+                assertTrue(res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false) && res.getValue().map(s -> 
s.stream().filter(x -> x.assignment.isEmpty()).count()).orElse(0L) == 0,
+                        "Expected additional columns in verbose version of 
describe members");
+            }
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeWithMultiPartitionTopicAndMultipleConsumers(String 
quorum, String groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-        String topic2 = "foo2";
-        createTopic(topic2, 2, 1, new Properties(), listenerName(), new 
Properties());
+    @ClusterTemplate("generator")
+    public void testDescribeStateWithMultiPartitionTopicAndMultipleConsumers() 
throws Exception {
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic, 2);
 
-        for (List<String> describeType : DESCRIBE_TYPES) {
-            String group = GROUP + String.join("", describeType);
             // run two consumers in the group consuming from a two-partition 
topic
-            addConsumerGroupExecutor(2, topic2, 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]));
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2);
+                 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) && Objects.equals(state.group, group) && 
state.numMembers == 2;
+                }, "Expected a stable group with two members in describe group 
state result.");
+            }
+        }
+    }
 
-            TestUtils.waitForCondition(() -> {
-                Entry<String, String> res = 
ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service));
-                int expectedNumRows = 
DESCRIBE_TYPE_STATE.contains(describeType) ? 2 : 3;
-                return res.getValue().isEmpty() && 
res.getKey().trim().split("\n").length == expectedNumRows;
-            }, "Expected a single data row in describe group result with 
describe type '" + String.join(" ", describeType) + "'");
+    @ClusterTemplate("generator")
+    public void testDescribeSimpleConsumerGroup() throws Exception {
+        // Ensure that the offsets of consumers which don't use group 
management are still displayed
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic, 2);
+
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(GroupProtocol.CLASSIC, group, new 
HashSet<>(Arrays.asList(new TopicPartition(topic, 0), new TopicPartition(topic, 
1))), Collections.emptyMap());
+                 ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group});
+            ) {
+                TestUtils.waitForCondition(() -> {
+                    Entry<Optional<ConsumerGroupState>, 
Optional<Collection<PartitionAssignmentState>>> res = 
service.collectGroupOffsets(group);
+                    return res.getKey().map(s -> 
s.equals(ConsumerGroupState.EMPTY)).orElse(false)
+                            && res.getValue().isPresent() && 
res.getValue().get().stream().filter(s -> Objects.equals(s.group, 
group)).count() == 2;
+                }, "Expected a stable group with two members in describe group 
state result.");
+            }
         }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void 
testDescribeOffsetsWithMultiPartitionTopicAndMultipleConsumers(String quorum, 
String groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-        String topic2 = "foo2";
-        createTopic(topic2, 2, 1, new Properties(), listenerName(), new 
Properties());
+    @ClusterTemplate("generator")
+    public void testDescribeGroupWithShortInitializationTimeout() throws 
Exception {
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            createTopic(topic);
 
-        // run two consumers in the group consuming from a two-partition topic
-        addConsumerGroupExecutor(2, topic2, GROUP, groupProtocol);
+            // Let creation of the offsets topic happen during group 
initialization to ensure that initialization doesn't
+            // complete before the timeout expires
+            List<String> describeType = 
DESCRIBE_TYPES.get(RANDOM.nextInt(DESCRIBE_TYPES.size()));
+            String group = GROUP_PREFIX + groupProtocol.name() + 
String.join("", describeType);
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            // set the group initialization timeout too low for the group to 
stabilize
+            List<String> cgcArgs = new 
ArrayList<>(Arrays.asList("--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--timeout", "1", "--group", 
group));
+            cgcArgs.addAll(describeType);
 
-        TestUtils.waitForCondition(() -> {
-            Entry<Optional<ConsumerGroupState>, 
Optional<Collection<PartitionAssignmentState>>> res = 
service.collectGroupOffsets(GROUP);
-            return res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false) &&
-                res.getValue().isPresent() &&
-                res.getValue().get().stream().filter(s -> 
Objects.equals(s.group, GROUP)).count() == 2 &&
-                res.getValue().get().stream().filter(x -> 
Objects.equals(x.group, GROUP) && x.partition.isPresent()).count() == 2 &&
-                res.getValue().get().stream().noneMatch(x -> 
Objects.equals(x.group, GROUP) && !x.partition.isPresent());
-        }, "Expected two rows (one row per consumer) in describe group 
results.");
+            // 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(cgcArgs.toArray(new String[0]));
+            ) {
+                ExecutionException e = assertThrows(ExecutionException.class, 
service::describeGroups);
+                assertInstanceOf(TimeoutException.class, e.getCause());
+            }
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void 
testDescribeMembersWithMultiPartitionTopicAndMultipleConsumers(String quorum, 
String groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-        String topic2 = "foo2";
-        createTopic(topic2, 2, 1, new Properties(), listenerName(), new 
Properties());
-
-        // run two consumers in the group consuming from a two-partition topic
-        addConsumerGroupExecutor(2, topic2, GROUP, groupProtocol);
+    @ClusterTemplate("generator")
+    public void testDescribeGroupOffsetsWithShortInitializationTimeout() 
throws Exception {
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic);
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            // Let creation of the offsets topic happen during group 
initialization to ensure that initialization doesn't
+            // complete before the timeout expires
 
-        TestUtils.waitForCondition(() -> {
-            Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res = 
service.collectGroupMembers(GROUP, false);
-            return res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false) &&
-                res.getValue().isPresent() &&
-                res.getValue().get().stream().filter(s -> 
Objects.equals(s.group, GROUP)).count() == 2 &&
-                res.getValue().get().stream().filter(x -> 
Objects.equals(x.group, GROUP) && x.numPartitions == 1).count() == 2 &&
-                res.getValue().get().stream().noneMatch(x -> 
Objects.equals(x.group, GROUP) && x.numPartitions == 0);
-        }, "Expected two rows (one row per consumer) in describe group members 
results.");
-
-        Entry<Optional<ConsumerGroupState>, 
Optional<Collection<MemberAssignmentState>>> res = 
service.collectGroupMembers(GROUP, true);
-        assertTrue(res.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false) && res.getValue().map(s -> 
s.stream().filter(x -> x.assignment.isEmpty()).count()).orElse(0L) == 0,
-            "Expected additional columns in verbose version of describe 
members");
+            // run one consumer in the group consuming from a single-partition 
topic
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap());
+                 // set the group initialization timeout too low for the group 
to stabilize
+                 ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group, 
"--timeout", "1"});
+            ) {
+                Throwable e = assertThrows(ExecutionException.class, () -> 
service.collectGroupOffsets(group));
+                assertEquals(TimeoutException.class, e.getCause().getClass());
+            }
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void 
testDescribeStateWithMultiPartitionTopicAndMultipleConsumers(String quorum, 
String groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-        String topic2 = "foo2";
-        createTopic(topic2, 2, 1, new Properties(), listenerName(), new 
Properties());
+    @ClusterTemplate("generator")
+    public void testDescribeGroupMembersWithShortInitializationTimeout() 
throws Exception {
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic);
 
-        // run two consumers in the group consuming from a two-partition topic
-        addConsumerGroupExecutor(2, topic2, GROUP, groupProtocol);
+            // Let creation of the offsets topic happen during group 
initialization to ensure that initialization doesn't
+            // complete before the timeout expires
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(() -> {
-            GroupState state = service.collectGroupState(GROUP);
-            return Objects.equals(state.state, ConsumerGroupState.STABLE) && 
Objects.equals(state.group, GROUP) && state.numMembers == 2;
-        }, "Expected a stable group with two members in describe group state 
result.");
+            // run one consumer in the group consuming from a single-partition 
topic
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap());
+                 // set the group initialization timeout too low for the group 
to stabilize
+                 ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group, 
"--timeout", "1"});
+            ) {
+                Throwable e = assertThrows(ExecutionException.class, () -> 
service.collectGroupMembers(group, false));
+                assertEquals(TimeoutException.class, e.getCause().getClass());
+                e = assertThrows(ExecutionException.class, () -> 
service.collectGroupMembers(group, true));
+                assertEquals(TimeoutException.class, e.getCause().getClass());
+            }
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft", "kraft+kip848"})
-    public void testDescribeSimpleConsumerGroup(String quorum) throws 
Exception {
-        // Ensure that the offsets of consumers which don't use group 
management are still displayed
-
-        createOffsetsTopic(listenerName(), new Properties());
-        String topic2 = "foo2";
-        createTopic(topic2, 2, 1, new Properties(), listenerName(), new 
Properties());
-        addSimpleGroupExecutor(Arrays.asList(new TopicPartition(topic2, 0), 
new TopicPartition(topic2, 1)), GROUP);
+    @ClusterTemplate("generator")
+    public void testDescribeGroupStateWithShortInitializationTimeout() throws 
Exception {
+        for (GroupProtocol groupProtocol: 
clusterInstance.supportedGroupProtocols()) {
+            String topic = TOPIC_PREFIX + groupProtocol.name();
+            String group = GROUP_PREFIX + groupProtocol.name();
+            createTopic(topic);
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            // Let creation of the offsets topic happen during group 
initialization to ensure that initialization doesn't
+            // complete before the timeout expires
 
-        TestUtils.waitForCondition(() -> {
-            Entry<Optional<ConsumerGroupState>, 
Optional<Collection<PartitionAssignmentState>>> res = 
service.collectGroupOffsets(GROUP);
-            return res.getKey().map(s -> 
s.equals(ConsumerGroupState.EMPTY)).orElse(false)
-                && res.getValue().isPresent() && 
res.getValue().get().stream().filter(s -> Objects.equals(s.group, 
GROUP)).count() == 2;
-        }, "Expected a stable group with two members in describe group state 
result.");
+            // run one consumer in the group consuming from a single-partition 
topic
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap());
+                 // set the group initialization timeout too low for the group 
to stabilize
+                 ConsumerGroupCommand.ConsumerGroupService service = 
consumerGroupService(new String[]{"--bootstrap-server", 
clusterInstance.bootstrapServers(), "--describe", "--group", group, 
"--timeout", "1"});
+            ) {
+                Throwable e = assertThrows(ExecutionException.class, () -> 
service.collectGroupState(group));
+                assertEquals(TimeoutException.class, e.getCause().getClass());
+            }
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeGroupWithShortInitializationTimeout(String quorum, 
String groupProtocol) {
-        // Let creation of the offsets topic happen during group 
initialization to ensure that initialization doesn't
-        // complete before the timeout expires
-
-        List<String> describeType = 
DESCRIBE_TYPES.get(RANDOM.nextInt(DESCRIBE_TYPES.size()));
-        String group = GROUP + String.join("", describeType);
-        // run one consumer in the group consuming from a single-partition 
topic
-        addConsumerGroupExecutor(1, groupProtocol);
-        // set the group initialization timeout too low for the group to 
stabilize
-        List<String> cgcArgs = new 
ArrayList<>(Arrays.asList("--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--timeout", "1", "--group", 
group));
-        cgcArgs.addAll(describeType);
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs.toArray(new String[0]));
-
-        ExecutionException e = assertThrows(ExecutionException.class, 
service::describeGroups);
-        assertInstanceOf(TimeoutException.class, e.getCause());
+    @ClusterTemplate("generator")
+    public void testDescribeWithUnrecognizedNewConsumerOption() {
+        String group = GROUP_PREFIX +  "unrecognized";
+        String[] cgcArgs = new String[]{"--new-consumer", 
"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", 
"--group", group};
+        assertThrows(joptsimple.OptionException.class, () -> 
ConsumerGroupCommandOptions.fromArgs(cgcArgs));
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeGroupOffsetsWithShortInitializationTimeout(String 
quorum, String groupProtocol) {
-        // Let creation of the offsets topic happen during group 
initialization to ensure that initialization doesn't
-        // complete before the timeout expires
+    @ClusterTemplate("generator")
+    public void testDescribeNonOffsetCommitGroup() throws Exception {
+        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
-        addConsumerGroupExecutor(1, groupProtocol);
+            // run one consumer in the group consuming from a single-partition 
topic
+            try (AutoCloseable protocolConsumerGroupExecutor = 
consumerGroupClosable(groupProtocol, group, topic, 
Collections.singletonMap(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"));
+                 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);
 
-        // set the group initialization timeout too low for the group to 
stabilize
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP, "--timeout", 
"1"};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+                    Predicate<PartitionAssignmentState> isGrp = s -> 
Objects.equals(s.group, group);
 
-        Throwable e = assertThrows(ExecutionException.class, () -> 
service.collectGroupOffsets(GROUP));
-        assertEquals(TimeoutException.class, e.getCause().getClass());
-    }
+                    boolean res = groupOffsets.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false) &&
+                            groupOffsets.getValue().isPresent() &&
+                            
groupOffsets.getValue().get().stream().filter(isGrp).count() == 1;
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeGroupMembersWithShortInitializationTimeout(String 
quorum, String groupProtocol) {
-        // Let creation of the offsets topic happen during group 
initialization to ensure that initialization doesn't
-        // complete before the timeout expires
+                    if (!res)
+                        return false;
 
-        // run one consumer in the group consuming from a single-partition 
topic
-        addConsumerGroupExecutor(1, groupProtocol);
+                    Optional<PartitionAssignmentState> maybeAssignmentState = 
groupOffsets.getValue().get().stream().filter(isGrp).findFirst();
+                    if (!maybeAssignmentState.isPresent())
+                        return false;
 
-        // set the group initialization timeout too low for the group to 
stabilize
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP, "--timeout", 
"1"};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+                    PartitionAssignmentState assignmentState = 
maybeAssignmentState.get();
 
-        Throwable e = assertThrows(ExecutionException.class, () -> 
service.collectGroupMembers(GROUP, false));
-        assertEquals(TimeoutException.class, e.getCause().getClass());
-        e = assertThrows(ExecutionException.class, () -> 
service.collectGroupMembers(GROUP, true));
-        assertEquals(TimeoutException.class, e.getCause().getClass());
+                    return assignmentState.consumerId.map(c -> 
!c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) &&
+                            assignmentState.clientId.map(c -> 
!c.trim().equals(ConsumerGroupCommand.MISSING_COLUMN_VALUE)).orElse(false) &&
+                            assignmentState.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 
non-offset-committing group " + group + ".");
+            }
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeGroupStateWithShortInitializationTimeout(String 
quorum, String groupProtocol) {
-        // Let creation of the offsets topic happen during group 
initialization to ensure that initialization doesn't
-        // complete before the timeout expires
-
-        // run one consumer in the group consuming from a single-partition 
topic
-        addConsumerGroupExecutor(1, groupProtocol);
-
-        // set the group initialization timeout too low for the group to 
stabilize
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP, "--timeout", 
"1"};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+    private static ConsumerGroupCommand.ConsumerGroupService 
consumerGroupService(String[] args) {
+        return new ConsumerGroupCommand.ConsumerGroupService(
+                ConsumerGroupCommandOptions.fromArgs(args),
+                Collections.singletonMap(AdminClientConfig.RETRIES_CONFIG, 
Integer.toString(Integer.MAX_VALUE))
+        );
+    }
 
-        Throwable e = assertThrows(ExecutionException.class, () -> 
service.collectGroupState(GROUP));
-        assertEquals(TimeoutException.class, e.getCause().getClass());
+    private void createTopic(String topic) {
+        createTopic(topic, 1);
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDescribeWithUnrecognizedNewConsumerOption(String quorum) {
-        String[] cgcArgs = new String[]{"--new-consumer", 
"--bootstrap-server", bootstrapServers(listenerName()), "--describe", 
"--group", GROUP};
-        assertThrows(joptsimple.OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+    private void createTopic(String topic, int numPartitions) {
+        try (Admin admin = 
Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
 clusterInstance.bootstrapServers()))) {
+            Assertions.assertDoesNotThrow(() -> 
admin.createTopics(Collections.singletonList(new NewTopic(topic, numPartitions, 
(short) 1))).topicId(topic).get());
+        }
     }
 
-    @ParameterizedTest(name = 
TEST_WITH_PARAMETERIZED_QUORUM_AND_GROUP_PROTOCOL_NAMES)
-    @MethodSource({"getTestQuorumAndGroupProtocolParametersAll"})
-    public void testDescribeNonOffsetCommitGroup(String quorum, String 
groupProtocol) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    private void deleteConsumerGroups(Collection<String> groupIds) {
+        try (Admin admin = 
Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
 clusterInstance.bootstrapServers()))) {
+            Assertions.assertDoesNotThrow(() -> 
admin.deleteConsumerGroups(groupIds).all().get());
+        }
+    }
 
-        Properties customProps = new Properties();
-        // create a consumer group that never commits offsets
-        customProps.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, 
"false");
-        // run one consumer in the group consuming from a single-partition 
topic
-        addConsumerGroupExecutor(1, TOPIC, GROUP, 
RangeAssignor.class.getName(), Optional.empty(), Optional.of(customProps), 
false, groupProtocol);
+    private void deleteTopic(String topic) {
+        try (Admin admin = 
Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG,
 clusterInstance.bootstrapServers()))) {
+            Assertions.assertDoesNotThrow(() -> 
admin.deleteTopics(Collections.singletonList(topic)).topicNameValues().get(topic).get());
+        }
+    }
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--describe", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+    private AutoCloseable consumerGroupClosable(GroupProtocol protocol, String 
groupId, Set<TopicPartition> topicPartitions, Map<String, Object> 
customConfigs) {
+        Map<String, Object> configs = composeConfigs(
+                groupId,
+                protocol.name,
+                customConfigs
+        );
+        return ConsumerGroupCommandTestUtils.buildConsumers(
+                1,
+                topicPartitions,
+                () -> new KafkaConsumer<String, String>(configs)
+        );
+    }
 
-        TestUtils.waitForCondition(() -> {
-            Entry<Optional<ConsumerGroupState>, 
Optional<Collection<PartitionAssignmentState>>> groupOffsets = 
service.collectGroupOffsets(GROUP);
+    private AutoCloseable consumerGroupClosable(GroupProtocol protocol, String 
groupId, String topicName, Map<String, Object> customConfigs) {
+        return consumerGroupClosable(protocol, groupId, topicName, 
customConfigs, 1);
+    }
 
-            Predicate<PartitionAssignmentState> isGrp = s -> 
Objects.equals(s.group, GROUP);
+    private AutoCloseable consumerGroupClosable(GroupProtocol protocol, String 
groupId, String topicName, Map<String, Object> customConfigs, int numConsumers) 
{
+        Map<String, Object> configs = composeConfigs(
+                groupId,
+                protocol.name,
+                customConfigs
+        );
+        return ConsumerGroupCommandTestUtils.buildConsumers(
+                numConsumers,
+                false,
+                topicName,
+                () -> new KafkaConsumer<String, String>(configs)
+        );
+    }
 
-            boolean res = groupOffsets.getKey().map(s -> 
s.equals(ConsumerGroupState.STABLE)).orElse(false) &&
-                groupOffsets.getValue().isPresent() &&
-                groupOffsets.getValue().get().stream().filter(isGrp).count() 
== 1;
+    private Map<String, Object> composeConfigs(String groupId, String 
groupProtocol, Map<String, Object> customConfigs) {
+        Map<String, Object> configs = new HashMap<>();
+        configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
clusterInstance.bootstrapServers());
+        configs.put(ConsumerConfig.GROUP_ID_CONFIG, groupId);
+        configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class.getName());
+        configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class.getName());
+        configs.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
+        configs.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, 
RangeAssignor.class.getName());
 
-            if (!res)
-                return false;
+        configs.putAll(customConfigs);
+        return configs;
+    }
 
-            Optional<PartitionAssignmentState> maybeAssignmentState = 
groupOffsets.getValue().get().stream().filter(isGrp).findFirst();
-            if (!maybeAssignmentState.isPresent())
-                return false;
+    private Runnable describeGroups(ConsumerGroupCommand.ConsumerGroupService 
service) {
+        return () -> Assertions.assertDoesNotThrow(service::describeGroups);
+    }
+}
 
-            PartitionAssignmentState assignmentState = 
maybeAssignmentState.get();
+class DescribeConsumerGroupUnitTest {

Review Comment:
   Thanks for the suggestion. I move unit tests back to 
`DescribeConsumerGroupTest` class.



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