chia7712 commented on code in PR #15766:
URL: https://github.com/apache/kafka/pull/15766#discussion_r1590370017


##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -17,279 +17,374 @@
 package org.apache.kafka.tools.consumer.group;
 
 import joptsimple.OptionException;
+import kafka.test.ClusterConfig;
+import kafka.test.ClusterGenerator;
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTemplate;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.AdminClientConfig;
 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.common.ConsumerGroupState;
 import org.apache.kafka.common.errors.GroupIdNotFoundException;
 import org.apache.kafka.common.errors.GroupNotEmptyException;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.test.TestUtils;
 import org.apache.kafka.tools.ToolsTestUtils;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.ValueSource;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Optional;
-import java.util.Properties;
 import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.singletonMap;
+import static kafka.test.annotation.Type.CO_KRAFT;
+import static kafka.test.annotation.Type.KRAFT;
+import static kafka.test.annotation.Type.ZK;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG;
+import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
+import static org.apache.kafka.common.ConsumerGroupState.EMPTY;
+import static org.apache.kafka.common.ConsumerGroupState.STABLE;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest {
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteWithTopicOption(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--topic"};
-        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
-    }
-
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+@ExtendWith(value = ClusterTestExtensions.class)
+public class DeleteConsumerGroupsTest {
+    private final ClusterInstance cluster;
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Group '" + missingGroup + "' could not be 
deleted due to:") && output.contains(Errors.GROUP_ID_NOT_FOUND.message()),
-            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    public DeleteConsumerGroupsTest(ClusterInstance cluster) {
+        this.cluster = cluster;
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
-
-        // note the group to be deleted is a different (non-existing) group
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        Map<String, Throwable> result = service.deleteGroups();
-        assertTrue(result.size() == 1 && result.containsKey(missingGroup) && 
result.get(missingGroup).getCause() instanceof GroupIdNotFoundException,
-            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    private static void generator(ClusterGenerator clusterGenerator) {
+        Map<String, String> serverProperties = new HashMap<>();
+        serverProperties.put(OFFSETS_TOPIC_PARTITIONS_CONFIG, "1");
+        serverProperties.put(OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1");
+        serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "false");
+
+        ClusterConfig zk = ClusterConfig.defaultBuilder()
+                .setType(ZK)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(zk);
+
+        ClusterConfig raftWithLegacyCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(KRAFT)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(raftWithLegacyCoordinator);
+
+        ClusterConfig combinedKRaftWithLegacyCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(CO_KRAFT)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(combinedKRaftWithLegacyCoordinator);
+
+        // Following are test case config with new group coordinator
+        serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true");
+
+        ClusterConfig raftWithNewGroupCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(KRAFT)
+                .setName("newGroupCoordinator")
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(raftWithNewGroupCoordinator);
+
+        ClusterConfig combinedKRaftWithNewGroupCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(CO_KRAFT)
+                .setName("newGroupCoordinator")
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(combinedKRaftWithNewGroupCoordinator);
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
-
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Group '" + GROUP + "' could not be deleted 
due to:") && output.contains(Errors.NON_EMPTY_GROUP.message()),
-            "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not 
detected while deleting consumer group. Output was: (" + output + ")");
+    @ClusterTemplate("generator")
+    public void testDeleteWithTopicOption() {
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", getDummyGroupId(), 
"--topic"};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
-
-        Map<String, Throwable> result = service.deleteGroups();
-        assertNotNull(result.get(GROUP),
-            "Group was deleted successfully, but it shouldn't have been. 
Result was:(" + result + ")");
-        assertTrue(result.size() == 1 && result.containsKey(GROUP) && 
result.get(GROUP).getCause() instanceof GroupNotEmptyException,
-            "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not 
detected while deleting consumer group. Result was:(" + result + ")");
+    @ClusterTemplate("generator")
+    public void testDeleteCmdNonExistingGroup() {
+        String missingGroupId = getDummyGroupId();
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", missingGroupId};
+        try (ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)) {
+            String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
+            assertTrue(output.contains("Group '" + missingGroupId + "' could 
not be deleted due to:") && 
output.contains(Errors.GROUP_ID_NOT_FOUND.message()),
+                    "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") 
was not detected while deleting consumer group");
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected."
-        );
-
-        executor.shutdown();
+    @ClusterTemplate("generator")
+    public void testDeleteNonExistingGroup() {
+        String missingGroupId = getDummyGroupId();
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", missingGroupId};
+        try (ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)) {
+            Map<String, Throwable> result = service.deleteGroups();
+            assertEquals(1, result.size());
+            assertNotNull(result.get(missingGroupId));
+            assertInstanceOf(GroupIdNotFoundException.class,
+                    result.get(missingGroupId).getCause(),
+                    "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") 
was not detected while deleting consumer group");
+        }
+    }
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected."
-        );
+    @ClusterTemplate("generator")
+    public void testDeleteNonEmptyGroup() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable consumerGroupCloseable = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.collectGroupMembers(groupId, 
false).getValue().get().size() == 1,
+                        "The group did not initialize as expected."
+                );
+
+                String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
+                Map<String, Throwable> result = service.deleteGroups();
+
+                assertTrue(output.contains("Group '" + groupId + "' could not 
be deleted due to:") && output.contains(Errors.NON_EMPTY_GROUP.message()),
+                        "The expected error (" + Errors.NON_EMPTY_GROUP + ") 
was not detected while deleting consumer group. Output was: (" + output + ")");
+
+                assertNotNull(result.get(groupId),
+                        "Group was deleted successfully, but it shouldn't have 
been. Result was:(" + result + ")");
+
+                assertEquals(1, result.size());
+                assertNotNull(result.get(groupId));
+                assertInstanceOf(GroupNotEmptyException.class,
+                        result.get(groupId).getCause(),
+                        "The expected error (" + Errors.NON_EMPTY_GROUP + ") 
was not detected while deleting consumer group. Result was:(" + result + ")");
+            }
+        }
+    }
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Deletion of requested consumer groups ('" 
+ GROUP + "') was successful."),
-            "The consumer group could not be deleted as expected");
+    @ClusterTemplate("generator")
+    void testDeleteEmptyGroup() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable consumerGroupCloseable = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.listConsumerGroups().contains(groupId) 
&& checkGroupState(service, groupId, STABLE),
+                        "The group did not initialize as expected."
+                );
+
+                consumerGroupCloseable.close();
+
+                TestUtils.waitForCondition(
+                        () -> checkGroupState(service, groupId, EMPTY),
+                        "The group did not become empty as expected."
+                );
+
+                Map<String, Throwable> result = new HashMap<>();
+                String output = ToolsTestUtils.grabConsoleOutput(() -> 
result.putAll(service.deleteGroups()));
+
+                assertTrue(output.contains("Deletion of requested consumer 
groups ('" + groupId + "') was successful."),
+                        "The consumer group could not be deleted as expected");
+                assertEquals(1, result.size());
+                assertTrue(result.containsKey(groupId));
+                assertNull(result.get(groupId), "The consumer group could not 
be deleted as expected");
+            }
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdAllGroups(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // Create 3 groups with 1 consumer per each
-        Map<String, ConsumerGroupExecutor> groups = IntStream.rangeClosed(1, 
3).mapToObj(i -> GROUP + i).collect(Collectors.toMap(
-            Function.identity(),
-            group -> addConsumerGroupExecutor(1, TOPIC, group, 
RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), false, 
GroupProtocol.CLASSIC.name)
-        ));
-
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--all-groups"};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(() ->
-            new 
HashSet<>(service.listConsumerGroups()).equals(groups.keySet()) &&
-                groups.keySet().stream().allMatch(groupId -> {
-                    try {
-                        return 
Objects.equals(service.collectGroupState(groupId).state, "Stable");
-                    } catch (Exception e) {
-                        throw new RuntimeException(e);
-                    }
-                }),
-            "The group did not initialize as expected.");
-
-        // Shutdown consumers to empty out groups
-        groups.values().forEach(AbstractConsumerGroupExecutor::shutdown);
-
-        TestUtils.waitForCondition(() ->
-            groups.keySet().stream().allMatch(groupId -> {
-                try {
-                    return 
Objects.equals(service.collectGroupState(groupId).state, "Empty");
-                } catch (Exception e) {
-                    throw new RuntimeException(e);
+    @ClusterTemplate("generator")
+    public void testDeleteCmdAllGroups() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String topicName = composeTopicName(groupProtocol);
+            // Create 3 groups with 1 consumer each
+            Map<String, AutoCloseable> groupIdToExecutor = 
IntStream.rangeClosed(1, 3)
+                    .mapToObj(i -> composeGroupId(groupProtocol) + i)
+                    .collect(Collectors.toMap(Function.identity(), group -> 
consumerGroupClosable(groupProtocol, group, topicName)));
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--all-groups"};
+
+            try (ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)) {
+                TestUtils.waitForCondition(() ->
+                                new 
HashSet<>(service.listConsumerGroups()).equals(groupIdToExecutor.keySet()) &&
+                                        
groupIdToExecutor.keySet().stream().allMatch(groupId -> assertDoesNotThrow(() 
-> checkGroupState(service, groupId, STABLE))),
+                        "The group did not initialize as expected.");
+
+                // Shutdown consumers to empty out groups
+                for (AutoCloseable consumerGroupExecutor : 
groupIdToExecutor.values()) {
+                    consumerGroupExecutor.close();
                 }
-            }),
-            "The group did not become empty as expected.");
-
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups).trim();
-        Set<String> expectedGroupsForDeletion = groups.keySet();
-        Set<String> deletedGroupsGrepped = 
Arrays.stream(output.substring(output.indexOf('(') + 1, 
output.indexOf(')')).split(","))
-            .map(str -> str.replaceAll("'", 
"").trim()).collect(Collectors.toSet());
 
-        assertTrue(output.matches("Deletion of requested consumer groups (.*) 
was successful.")
-            && Objects.equals(deletedGroupsGrepped, expectedGroupsForDeletion),
-            "The consumer group(s) could not be deleted as expected");
+                TestUtils.waitForCondition(() ->
+                                
groupIdToExecutor.keySet().stream().allMatch(groupId -> assertDoesNotThrow(() 
-> checkGroupState(service, groupId, EMPTY))),
+                        "The group did not become empty as expected.");
+
+                String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups).trim();
+                Set<String> expectedGroupsForDeletion = 
groupIdToExecutor.keySet();
+                Set<String> deletedGroupsGrepped = 
Arrays.stream(output.substring(output.indexOf('(') + 1, 
output.indexOf(')')).split(","))
+                        .map(str -> str.replaceAll("'", "").trim())
+                        .collect(Collectors.toSet());
+
+                assertTrue(output.matches("Deletion of requested consumer 
groups (.*) was successful.")
+                                && Objects.equals(deletedGroupsGrepped, 
expectedGroupsForDeletion),
+                        "The consumer group(s) could not be deleted as 
expected");
+            }
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
-
-        executor.shutdown();
-
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
-
-        Map<String, Throwable> result = service.deleteGroups();
-        assertTrue(result.size() == 1 && result.containsKey(GROUP) && 
result.get(GROUP) == null,
-            "The consumer group could not be deleted as expected");
+    @ClusterTemplate("generator")
+    public void testDeleteCmdWithMixOfSuccessAndError() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String missingGroupId = composeMissingGroupId(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable consumerGroupClosable = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.listConsumerGroups().contains(groupId) 
&& checkGroupState(service, groupId, STABLE),
+                        "The group did not initialize as expected.");
+
+                consumerGroupClosable.close();
+                TestUtils.waitForCondition(
+                        () -> checkGroupState(service, groupId, EMPTY),
+                        "The group did not become empty as expected.");
+
+                cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId, "--group", 
missingGroupId};
+
+                ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);

Review Comment:
   Please close this `service2`



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -17,279 +17,374 @@
 package org.apache.kafka.tools.consumer.group;
 
 import joptsimple.OptionException;
+import kafka.test.ClusterConfig;
+import kafka.test.ClusterGenerator;
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTemplate;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.AdminClientConfig;
 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.common.ConsumerGroupState;
 import org.apache.kafka.common.errors.GroupIdNotFoundException;
 import org.apache.kafka.common.errors.GroupNotEmptyException;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.test.TestUtils;
 import org.apache.kafka.tools.ToolsTestUtils;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.ValueSource;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Optional;
-import java.util.Properties;
 import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.singletonMap;
+import static kafka.test.annotation.Type.CO_KRAFT;
+import static kafka.test.annotation.Type.KRAFT;
+import static kafka.test.annotation.Type.ZK;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG;
+import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
+import static org.apache.kafka.common.ConsumerGroupState.EMPTY;
+import static org.apache.kafka.common.ConsumerGroupState.STABLE;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest {
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteWithTopicOption(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--topic"};
-        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
-    }
-
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+@ExtendWith(value = ClusterTestExtensions.class)
+public class DeleteConsumerGroupsTest {
+    private final ClusterInstance cluster;
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Group '" + missingGroup + "' could not be 
deleted due to:") && output.contains(Errors.GROUP_ID_NOT_FOUND.message()),
-            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    public DeleteConsumerGroupsTest(ClusterInstance cluster) {
+        this.cluster = cluster;
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
-
-        // note the group to be deleted is a different (non-existing) group
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        Map<String, Throwable> result = service.deleteGroups();
-        assertTrue(result.size() == 1 && result.containsKey(missingGroup) && 
result.get(missingGroup).getCause() instanceof GroupIdNotFoundException,
-            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    private static void generator(ClusterGenerator clusterGenerator) {
+        Map<String, String> serverProperties = new HashMap<>();
+        serverProperties.put(OFFSETS_TOPIC_PARTITIONS_CONFIG, "1");
+        serverProperties.put(OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1");
+        serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "false");
+
+        ClusterConfig zk = ClusterConfig.defaultBuilder()
+                .setType(ZK)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(zk);
+
+        ClusterConfig raftWithLegacyCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(KRAFT)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(raftWithLegacyCoordinator);
+
+        ClusterConfig combinedKRaftWithLegacyCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(CO_KRAFT)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(combinedKRaftWithLegacyCoordinator);
+
+        // Following are test case config with new group coordinator
+        serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true");
+
+        ClusterConfig raftWithNewGroupCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(KRAFT)
+                .setName("newGroupCoordinator")
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(raftWithNewGroupCoordinator);
+
+        ClusterConfig combinedKRaftWithNewGroupCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(CO_KRAFT)
+                .setName("newGroupCoordinator")
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(combinedKRaftWithNewGroupCoordinator);
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
-
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Group '" + GROUP + "' could not be deleted 
due to:") && output.contains(Errors.NON_EMPTY_GROUP.message()),
-            "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not 
detected while deleting consumer group. Output was: (" + output + ")");
+    @ClusterTemplate("generator")
+    public void testDeleteWithTopicOption() {
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", getDummyGroupId(), 
"--topic"};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
-
-        Map<String, Throwable> result = service.deleteGroups();
-        assertNotNull(result.get(GROUP),
-            "Group was deleted successfully, but it shouldn't have been. 
Result was:(" + result + ")");
-        assertTrue(result.size() == 1 && result.containsKey(GROUP) && 
result.get(GROUP).getCause() instanceof GroupNotEmptyException,
-            "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not 
detected while deleting consumer group. Result was:(" + result + ")");
+    @ClusterTemplate("generator")
+    public void testDeleteCmdNonExistingGroup() {
+        String missingGroupId = getDummyGroupId();
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", missingGroupId};
+        try (ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)) {
+            String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
+            assertTrue(output.contains("Group '" + missingGroupId + "' could 
not be deleted due to:") && 
output.contains(Errors.GROUP_ID_NOT_FOUND.message()),
+                    "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") 
was not detected while deleting consumer group");
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected."
-        );
-
-        executor.shutdown();
+    @ClusterTemplate("generator")
+    public void testDeleteNonExistingGroup() {
+        String missingGroupId = getDummyGroupId();
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", missingGroupId};
+        try (ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)) {
+            Map<String, Throwable> result = service.deleteGroups();
+            assertEquals(1, result.size());
+            assertNotNull(result.get(missingGroupId));
+            assertInstanceOf(GroupIdNotFoundException.class,
+                    result.get(missingGroupId).getCause(),
+                    "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") 
was not detected while deleting consumer group");
+        }
+    }
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected."
-        );
+    @ClusterTemplate("generator")
+    public void testDeleteNonEmptyGroup() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable consumerGroupCloseable = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.collectGroupMembers(groupId, 
false).getValue().get().size() == 1,
+                        "The group did not initialize as expected."
+                );
+
+                String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
+                Map<String, Throwable> result = service.deleteGroups();
+
+                assertTrue(output.contains("Group '" + groupId + "' could not 
be deleted due to:") && output.contains(Errors.NON_EMPTY_GROUP.message()),
+                        "The expected error (" + Errors.NON_EMPTY_GROUP + ") 
was not detected while deleting consumer group. Output was: (" + output + ")");
+
+                assertNotNull(result.get(groupId),
+                        "Group was deleted successfully, but it shouldn't have 
been. Result was:(" + result + ")");
+
+                assertEquals(1, result.size());
+                assertNotNull(result.get(groupId));
+                assertInstanceOf(GroupNotEmptyException.class,
+                        result.get(groupId).getCause(),
+                        "The expected error (" + Errors.NON_EMPTY_GROUP + ") 
was not detected while deleting consumer group. Result was:(" + result + ")");
+            }
+        }
+    }
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Deletion of requested consumer groups ('" 
+ GROUP + "') was successful."),
-            "The consumer group could not be deleted as expected");
+    @ClusterTemplate("generator")
+    void testDeleteEmptyGroup() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable consumerGroupCloseable = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.listConsumerGroups().contains(groupId) 
&& checkGroupState(service, groupId, STABLE),
+                        "The group did not initialize as expected."
+                );
+
+                consumerGroupCloseable.close();
+
+                TestUtils.waitForCondition(
+                        () -> checkGroupState(service, groupId, EMPTY),
+                        "The group did not become empty as expected."
+                );
+
+                Map<String, Throwable> result = new HashMap<>();
+                String output = ToolsTestUtils.grabConsoleOutput(() -> 
result.putAll(service.deleteGroups()));
+
+                assertTrue(output.contains("Deletion of requested consumer 
groups ('" + groupId + "') was successful."),
+                        "The consumer group could not be deleted as expected");
+                assertEquals(1, result.size());
+                assertTrue(result.containsKey(groupId));
+                assertNull(result.get(groupId), "The consumer group could not 
be deleted as expected");
+            }
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdAllGroups(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // Create 3 groups with 1 consumer per each
-        Map<String, ConsumerGroupExecutor> groups = IntStream.rangeClosed(1, 
3).mapToObj(i -> GROUP + i).collect(Collectors.toMap(
-            Function.identity(),
-            group -> addConsumerGroupExecutor(1, TOPIC, group, 
RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), false, 
GroupProtocol.CLASSIC.name)
-        ));
-
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--all-groups"};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(() ->
-            new 
HashSet<>(service.listConsumerGroups()).equals(groups.keySet()) &&
-                groups.keySet().stream().allMatch(groupId -> {
-                    try {
-                        return 
Objects.equals(service.collectGroupState(groupId).state, "Stable");
-                    } catch (Exception e) {
-                        throw new RuntimeException(e);
-                    }
-                }),
-            "The group did not initialize as expected.");
-
-        // Shutdown consumers to empty out groups
-        groups.values().forEach(AbstractConsumerGroupExecutor::shutdown);
-
-        TestUtils.waitForCondition(() ->
-            groups.keySet().stream().allMatch(groupId -> {
-                try {
-                    return 
Objects.equals(service.collectGroupState(groupId).state, "Empty");
-                } catch (Exception e) {
-                    throw new RuntimeException(e);
+    @ClusterTemplate("generator")
+    public void testDeleteCmdAllGroups() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String topicName = composeTopicName(groupProtocol);
+            // Create 3 groups with 1 consumer each
+            Map<String, AutoCloseable> groupIdToExecutor = 
IntStream.rangeClosed(1, 3)
+                    .mapToObj(i -> composeGroupId(groupProtocol) + i)
+                    .collect(Collectors.toMap(Function.identity(), group -> 
consumerGroupClosable(groupProtocol, group, topicName)));
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--all-groups"};
+
+            try (ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)) {
+                TestUtils.waitForCondition(() ->
+                                new 
HashSet<>(service.listConsumerGroups()).equals(groupIdToExecutor.keySet()) &&
+                                        
groupIdToExecutor.keySet().stream().allMatch(groupId -> assertDoesNotThrow(() 
-> checkGroupState(service, groupId, STABLE))),
+                        "The group did not initialize as expected.");
+
+                // Shutdown consumers to empty out groups
+                for (AutoCloseable consumerGroupExecutor : 
groupIdToExecutor.values()) {
+                    consumerGroupExecutor.close();
                 }
-            }),
-            "The group did not become empty as expected.");
-
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups).trim();
-        Set<String> expectedGroupsForDeletion = groups.keySet();
-        Set<String> deletedGroupsGrepped = 
Arrays.stream(output.substring(output.indexOf('(') + 1, 
output.indexOf(')')).split(","))
-            .map(str -> str.replaceAll("'", 
"").trim()).collect(Collectors.toSet());
 
-        assertTrue(output.matches("Deletion of requested consumer groups (.*) 
was successful.")
-            && Objects.equals(deletedGroupsGrepped, expectedGroupsForDeletion),
-            "The consumer group(s) could not be deleted as expected");
+                TestUtils.waitForCondition(() ->
+                                
groupIdToExecutor.keySet().stream().allMatch(groupId -> assertDoesNotThrow(() 
-> checkGroupState(service, groupId, EMPTY))),
+                        "The group did not become empty as expected.");
+
+                String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups).trim();
+                Set<String> expectedGroupsForDeletion = 
groupIdToExecutor.keySet();
+                Set<String> deletedGroupsGrepped = 
Arrays.stream(output.substring(output.indexOf('(') + 1, 
output.indexOf(')')).split(","))
+                        .map(str -> str.replaceAll("'", "").trim())
+                        .collect(Collectors.toSet());
+
+                assertTrue(output.matches("Deletion of requested consumer 
groups (.*) was successful.")
+                                && Objects.equals(deletedGroupsGrepped, 
expectedGroupsForDeletion),
+                        "The consumer group(s) could not be deleted as 
expected");
+            }
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
-
-        executor.shutdown();
-
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
-
-        Map<String, Throwable> result = service.deleteGroups();
-        assertTrue(result.size() == 1 && result.containsKey(GROUP) && 
result.get(GROUP) == null,
-            "The consumer group could not be deleted as expected");
+    @ClusterTemplate("generator")
+    public void testDeleteCmdWithMixOfSuccessAndError() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String missingGroupId = composeMissingGroupId(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable consumerGroupClosable = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.listConsumerGroups().contains(groupId) 
&& checkGroupState(service, groupId, STABLE),
+                        "The group did not initialize as expected.");
+
+                consumerGroupClosable.close();
+                TestUtils.waitForCondition(
+                        () -> checkGroupState(service, groupId, EMPTY),
+                        "The group did not become empty as expected.");
+
+                cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId, "--group", 
missingGroupId};
+
+                ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+
+                String output = 
ToolsTestUtils.grabConsoleOutput(service2::deleteGroups);
+                assertTrue(output.contains("Group '" + missingGroupId + "' 
could not be deleted due to:")
+                                && 
output.contains(Errors.GROUP_ID_NOT_FOUND.message())
+                                && output.contains("These consumer groups were 
deleted successfully: '" + groupId + "'"),
+                        "The consumer group deletion did not work as 
expected");
+            }
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdWithMixOfSuccessAndError(String quorum) throws 
Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
-
-        // run one consumer in the group
-        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
-
-        executor.shutdown();
-
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
-
-        cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--group", 
missingGroup};
+    @ClusterTemplate("generator")
+    public void testDeleteWithMixOfSuccessAndError() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String missingGroupId = composeMissingGroupId(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable executor = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.listConsumerGroups().contains(groupId) 
&& checkGroupState(service, groupId, STABLE),
+                        "The group did not initialize as expected.");
+
+                executor.close();
+
+                TestUtils.waitForCondition(
+                        () -> checkGroupState(service, groupId, EMPTY),
+                        "The group did not become empty as expected.");
+
+                cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId, "--group", 
missingGroupId};
+
+                ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);

Review Comment:
   ditto



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -17,279 +17,374 @@
 package org.apache.kafka.tools.consumer.group;
 
 import joptsimple.OptionException;
+import kafka.test.ClusterConfig;
+import kafka.test.ClusterGenerator;
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTemplate;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.AdminClientConfig;
 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.common.ConsumerGroupState;
 import org.apache.kafka.common.errors.GroupIdNotFoundException;
 import org.apache.kafka.common.errors.GroupNotEmptyException;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.test.TestUtils;
 import org.apache.kafka.tools.ToolsTestUtils;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.ValueSource;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Optional;
-import java.util.Properties;
 import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.singletonMap;
+import static kafka.test.annotation.Type.CO_KRAFT;
+import static kafka.test.annotation.Type.KRAFT;
+import static kafka.test.annotation.Type.ZK;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG;
+import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
+import static org.apache.kafka.common.ConsumerGroupState.EMPTY;
+import static org.apache.kafka.common.ConsumerGroupState.STABLE;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest {
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteWithTopicOption(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--topic"};
-        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
-    }
-
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+@ExtendWith(value = ClusterTestExtensions.class)
+public class DeleteConsumerGroupsTest {
+    private final ClusterInstance cluster;
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Group '" + missingGroup + "' could not be 
deleted due to:") && output.contains(Errors.GROUP_ID_NOT_FOUND.message()),
-            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    public DeleteConsumerGroupsTest(ClusterInstance cluster) {

Review Comment:
   As some test cases don't require `ClusterInstance`, please inject 
`ClusterInstance` to method-level instead of class-level. For example:
   
   ```java
       @ClusterTemplate("generator")
       public void testDeleteCmdNonExistingGroup(ClusterInstance cluster)
   ``` 



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -17,279 +17,374 @@
 package org.apache.kafka.tools.consumer.group;
 
 import joptsimple.OptionException;
+import kafka.test.ClusterConfig;
+import kafka.test.ClusterGenerator;
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTemplate;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.AdminClientConfig;
 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.common.ConsumerGroupState;
 import org.apache.kafka.common.errors.GroupIdNotFoundException;
 import org.apache.kafka.common.errors.GroupNotEmptyException;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.test.TestUtils;
 import org.apache.kafka.tools.ToolsTestUtils;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.ValueSource;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Optional;
-import java.util.Properties;
 import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.singletonMap;
+import static kafka.test.annotation.Type.CO_KRAFT;
+import static kafka.test.annotation.Type.KRAFT;
+import static kafka.test.annotation.Type.ZK;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG;
+import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
+import static org.apache.kafka.common.ConsumerGroupState.EMPTY;
+import static org.apache.kafka.common.ConsumerGroupState.STABLE;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest {
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteWithTopicOption(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--topic"};
-        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
-    }
-
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+@ExtendWith(value = ClusterTestExtensions.class)
+public class DeleteConsumerGroupsTest {
+    private final ClusterInstance cluster;
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Group '" + missingGroup + "' could not be 
deleted due to:") && output.contains(Errors.GROUP_ID_NOT_FOUND.message()),
-            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    public DeleteConsumerGroupsTest(ClusterInstance cluster) {
+        this.cluster = cluster;
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
-
-        // note the group to be deleted is a different (non-existing) group
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        Map<String, Throwable> result = service.deleteGroups();
-        assertTrue(result.size() == 1 && result.containsKey(missingGroup) && 
result.get(missingGroup).getCause() instanceof GroupIdNotFoundException,
-            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    private static void generator(ClusterGenerator clusterGenerator) {
+        Map<String, String> serverProperties = new HashMap<>();
+        serverProperties.put(OFFSETS_TOPIC_PARTITIONS_CONFIG, "1");
+        serverProperties.put(OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1");
+        serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "false");
+
+        ClusterConfig zk = ClusterConfig.defaultBuilder()
+                .setType(ZK)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(zk);
+
+        ClusterConfig raftWithLegacyCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(KRAFT)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(raftWithLegacyCoordinator);
+
+        ClusterConfig combinedKRaftWithLegacyCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(CO_KRAFT)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(combinedKRaftWithLegacyCoordinator);
+
+        // Following are test case config with new group coordinator
+        serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true");
+
+        ClusterConfig raftWithNewGroupCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(KRAFT)
+                .setName("newGroupCoordinator")
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(raftWithNewGroupCoordinator);
+
+        ClusterConfig combinedKRaftWithNewGroupCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(CO_KRAFT)
+                .setName("newGroupCoordinator")
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(combinedKRaftWithNewGroupCoordinator);
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
-
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Group '" + GROUP + "' could not be deleted 
due to:") && output.contains(Errors.NON_EMPTY_GROUP.message()),
-            "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not 
detected while deleting consumer group. Output was: (" + output + ")");
+    @ClusterTemplate("generator")

Review Comment:
   ditto. replace it by `@Test`



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -17,279 +17,374 @@
 package org.apache.kafka.tools.consumer.group;
 
 import joptsimple.OptionException;
+import kafka.test.ClusterConfig;
+import kafka.test.ClusterGenerator;
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTemplate;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.AdminClientConfig;
 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.common.ConsumerGroupState;
 import org.apache.kafka.common.errors.GroupIdNotFoundException;
 import org.apache.kafka.common.errors.GroupNotEmptyException;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.test.TestUtils;
 import org.apache.kafka.tools.ToolsTestUtils;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.ValueSource;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Optional;
-import java.util.Properties;
 import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.singletonMap;
+import static kafka.test.annotation.Type.CO_KRAFT;
+import static kafka.test.annotation.Type.KRAFT;
+import static kafka.test.annotation.Type.ZK;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG;
+import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
+import static org.apache.kafka.common.ConsumerGroupState.EMPTY;
+import static org.apache.kafka.common.ConsumerGroupState.STABLE;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest {
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteWithTopicOption(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--topic"};
-        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
-    }
-
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+@ExtendWith(value = ClusterTestExtensions.class)
+public class DeleteConsumerGroupsTest {
+    private final ClusterInstance cluster;
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Group '" + missingGroup + "' could not be 
deleted due to:") && output.contains(Errors.GROUP_ID_NOT_FOUND.message()),
-            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    public DeleteConsumerGroupsTest(ClusterInstance cluster) {
+        this.cluster = cluster;
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
-
-        // note the group to be deleted is a different (non-existing) group
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        Map<String, Throwable> result = service.deleteGroups();
-        assertTrue(result.size() == 1 && result.containsKey(missingGroup) && 
result.get(missingGroup).getCause() instanceof GroupIdNotFoundException,
-            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    private static void generator(ClusterGenerator clusterGenerator) {
+        Map<String, String> serverProperties = new HashMap<>();
+        serverProperties.put(OFFSETS_TOPIC_PARTITIONS_CONFIG, "1");
+        serverProperties.put(OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1");
+        serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "false");
+
+        ClusterConfig zk = ClusterConfig.defaultBuilder()
+                .setType(ZK)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(zk);
+
+        ClusterConfig raftWithLegacyCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(KRAFT)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(raftWithLegacyCoordinator);
+
+        ClusterConfig combinedKRaftWithLegacyCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(CO_KRAFT)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(combinedKRaftWithLegacyCoordinator);
+
+        // Following are test case config with new group coordinator
+        serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true");
+
+        ClusterConfig raftWithNewGroupCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(KRAFT)
+                .setName("newGroupCoordinator")
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(raftWithNewGroupCoordinator);
+
+        ClusterConfig combinedKRaftWithNewGroupCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(CO_KRAFT)
+                .setName("newGroupCoordinator")
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(combinedKRaftWithNewGroupCoordinator);
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
-
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Group '" + GROUP + "' could not be deleted 
due to:") && output.contains(Errors.NON_EMPTY_GROUP.message()),
-            "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not 
detected while deleting consumer group. Output was: (" + output + ")");
+    @ClusterTemplate("generator")
+    public void testDeleteWithTopicOption() {
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", getDummyGroupId(), 
"--topic"};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));

Review Comment:
   ditto



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -17,279 +17,374 @@
 package org.apache.kafka.tools.consumer.group;
 
 import joptsimple.OptionException;
+import kafka.test.ClusterConfig;
+import kafka.test.ClusterGenerator;
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTemplate;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.AdminClientConfig;
 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.common.ConsumerGroupState;
 import org.apache.kafka.common.errors.GroupIdNotFoundException;
 import org.apache.kafka.common.errors.GroupNotEmptyException;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.test.TestUtils;
 import org.apache.kafka.tools.ToolsTestUtils;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.ValueSource;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Optional;
-import java.util.Properties;
 import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.singletonMap;
+import static kafka.test.annotation.Type.CO_KRAFT;
+import static kafka.test.annotation.Type.KRAFT;
+import static kafka.test.annotation.Type.ZK;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG;
+import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
+import static org.apache.kafka.common.ConsumerGroupState.EMPTY;
+import static org.apache.kafka.common.ConsumerGroupState.STABLE;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest {
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteWithTopicOption(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--topic"};
-        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
-    }
-
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+@ExtendWith(value = ClusterTestExtensions.class)
+public class DeleteConsumerGroupsTest {
+    private final ClusterInstance cluster;
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Group '" + missingGroup + "' could not be 
deleted due to:") && output.contains(Errors.GROUP_ID_NOT_FOUND.message()),
-            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    public DeleteConsumerGroupsTest(ClusterInstance cluster) {
+        this.cluster = cluster;
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
-
-        // note the group to be deleted is a different (non-existing) group
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        Map<String, Throwable> result = service.deleteGroups();
-        assertTrue(result.size() == 1 && result.containsKey(missingGroup) && 
result.get(missingGroup).getCause() instanceof GroupIdNotFoundException,
-            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    private static void generator(ClusterGenerator clusterGenerator) {
+        Map<String, String> serverProperties = new HashMap<>();
+        serverProperties.put(OFFSETS_TOPIC_PARTITIONS_CONFIG, "1");
+        serverProperties.put(OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1");
+        serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "false");
+
+        ClusterConfig zk = ClusterConfig.defaultBuilder()
+                .setType(ZK)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(zk);
+
+        ClusterConfig raftWithLegacyCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(KRAFT)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(raftWithLegacyCoordinator);
+
+        ClusterConfig combinedKRaftWithLegacyCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(CO_KRAFT)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(combinedKRaftWithLegacyCoordinator);
+
+        // Following are test case config with new group coordinator
+        serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true");
+
+        ClusterConfig raftWithNewGroupCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(KRAFT)
+                .setName("newGroupCoordinator")
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(raftWithNewGroupCoordinator);
+
+        ClusterConfig combinedKRaftWithNewGroupCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(CO_KRAFT)
+                .setName("newGroupCoordinator")
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(combinedKRaftWithNewGroupCoordinator);
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
-
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Group '" + GROUP + "' could not be deleted 
due to:") && output.contains(Errors.NON_EMPTY_GROUP.message()),
-            "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not 
detected while deleting consumer group. Output was: (" + output + ")");
+    @ClusterTemplate("generator")
+    public void testDeleteWithTopicOption() {
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", getDummyGroupId(), 
"--topic"};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
-
-        Map<String, Throwable> result = service.deleteGroups();
-        assertNotNull(result.get(GROUP),
-            "Group was deleted successfully, but it shouldn't have been. 
Result was:(" + result + ")");
-        assertTrue(result.size() == 1 && result.containsKey(GROUP) && 
result.get(GROUP).getCause() instanceof GroupNotEmptyException,
-            "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not 
detected while deleting consumer group. Result was:(" + result + ")");
+    @ClusterTemplate("generator")
+    public void testDeleteCmdNonExistingGroup() {
+        String missingGroupId = getDummyGroupId();
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", missingGroupId};
+        try (ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)) {
+            String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
+            assertTrue(output.contains("Group '" + missingGroupId + "' could 
not be deleted due to:") && 
output.contains(Errors.GROUP_ID_NOT_FOUND.message()),
+                    "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") 
was not detected while deleting consumer group");
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected."
-        );
-
-        executor.shutdown();
+    @ClusterTemplate("generator")
+    public void testDeleteNonExistingGroup() {
+        String missingGroupId = getDummyGroupId();
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", missingGroupId};
+        try (ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)) {
+            Map<String, Throwable> result = service.deleteGroups();
+            assertEquals(1, result.size());
+            assertNotNull(result.get(missingGroupId));
+            assertInstanceOf(GroupIdNotFoundException.class,
+                    result.get(missingGroupId).getCause(),
+                    "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") 
was not detected while deleting consumer group");
+        }
+    }
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected."
-        );
+    @ClusterTemplate("generator")
+    public void testDeleteNonEmptyGroup() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable consumerGroupCloseable = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.collectGroupMembers(groupId, 
false).getValue().get().size() == 1,
+                        "The group did not initialize as expected."
+                );
+
+                String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
+                Map<String, Throwable> result = service.deleteGroups();
+
+                assertTrue(output.contains("Group '" + groupId + "' could not 
be deleted due to:") && output.contains(Errors.NON_EMPTY_GROUP.message()),
+                        "The expected error (" + Errors.NON_EMPTY_GROUP + ") 
was not detected while deleting consumer group. Output was: (" + output + ")");
+
+                assertNotNull(result.get(groupId),
+                        "Group was deleted successfully, but it shouldn't have 
been. Result was:(" + result + ")");
+
+                assertEquals(1, result.size());
+                assertNotNull(result.get(groupId));
+                assertInstanceOf(GroupNotEmptyException.class,
+                        result.get(groupId).getCause(),
+                        "The expected error (" + Errors.NON_EMPTY_GROUP + ") 
was not detected while deleting consumer group. Result was:(" + result + ")");
+            }
+        }
+    }
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Deletion of requested consumer groups ('" 
+ GROUP + "') was successful."),
-            "The consumer group could not be deleted as expected");
+    @ClusterTemplate("generator")
+    void testDeleteEmptyGroup() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable consumerGroupCloseable = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.listConsumerGroups().contains(groupId) 
&& checkGroupState(service, groupId, STABLE),
+                        "The group did not initialize as expected."
+                );
+
+                consumerGroupCloseable.close();
+
+                TestUtils.waitForCondition(
+                        () -> checkGroupState(service, groupId, EMPTY),
+                        "The group did not become empty as expected."
+                );
+
+                Map<String, Throwable> result = new HashMap<>();
+                String output = ToolsTestUtils.grabConsoleOutput(() -> 
result.putAll(service.deleteGroups()));
+
+                assertTrue(output.contains("Deletion of requested consumer 
groups ('" + groupId + "') was successful."),
+                        "The consumer group could not be deleted as expected");
+                assertEquals(1, result.size());
+                assertTrue(result.containsKey(groupId));
+                assertNull(result.get(groupId), "The consumer group could not 
be deleted as expected");
+            }
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdAllGroups(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // Create 3 groups with 1 consumer per each
-        Map<String, ConsumerGroupExecutor> groups = IntStream.rangeClosed(1, 
3).mapToObj(i -> GROUP + i).collect(Collectors.toMap(
-            Function.identity(),
-            group -> addConsumerGroupExecutor(1, TOPIC, group, 
RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), false, 
GroupProtocol.CLASSIC.name)
-        ));
-
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--all-groups"};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(() ->
-            new 
HashSet<>(service.listConsumerGroups()).equals(groups.keySet()) &&
-                groups.keySet().stream().allMatch(groupId -> {
-                    try {
-                        return 
Objects.equals(service.collectGroupState(groupId).state, "Stable");
-                    } catch (Exception e) {
-                        throw new RuntimeException(e);
-                    }
-                }),
-            "The group did not initialize as expected.");
-
-        // Shutdown consumers to empty out groups
-        groups.values().forEach(AbstractConsumerGroupExecutor::shutdown);
-
-        TestUtils.waitForCondition(() ->
-            groups.keySet().stream().allMatch(groupId -> {
-                try {
-                    return 
Objects.equals(service.collectGroupState(groupId).state, "Empty");
-                } catch (Exception e) {
-                    throw new RuntimeException(e);
+    @ClusterTemplate("generator")
+    public void testDeleteCmdAllGroups() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String topicName = composeTopicName(groupProtocol);
+            // Create 3 groups with 1 consumer each
+            Map<String, AutoCloseable> groupIdToExecutor = 
IntStream.rangeClosed(1, 3)
+                    .mapToObj(i -> composeGroupId(groupProtocol) + i)
+                    .collect(Collectors.toMap(Function.identity(), group -> 
consumerGroupClosable(groupProtocol, group, topicName)));
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--all-groups"};
+
+            try (ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)) {
+                TestUtils.waitForCondition(() ->
+                                new 
HashSet<>(service.listConsumerGroups()).equals(groupIdToExecutor.keySet()) &&
+                                        
groupIdToExecutor.keySet().stream().allMatch(groupId -> assertDoesNotThrow(() 
-> checkGroupState(service, groupId, STABLE))),
+                        "The group did not initialize as expected.");
+
+                // Shutdown consumers to empty out groups
+                for (AutoCloseable consumerGroupExecutor : 
groupIdToExecutor.values()) {
+                    consumerGroupExecutor.close();
                 }
-            }),
-            "The group did not become empty as expected.");
-
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups).trim();
-        Set<String> expectedGroupsForDeletion = groups.keySet();
-        Set<String> deletedGroupsGrepped = 
Arrays.stream(output.substring(output.indexOf('(') + 1, 
output.indexOf(')')).split(","))
-            .map(str -> str.replaceAll("'", 
"").trim()).collect(Collectors.toSet());
 
-        assertTrue(output.matches("Deletion of requested consumer groups (.*) 
was successful.")
-            && Objects.equals(deletedGroupsGrepped, expectedGroupsForDeletion),
-            "The consumer group(s) could not be deleted as expected");
+                TestUtils.waitForCondition(() ->
+                                
groupIdToExecutor.keySet().stream().allMatch(groupId -> assertDoesNotThrow(() 
-> checkGroupState(service, groupId, EMPTY))),
+                        "The group did not become empty as expected.");
+
+                String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups).trim();
+                Set<String> expectedGroupsForDeletion = 
groupIdToExecutor.keySet();
+                Set<String> deletedGroupsGrepped = 
Arrays.stream(output.substring(output.indexOf('(') + 1, 
output.indexOf(')')).split(","))
+                        .map(str -> str.replaceAll("'", "").trim())
+                        .collect(Collectors.toSet());
+
+                assertTrue(output.matches("Deletion of requested consumer 
groups (.*) was successful.")
+                                && Objects.equals(deletedGroupsGrepped, 
expectedGroupsForDeletion),
+                        "The consumer group(s) could not be deleted as 
expected");
+            }
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
-
-        executor.shutdown();
-
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
-
-        Map<String, Throwable> result = service.deleteGroups();
-        assertTrue(result.size() == 1 && result.containsKey(GROUP) && 
result.get(GROUP) == null,
-            "The consumer group could not be deleted as expected");
+    @ClusterTemplate("generator")
+    public void testDeleteCmdWithMixOfSuccessAndError() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String missingGroupId = composeMissingGroupId(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable consumerGroupClosable = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.listConsumerGroups().contains(groupId) 
&& checkGroupState(service, groupId, STABLE),
+                        "The group did not initialize as expected.");
+
+                consumerGroupClosable.close();
+                TestUtils.waitForCondition(
+                        () -> checkGroupState(service, groupId, EMPTY),
+                        "The group did not become empty as expected.");
+
+                cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId, "--group", 
missingGroupId};
+
+                ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+
+                String output = 
ToolsTestUtils.grabConsoleOutput(service2::deleteGroups);
+                assertTrue(output.contains("Group '" + missingGroupId + "' 
could not be deleted due to:")
+                                && 
output.contains(Errors.GROUP_ID_NOT_FOUND.message())
+                                && output.contains("These consumer groups were 
deleted successfully: '" + groupId + "'"),
+                        "The consumer group deletion did not work as 
expected");
+            }
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdWithMixOfSuccessAndError(String quorum) throws 
Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
-
-        // run one consumer in the group
-        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
-
-        executor.shutdown();
-
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
-
-        cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--group", 
missingGroup};
+    @ClusterTemplate("generator")
+    public void testDeleteWithMixOfSuccessAndError() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String missingGroupId = composeMissingGroupId(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable executor = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.listConsumerGroups().contains(groupId) 
&& checkGroupState(service, groupId, STABLE),
+                        "The group did not initialize as expected.");
+
+                executor.close();
+
+                TestUtils.waitForCondition(
+                        () -> checkGroupState(service, groupId, EMPTY),
+                        "The group did not become empty as expected.");
+
+                cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId, "--group", 
missingGroupId};
+
+                ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+                Map<String, Throwable> result = service2.deleteGroups();
+                assertTrue(result.size() == 2 &&
+                                result.containsKey(groupId) && 
result.get(groupId) == null &&
+                                result.containsKey(missingGroupId) &&
+                                
result.get(missingGroupId).getMessage().contains(Errors.GROUP_ID_NOT_FOUND.message()),
+                        "The consumer group deletion did not work as 
expected");
+            }
+        }
+    }
 
-        ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+    @ClusterTemplate("generator")

Review Comment:
   It tests only the arguments, so we don't need to create a embedded server.



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -17,279 +17,374 @@
 package org.apache.kafka.tools.consumer.group;
 
 import joptsimple.OptionException;
+import kafka.test.ClusterConfig;
+import kafka.test.ClusterGenerator;
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterTemplate;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.AdminClientConfig;
 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.common.ConsumerGroupState;
 import org.apache.kafka.common.errors.GroupIdNotFoundException;
 import org.apache.kafka.common.errors.GroupNotEmptyException;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.serialization.StringDeserializer;
 import org.apache.kafka.test.TestUtils;
 import org.apache.kafka.tools.ToolsTestUtils;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.ValueSource;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Optional;
-import java.util.Properties;
 import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.singletonMap;
+import static kafka.test.annotation.Type.CO_KRAFT;
+import static kafka.test.annotation.Type.KRAFT;
+import static kafka.test.annotation.Type.ZK;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG;
+import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
+import static org.apache.kafka.common.ConsumerGroupState.EMPTY;
+import static org.apache.kafka.common.ConsumerGroupState.STABLE;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-public class DeleteConsumerGroupsTest extends ConsumerGroupCommandTest {
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteWithTopicOption(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--topic"};
-        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
-    }
-
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+@ExtendWith(value = ClusterTestExtensions.class)
+public class DeleteConsumerGroupsTest {
+    private final ClusterInstance cluster;
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Group '" + missingGroup + "' could not be 
deleted due to:") && output.contains(Errors.GROUP_ID_NOT_FOUND.message()),
-            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    public DeleteConsumerGroupsTest(ClusterInstance cluster) {
+        this.cluster = cluster;
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
-
-        // note the group to be deleted is a different (non-existing) group
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        Map<String, Throwable> result = service.deleteGroups();
-        assertTrue(result.size() == 1 && result.containsKey(missingGroup) && 
result.get(missingGroup).getCause() instanceof GroupIdNotFoundException,
-            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    private static void generator(ClusterGenerator clusterGenerator) {
+        Map<String, String> serverProperties = new HashMap<>();
+        serverProperties.put(OFFSETS_TOPIC_PARTITIONS_CONFIG, "1");
+        serverProperties.put(OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1");
+        serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "false");
+
+        ClusterConfig zk = ClusterConfig.defaultBuilder()
+                .setType(ZK)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(zk);
+
+        ClusterConfig raftWithLegacyCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(KRAFT)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(raftWithLegacyCoordinator);
+
+        ClusterConfig combinedKRaftWithLegacyCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(CO_KRAFT)
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(combinedKRaftWithLegacyCoordinator);
+
+        // Following are test case config with new group coordinator
+        serverProperties.put(NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true");
+
+        ClusterConfig raftWithNewGroupCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(KRAFT)
+                .setName("newGroupCoordinator")
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(raftWithNewGroupCoordinator);
+
+        ClusterConfig combinedKRaftWithNewGroupCoordinator = 
ClusterConfig.defaultBuilder()
+                .setType(CO_KRAFT)
+                .setName("newGroupCoordinator")
+                .setServerProperties(serverProperties)
+                .build();
+        clusterGenerator.accept(combinedKRaftWithNewGroupCoordinator);
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
-
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Group '" + GROUP + "' could not be deleted 
due to:") && output.contains(Errors.NON_EMPTY_GROUP.message()),
-            "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not 
detected while deleting consumer group. Output was: (" + output + ")");
+    @ClusterTemplate("generator")
+    public void testDeleteWithTopicOption() {
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", getDummyGroupId(), 
"--topic"};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
-
-        Map<String, Throwable> result = service.deleteGroups();
-        assertNotNull(result.get(GROUP),
-            "Group was deleted successfully, but it shouldn't have been. 
Result was:(" + result + ")");
-        assertTrue(result.size() == 1 && result.containsKey(GROUP) && 
result.get(GROUP).getCause() instanceof GroupNotEmptyException,
-            "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not 
detected while deleting consumer group. Result was:(" + result + ")");
+    @ClusterTemplate("generator")
+    public void testDeleteCmdNonExistingGroup() {
+        String missingGroupId = getDummyGroupId();
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", missingGroupId};
+        try (ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)) {
+            String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
+            assertTrue(output.contains("Group '" + missingGroupId + "' could 
not be deleted due to:") && 
output.contains(Errors.GROUP_ID_NOT_FOUND.message()),
+                    "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") 
was not detected while deleting consumer group");
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected."
-        );
-
-        executor.shutdown();
+    @ClusterTemplate("generator")
+    public void testDeleteNonExistingGroup() {
+        String missingGroupId = getDummyGroupId();
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", missingGroupId};
+        try (ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)) {
+            Map<String, Throwable> result = service.deleteGroups();
+            assertEquals(1, result.size());
+            assertNotNull(result.get(missingGroupId));
+            assertInstanceOf(GroupIdNotFoundException.class,
+                    result.get(missingGroupId).getCause(),
+                    "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") 
was not detected while deleting consumer group");
+        }
+    }
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected."
-        );
+    @ClusterTemplate("generator")
+    public void testDeleteNonEmptyGroup() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable consumerGroupCloseable = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.collectGroupMembers(groupId, 
false).getValue().get().size() == 1,
+                        "The group did not initialize as expected."
+                );
+
+                String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
+                Map<String, Throwable> result = service.deleteGroups();
+
+                assertTrue(output.contains("Group '" + groupId + "' could not 
be deleted due to:") && output.contains(Errors.NON_EMPTY_GROUP.message()),
+                        "The expected error (" + Errors.NON_EMPTY_GROUP + ") 
was not detected while deleting consumer group. Output was: (" + output + ")");
+
+                assertNotNull(result.get(groupId),
+                        "Group was deleted successfully, but it shouldn't have 
been. Result was:(" + result + ")");
+
+                assertEquals(1, result.size());
+                assertNotNull(result.get(groupId));
+                assertInstanceOf(GroupNotEmptyException.class,
+                        result.get(groupId).getCause(),
+                        "The expected error (" + Errors.NON_EMPTY_GROUP + ") 
was not detected while deleting consumer group. Result was:(" + result + ")");
+            }
+        }
+    }
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups);
-        assertTrue(output.contains("Deletion of requested consumer groups ('" 
+ GROUP + "') was successful."),
-            "The consumer group could not be deleted as expected");
+    @ClusterTemplate("generator")
+    void testDeleteEmptyGroup() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable consumerGroupCloseable = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.listConsumerGroups().contains(groupId) 
&& checkGroupState(service, groupId, STABLE),
+                        "The group did not initialize as expected."
+                );
+
+                consumerGroupCloseable.close();
+
+                TestUtils.waitForCondition(
+                        () -> checkGroupState(service, groupId, EMPTY),
+                        "The group did not become empty as expected."
+                );
+
+                Map<String, Throwable> result = new HashMap<>();
+                String output = ToolsTestUtils.grabConsoleOutput(() -> 
result.putAll(service.deleteGroups()));
+
+                assertTrue(output.contains("Deletion of requested consumer 
groups ('" + groupId + "') was successful."),
+                        "The consumer group could not be deleted as expected");
+                assertEquals(1, result.size());
+                assertTrue(result.containsKey(groupId));
+                assertNull(result.get(groupId), "The consumer group could not 
be deleted as expected");
+            }
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdAllGroups(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // Create 3 groups with 1 consumer per each
-        Map<String, ConsumerGroupExecutor> groups = IntStream.rangeClosed(1, 
3).mapToObj(i -> GROUP + i).collect(Collectors.toMap(
-            Function.identity(),
-            group -> addConsumerGroupExecutor(1, TOPIC, group, 
RangeAssignor.class.getName(), Optional.empty(), Optional.empty(), false, 
GroupProtocol.CLASSIC.name)
-        ));
-
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--all-groups"};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(() ->
-            new 
HashSet<>(service.listConsumerGroups()).equals(groups.keySet()) &&
-                groups.keySet().stream().allMatch(groupId -> {
-                    try {
-                        return 
Objects.equals(service.collectGroupState(groupId).state, "Stable");
-                    } catch (Exception e) {
-                        throw new RuntimeException(e);
-                    }
-                }),
-            "The group did not initialize as expected.");
-
-        // Shutdown consumers to empty out groups
-        groups.values().forEach(AbstractConsumerGroupExecutor::shutdown);
-
-        TestUtils.waitForCondition(() ->
-            groups.keySet().stream().allMatch(groupId -> {
-                try {
-                    return 
Objects.equals(service.collectGroupState(groupId).state, "Empty");
-                } catch (Exception e) {
-                    throw new RuntimeException(e);
+    @ClusterTemplate("generator")
+    public void testDeleteCmdAllGroups() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String topicName = composeTopicName(groupProtocol);
+            // Create 3 groups with 1 consumer each
+            Map<String, AutoCloseable> groupIdToExecutor = 
IntStream.rangeClosed(1, 3)
+                    .mapToObj(i -> composeGroupId(groupProtocol) + i)
+                    .collect(Collectors.toMap(Function.identity(), group -> 
consumerGroupClosable(groupProtocol, group, topicName)));
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--all-groups"};
+
+            try (ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)) {
+                TestUtils.waitForCondition(() ->
+                                new 
HashSet<>(service.listConsumerGroups()).equals(groupIdToExecutor.keySet()) &&
+                                        
groupIdToExecutor.keySet().stream().allMatch(groupId -> assertDoesNotThrow(() 
-> checkGroupState(service, groupId, STABLE))),
+                        "The group did not initialize as expected.");
+
+                // Shutdown consumers to empty out groups
+                for (AutoCloseable consumerGroupExecutor : 
groupIdToExecutor.values()) {
+                    consumerGroupExecutor.close();
                 }
-            }),
-            "The group did not become empty as expected.");
-
-        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups).trim();
-        Set<String> expectedGroupsForDeletion = groups.keySet();
-        Set<String> deletedGroupsGrepped = 
Arrays.stream(output.substring(output.indexOf('(') + 1, 
output.indexOf(')')).split(","))
-            .map(str -> str.replaceAll("'", 
"").trim()).collect(Collectors.toSet());
 
-        assertTrue(output.matches("Deletion of requested consumer groups (.*) 
was successful.")
-            && Objects.equals(deletedGroupsGrepped, expectedGroupsForDeletion),
-            "The consumer group(s) could not be deleted as expected");
+                TestUtils.waitForCondition(() ->
+                                
groupIdToExecutor.keySet().stream().allMatch(groupId -> assertDoesNotThrow(() 
-> checkGroupState(service, groupId, EMPTY))),
+                        "The group did not become empty as expected.");
+
+                String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups).trim();
+                Set<String> expectedGroupsForDeletion = 
groupIdToExecutor.keySet();
+                Set<String> deletedGroupsGrepped = 
Arrays.stream(output.substring(output.indexOf('(') + 1, 
output.indexOf(')')).split(","))
+                        .map(str -> str.replaceAll("'", "").trim())
+                        .collect(Collectors.toSet());
+
+                assertTrue(output.matches("Deletion of requested consumer 
groups (.*) was successful.")
+                                && Objects.equals(deletedGroupsGrepped, 
expectedGroupsForDeletion),
+                        "The consumer group(s) could not be deleted as 
expected");
+            }
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-
-        // run one consumer in the group
-        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
-
-        executor.shutdown();
-
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
-
-        Map<String, Throwable> result = service.deleteGroups();
-        assertTrue(result.size() == 1 && result.containsKey(GROUP) && 
result.get(GROUP) == null,
-            "The consumer group could not be deleted as expected");
+    @ClusterTemplate("generator")
+    public void testDeleteCmdWithMixOfSuccessAndError() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String missingGroupId = composeMissingGroupId(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable consumerGroupClosable = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.listConsumerGroups().contains(groupId) 
&& checkGroupState(service, groupId, STABLE),
+                        "The group did not initialize as expected.");
+
+                consumerGroupClosable.close();
+                TestUtils.waitForCondition(
+                        () -> checkGroupState(service, groupId, EMPTY),
+                        "The group did not become empty as expected.");
+
+                cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId, "--group", 
missingGroupId};
+
+                ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+
+                String output = 
ToolsTestUtils.grabConsoleOutput(service2::deleteGroups);
+                assertTrue(output.contains("Group '" + missingGroupId + "' 
could not be deleted due to:")
+                                && 
output.contains(Errors.GROUP_ID_NOT_FOUND.message())
+                                && output.contains("These consumer groups were 
deleted successfully: '" + groupId + "'"),
+                        "The consumer group deletion did not work as 
expected");
+            }
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdWithMixOfSuccessAndError(String quorum) throws 
Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
-
-        // run one consumer in the group
-        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
-
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
-
-        executor.shutdown();
-
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
-
-        cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--group", 
missingGroup};
+    @ClusterTemplate("generator")
+    public void testDeleteWithMixOfSuccessAndError() throws Exception {
+        for (GroupProtocol groupProtocol : cluster.supportedGroupProtocols()) {
+            String groupId = composeGroupId(groupProtocol);
+            String topicName = composeTopicName(groupProtocol);
+            String missingGroupId = composeMissingGroupId(groupProtocol);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId};
+            try (
+                    AutoCloseable executor = 
consumerGroupClosable(groupProtocol, groupId, topicName);
+                    ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs)
+            ) {
+                TestUtils.waitForCondition(
+                        () -> service.listConsumerGroups().contains(groupId) 
&& checkGroupState(service, groupId, STABLE),
+                        "The group did not initialize as expected.");
+
+                executor.close();
+
+                TestUtils.waitForCondition(
+                        () -> checkGroupState(service, groupId, EMPTY),
+                        "The group did not become empty as expected.");
+
+                cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", groupId, "--group", 
missingGroupId};
+
+                ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+                Map<String, Throwable> result = service2.deleteGroups();
+                assertTrue(result.size() == 2 &&
+                                result.containsKey(groupId) && 
result.get(groupId) == null &&
+                                result.containsKey(missingGroupId) &&
+                                
result.get(missingGroupId).getMessage().contains(Errors.GROUP_ID_NOT_FOUND.message()),
+                        "The consumer group deletion did not work as 
expected");
+            }
+        }
+    }
 
-        ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+    @ClusterTemplate("generator")
+    public void testDeleteWithUnrecognizedNewConsumerOption() {
+        String[] cgcArgs = new String[]{"--new-consumer", 
"--bootstrap-server", cluster.bootstrapServers(), "--delete", "--group", 
getDummyGroupId()};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));

Review Comment:
   We expect the option parse get failed, so it would be more suitable to test 
`ConsumerGroupCommandOptions.fromArgs` directly



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