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


##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -17,279 +17,477 @@
 package org.apache.kafka.tools.consumer.group;
 
 import joptsimple.OptionException;
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterConfigProperty;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.GroupProtocol;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.RangeAssignor;
 import org.apache.kafka.common.errors.GroupIdNotFoundException;
 import org.apache.kafka.common.errors.GroupNotEmptyException;
+import org.apache.kafka.common.errors.WakeupException;
 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.time.Duration;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static java.util.Collections.singleton;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 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";
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL, brokers = 3, serverProperties = {
+        @ClusterConfigProperty(key = "offsets.topic.num.partitions", value = 
"1"),
+        @ClusterConfigProperty(key = "offsets.topic.replication.factor", value 
= "1"),
+})
+public class DeleteConsumerGroupsTest {
+    private final ClusterInstance cluster;
+    private static final String TOPIC = "foo";
+    private static final String GROUP = "test.group";
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+    public DeleteConsumerGroupsTest(ClusterInstance cluster) {
+        this.cluster = 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");
+    @ClusterTest
+    public void testDeleteWithTopicOption() {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());

Review Comment:
   please call `get` to make sure the request is completed.



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -17,279 +17,477 @@
 package org.apache.kafka.tools.consumer.group;
 
 import joptsimple.OptionException;
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterConfigProperty;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.GroupProtocol;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.RangeAssignor;
 import org.apache.kafka.common.errors.GroupIdNotFoundException;
 import org.apache.kafka.common.errors.GroupNotEmptyException;
+import org.apache.kafka.common.errors.WakeupException;
 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.time.Duration;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static java.util.Collections.singleton;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 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";
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL, brokers = 3, serverProperties = {
+        @ClusterConfigProperty(key = "offsets.topic.num.partitions", value = 
"1"),
+        @ClusterConfigProperty(key = "offsets.topic.replication.factor", value 
= "1"),
+})
+public class DeleteConsumerGroupsTest {
+    private final ClusterInstance cluster;
+    private static final String TOPIC = "foo";
+    private static final String GROUP = "test.group";
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+    public DeleteConsumerGroupsTest(ClusterInstance cluster) {
+        this.cluster = 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");
+    @ClusterTest
+    public void testDeleteWithTopicOption() {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP, "--topic"};
+            assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
+    @ClusterTest
+    public void testDeleteCmdNonExistingGroup() {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", missingGroup};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            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");
+        }
+    }
 
-        // 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);
+    @ClusterTest
+    public void testDeleteNonExistingGroup() {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
+            // note the group to be deleted is a different (non-existing) group
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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");
+        }
+    }
 
-        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");
+    @ClusterTest
+    public void testDeleteCmdNonEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            // run one consumer in the group
+            ConsumerGroupExecutor consumerGroupExecutor = 
buildConsumerGroupExecutor(GROUP);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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 + ")");
+
+            consumerGroupExecutor.shutdown();
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTest
+    public void testDeleteNonEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
 
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            // run one consumer in the group
+            ConsumerGroupExecutor consumerGroupExecutor = 
buildConsumerGroupExecutor(GROUP);
 
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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 + ")");
 
-        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 + ")");
+            consumerGroupExecutor.shutdown();
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTest
+    public void testDeleteCmdEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
 
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            // run one consumer in the group
+            ConsumerGroupExecutor consumerGroupExecutor = 
buildConsumerGroupExecutor(GROUP);
 
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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."
+            );
+
+            consumerGroupExecutor.shutdown();
+
+            TestUtils.waitForCondition(
+                    () -> 
Objects.equals(service.collectGroupState(GROUP).state, "Empty"),
+                    "The group did not become empty 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 + ")");
+            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");
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTest
+    public void testDeleteCmdAllGroups() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+
+            // Create 3 groups with 1 consumer per each
+            Map<String, ConsumerGroupExecutor> groupNameToExecutor = 
IntStream.rangeClosed(1, 3)
+                    .mapToObj(i -> GROUP + i)
+                    .collect(Collectors.toMap(Function.identity(), 
this::buildConsumerGroupExecutor));
+
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--all-groups"};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+            TestUtils.waitForCondition(() ->
+                            new 
HashSet<>(service.listConsumerGroups()).equals(groupNameToExecutor.keySet()) &&
+                                    
groupNameToExecutor.keySet().stream().allMatch(checkGroupState(service, 
"Stable")),
+                    "The group did not initialize as expected.");
+
+            // Shutdown consumers to empty out groups
+            
groupNameToExecutor.values().forEach(ConsumerGroupExecutor::shutdown);
+
+            TestUtils.waitForCondition(() ->
+                            
groupNameToExecutor.keySet().stream().allMatch(checkGroupState(service, 
"Empty")),
+                    "The group did not become empty as expected.");
+
+            String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups).trim();
+            Set<String> expectedGroupsForDeletion = 
groupNameToExecutor.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");
+        }
+    }
 
-        // 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);
+    @ClusterTest
+    public void testDeleteEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
 
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected."
-        );
+            // run one consumer in the group
+            ConsumerGroupExecutor executor = buildConsumerGroupExecutor(GROUP);
 
-        executor.shutdown();
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected."
-        );
+            TestUtils.waitForCondition(
+                    () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
+                    "The group did not initialize as expected.");
+
+            executor.shutdown();
 
-        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");
+            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");
+        }
     }
 
-    @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);
-                }
-            }),
-            "The group did not become empty as expected.");
+    @ClusterTest
+    public void testDeleteCmdWithMixOfSuccessAndError() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
 
-        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());
+            // run one consumer in the group
+            ConsumerGroupExecutor executor = buildConsumerGroupExecutor(GROUP);
 
-        assertTrue(output.matches("Deletion of requested consumer groups (.*) 
was successful.")
-            && Objects.equals(deletedGroupsGrepped, expectedGroupsForDeletion),
-            "The consumer group(s) could not be deleted as expected");
-    }
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+            TestUtils.waitForCondition(
+                    () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
+                    "The group did not initialize as expected.");
 
-        // 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);
+            executor.shutdown();
 
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
+            TestUtils.waitForCondition(
+                    () -> 
Objects.equals(service.collectGroupState(GROUP).state, "Empty"),
+                    "The group did not become empty as expected.");
 
-        executor.shutdown();
+            cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP, "--group", 
missingGroup};
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
+            ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
 
-        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");
+            String output = 
ToolsTestUtils.grabConsoleOutput(service2::deleteGroups);
+            assertTrue(output.contains("Group '" + missingGroup + "' could not 
be deleted due to:")
+                            && 
output.contains(Errors.GROUP_ID_NOT_FOUND.message())
+                            && output.contains("These consumer groups were 
deleted successfully: '" + GROUP + "'"),
+                    "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";
+    @ClusterTest
+    public void testDeleteWithMixOfSuccessAndError() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
+
+            // run one consumer in the group
+            ConsumerGroupExecutor executor = buildConsumerGroupExecutor(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);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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.");
+            TestUtils.waitForCondition(
+                    () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
+                    "The group did not initialize as expected.");
 
-        executor.shutdown();
+            executor.shutdown();
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
+            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};
+            cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP, "--group", 
missingGroup};
 
-        ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+            ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+            Map<String, Throwable> result = service2.deleteGroups();
+            assertTrue(result.size() == 2 &&
+                            result.containsKey(GROUP) && result.get(GROUP) == 
null &&
+                            result.containsKey(missingGroup) &&
+                            
result.get(missingGroup).getMessage().contains(Errors.GROUP_ID_NOT_FOUND.message()),
+                    "The consumer group deletion did not work as expected");
+        }
+    }
+
+    @ClusterTest
+    public void testDeleteWithUnrecognizedNewConsumerOption() {
+        String[] cgcArgs = new String[]{"--new-consumer", 
"--bootstrap-server", cluster.bootstrapServers(), "--delete", "--group", GROUP};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+    }
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service2::deleteGroups);
-        assertTrue(output.contains("Group '" + missingGroup + "' could not be 
deleted due to:")
-            && output.contains(Errors.GROUP_ID_NOT_FOUND.message())
-            && output.contains("These consumer groups were deleted 
successfully: '" + GROUP + "'"),
-            "The consumer group deletion did not work as expected");
+    private ConsumerGroupExecutor buildConsumerGroupExecutor(String group) {
+        return new ConsumerGroupExecutor(cluster.bootstrapServers(),
+                1,
+                null != group ? group : GROUP,
+                GroupProtocol.CLASSIC.name,
+                TOPIC,
+                RangeAssignor.class.getName(),
+                Optional.empty(),
+                Optional.empty(),
+                false);
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteWithMixOfSuccessAndError(String quorum) throws 
Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
+    private Predicate<String> 
checkGroupState(ConsumerGroupCommand.ConsumerGroupService service, String 
stable) {
+        return groupId -> {
+            try {
+                return 
Objects.equals(service.collectGroupState(groupId).state, stable);
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        };
+    }
 
-        // 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);
+    private Set<NewTopic> buildSingletonTestTopic() {
+        return singleton(new NewTopic(TOPIC, 1, (short) 1));
+    }
 
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
+    ConsumerGroupCommand.ConsumerGroupService getConsumerGroupService(String[] 
args) {
+        ConsumerGroupCommandOptions opts = 
ConsumerGroupCommandOptions.fromArgs(args);
+        return new ConsumerGroupCommand.ConsumerGroupService(
+                opts,
+                Collections.singletonMap(AdminClientConfig.RETRIES_CONFIG, 
Integer.toString(Integer.MAX_VALUE))
+        );
+    }
 
-        executor.shutdown();
+    abstract class AbstractConsumerRunnable implements Runnable {
+        final String broker;
+        final String groupId;
+        final Optional<Properties> customPropsOpt;
+        final boolean syncCommit;
+
+        final Properties props = new Properties();
+        KafkaConsumer<String, String> consumer;
+
+        boolean configured = false;
+
+        public AbstractConsumerRunnable(String broker, String groupId, 
Optional<Properties> customPropsOpt, boolean syncCommit) {
+            this.broker = broker;
+            this.groupId = groupId;
+            this.customPropsOpt = customPropsOpt;
+            this.syncCommit = syncCommit;
+        }
+
+        void configure() {
+            configured = true;
+            configure(props);
+            customPropsOpt.ifPresent(props::putAll);
+            consumer = new KafkaConsumer<>(props);
+        }
+
+        void configure(Properties props) {
+            props.put("bootstrap.servers", broker);
+            props.put("group.id", groupId);
+            props.put("key.deserializer", StringDeserializer.class.getName());
+            props.put("value.deserializer", 
StringDeserializer.class.getName());
+        }
+
+        abstract void subscribe();
+
+        @Override
+        public void run() {
+            assert configured : "Must call configure before use";
+            try {
+                subscribe();
+                while (true) {
+                    consumer.poll(Duration.ofMillis(Long.MAX_VALUE));
+                    if (syncCommit)
+                        consumer.commitSync();
+                }
+            } catch (WakeupException e) {
+                // OK
+            } finally {
+                consumer.close();
+            }
+        }
+
+        void shutdown() {
+            consumer.wakeup();
+        }
+    }
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
+    class ConsumerRunnable extends AbstractConsumerRunnable {
+        final String topic;
+        final String groupProtocol;
+        final String strategy;
+        final Optional<String> remoteAssignor;
+
+        public ConsumerRunnable(String broker, String groupId, String 
groupProtocol, String topic, String strategy,
+                                Optional<String> remoteAssignor, 
Optional<Properties> customPropsOpt, boolean syncCommit) {
+            super(broker, groupId, customPropsOpt, syncCommit);
+
+            this.topic = topic;
+            this.groupProtocol = groupProtocol;
+            this.strategy = strategy;
+            this.remoteAssignor = remoteAssignor;
+        }
+
+        @Override
+        void configure(Properties props) {
+            super.configure(props);
+            props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
+            if 
(groupProtocol.toUpperCase(Locale.ROOT).equals(GroupProtocol.CONSUMER.toString()))
 {
+                remoteAssignor.ifPresent(assignor -> 
props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, assignor));
+            } else {
+                props.put("partition.assignment.strategy", strategy);
+            }
+        }
+
+        @Override
+        void subscribe() {
+            consumer.subscribe(Collections.singleton(topic));
+        }
+    }
 
-        cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--group", 
missingGroup};
 
-        ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
-        Map<String, Throwable> result = service2.deleteGroups();
-        assertTrue(result.size() == 2 &&
-                result.containsKey(GROUP) && result.get(GROUP) == null &&
-                result.containsKey(missingGroup) &&
-                
result.get(missingGroup).getMessage().contains(Errors.GROUP_ID_NOT_FOUND.message()),
-            "The consumer group deletion did not work as expected");
+    class AbstractConsumerGroupExecutor {

Review Comment:
   those classes can be static class.



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -17,279 +17,477 @@
 package org.apache.kafka.tools.consumer.group;
 
 import joptsimple.OptionException;
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterConfigProperty;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.GroupProtocol;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.RangeAssignor;
 import org.apache.kafka.common.errors.GroupIdNotFoundException;
 import org.apache.kafka.common.errors.GroupNotEmptyException;
+import org.apache.kafka.common.errors.WakeupException;
 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.time.Duration;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static java.util.Collections.singleton;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 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";
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL, brokers = 3, serverProperties = {
+        @ClusterConfigProperty(key = "offsets.topic.num.partitions", value = 
"1"),
+        @ClusterConfigProperty(key = "offsets.topic.replication.factor", value 
= "1"),
+})
+public class DeleteConsumerGroupsTest {
+    private final ClusterInstance cluster;
+    private static final String TOPIC = "foo";
+    private static final String GROUP = "test.group";
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+    public DeleteConsumerGroupsTest(ClusterInstance cluster) {
+        this.cluster = 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");
+    @ClusterTest
+    public void testDeleteWithTopicOption() {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP, "--topic"};
+            assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
+    @ClusterTest
+    public void testDeleteCmdNonExistingGroup() {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", missingGroup};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            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");
+        }
+    }
 
-        // 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);
+    @ClusterTest
+    public void testDeleteNonExistingGroup() {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
+            // note the group to be deleted is a different (non-existing) group
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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");
+        }
+    }
 
-        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");
+    @ClusterTest
+    public void testDeleteCmdNonEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            // run one consumer in the group
+            ConsumerGroupExecutor consumerGroupExecutor = 
buildConsumerGroupExecutor(GROUP);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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 + ")");
+
+            consumerGroupExecutor.shutdown();
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTest
+    public void testDeleteNonEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
 
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            // run one consumer in the group
+            ConsumerGroupExecutor consumerGroupExecutor = 
buildConsumerGroupExecutor(GROUP);
 
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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 + ")");
 
-        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 + ")");
+            consumerGroupExecutor.shutdown();
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTest
+    public void testDeleteCmdEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
 
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            // run one consumer in the group
+            ConsumerGroupExecutor consumerGroupExecutor = 
buildConsumerGroupExecutor(GROUP);
 
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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."
+            );
+
+            consumerGroupExecutor.shutdown();
+
+            TestUtils.waitForCondition(
+                    () -> 
Objects.equals(service.collectGroupState(GROUP).state, "Empty"),
+                    "The group did not become empty 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 + ")");
+            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");
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTest
+    public void testDeleteCmdAllGroups() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+
+            // Create 3 groups with 1 consumer per each
+            Map<String, ConsumerGroupExecutor> groupNameToExecutor = 
IntStream.rangeClosed(1, 3)
+                    .mapToObj(i -> GROUP + i)
+                    .collect(Collectors.toMap(Function.identity(), 
this::buildConsumerGroupExecutor));
+
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--all-groups"};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+            TestUtils.waitForCondition(() ->
+                            new 
HashSet<>(service.listConsumerGroups()).equals(groupNameToExecutor.keySet()) &&
+                                    
groupNameToExecutor.keySet().stream().allMatch(checkGroupState(service, 
"Stable")),
+                    "The group did not initialize as expected.");
+
+            // Shutdown consumers to empty out groups
+            
groupNameToExecutor.values().forEach(ConsumerGroupExecutor::shutdown);
+
+            TestUtils.waitForCondition(() ->
+                            
groupNameToExecutor.keySet().stream().allMatch(checkGroupState(service, 
"Empty")),
+                    "The group did not become empty as expected.");
+
+            String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups).trim();
+            Set<String> expectedGroupsForDeletion = 
groupNameToExecutor.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");
+        }
+    }
 
-        // 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);
+    @ClusterTest
+    public void testDeleteEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
 
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected."
-        );
+            // run one consumer in the group
+            ConsumerGroupExecutor executor = buildConsumerGroupExecutor(GROUP);
 
-        executor.shutdown();
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected."
-        );
+            TestUtils.waitForCondition(
+                    () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
+                    "The group did not initialize as expected.");
+
+            executor.shutdown();
 
-        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");
+            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");
+        }
     }
 
-    @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);
-                }
-            }),
-            "The group did not become empty as expected.");
+    @ClusterTest
+    public void testDeleteCmdWithMixOfSuccessAndError() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
 
-        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());
+            // run one consumer in the group
+            ConsumerGroupExecutor executor = buildConsumerGroupExecutor(GROUP);
 
-        assertTrue(output.matches("Deletion of requested consumer groups (.*) 
was successful.")
-            && Objects.equals(deletedGroupsGrepped, expectedGroupsForDeletion),
-            "The consumer group(s) could not be deleted as expected");
-    }
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+            TestUtils.waitForCondition(
+                    () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
+                    "The group did not initialize as expected.");
 
-        // 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);
+            executor.shutdown();
 
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
+            TestUtils.waitForCondition(
+                    () -> 
Objects.equals(service.collectGroupState(GROUP).state, "Empty"),
+                    "The group did not become empty as expected.");
 
-        executor.shutdown();
+            cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP, "--group", 
missingGroup};
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
+            ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
 
-        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");
+            String output = 
ToolsTestUtils.grabConsoleOutput(service2::deleteGroups);
+            assertTrue(output.contains("Group '" + missingGroup + "' could not 
be deleted due to:")
+                            && 
output.contains(Errors.GROUP_ID_NOT_FOUND.message())
+                            && output.contains("These consumer groups were 
deleted successfully: '" + GROUP + "'"),
+                    "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";
+    @ClusterTest
+    public void testDeleteWithMixOfSuccessAndError() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
+
+            // run one consumer in the group
+            ConsumerGroupExecutor executor = buildConsumerGroupExecutor(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);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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.");
+            TestUtils.waitForCondition(
+                    () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
+                    "The group did not initialize as expected.");
 
-        executor.shutdown();
+            executor.shutdown();
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
+            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};
+            cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP, "--group", 
missingGroup};
 
-        ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+            ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+            Map<String, Throwable> result = service2.deleteGroups();
+            assertTrue(result.size() == 2 &&
+                            result.containsKey(GROUP) && result.get(GROUP) == 
null &&
+                            result.containsKey(missingGroup) &&
+                            
result.get(missingGroup).getMessage().contains(Errors.GROUP_ID_NOT_FOUND.message()),
+                    "The consumer group deletion did not work as expected");
+        }
+    }
+
+    @ClusterTest
+    public void testDeleteWithUnrecognizedNewConsumerOption() {
+        String[] cgcArgs = new String[]{"--new-consumer", 
"--bootstrap-server", cluster.bootstrapServers(), "--delete", "--group", GROUP};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+    }
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service2::deleteGroups);
-        assertTrue(output.contains("Group '" + missingGroup + "' could not be 
deleted due to:")
-            && output.contains(Errors.GROUP_ID_NOT_FOUND.message())
-            && output.contains("These consumer groups were deleted 
successfully: '" + GROUP + "'"),
-            "The consumer group deletion did not work as expected");
+    private ConsumerGroupExecutor buildConsumerGroupExecutor(String group) {
+        return new ConsumerGroupExecutor(cluster.bootstrapServers(),
+                1,
+                null != group ? group : GROUP,
+                GroupProtocol.CLASSIC.name,
+                TOPIC,
+                RangeAssignor.class.getName(),
+                Optional.empty(),
+                Optional.empty(),
+                false);
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteWithMixOfSuccessAndError(String quorum) throws 
Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
+    private Predicate<String> 
checkGroupState(ConsumerGroupCommand.ConsumerGroupService service, String 
stable) {
+        return groupId -> {
+            try {
+                return 
Objects.equals(service.collectGroupState(groupId).state, stable);
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        };
+    }
 
-        // 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);
+    private Set<NewTopic> buildSingletonTestTopic() {
+        return singleton(new NewTopic(TOPIC, 1, (short) 1));
+    }
 
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
+    ConsumerGroupCommand.ConsumerGroupService getConsumerGroupService(String[] 
args) {
+        ConsumerGroupCommandOptions opts = 
ConsumerGroupCommandOptions.fromArgs(args);
+        return new ConsumerGroupCommand.ConsumerGroupService(
+                opts,
+                Collections.singletonMap(AdminClientConfig.RETRIES_CONFIG, 
Integer.toString(Integer.MAX_VALUE))
+        );
+    }
 
-        executor.shutdown();
+    abstract class AbstractConsumerRunnable implements Runnable {
+        final String broker;
+        final String groupId;
+        final Optional<Properties> customPropsOpt;
+        final boolean syncCommit;
+
+        final Properties props = new Properties();
+        KafkaConsumer<String, String> consumer;
+
+        boolean configured = false;
+
+        public AbstractConsumerRunnable(String broker, String groupId, 
Optional<Properties> customPropsOpt, boolean syncCommit) {
+            this.broker = broker;
+            this.groupId = groupId;
+            this.customPropsOpt = customPropsOpt;
+            this.syncCommit = syncCommit;
+        }
+
+        void configure() {
+            configured = true;
+            configure(props);
+            customPropsOpt.ifPresent(props::putAll);
+            consumer = new KafkaConsumer<>(props);
+        }
+
+        void configure(Properties props) {
+            props.put("bootstrap.servers", broker);
+            props.put("group.id", groupId);
+            props.put("key.deserializer", StringDeserializer.class.getName());
+            props.put("value.deserializer", 
StringDeserializer.class.getName());
+        }
+
+        abstract void subscribe();
+
+        @Override
+        public void run() {
+            assert configured : "Must call configure before use";
+            try {
+                subscribe();
+                while (true) {

Review Comment:
   Could we add a flag to break the loop when calling shutdown?



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -17,279 +17,477 @@
 package org.apache.kafka.tools.consumer.group;
 
 import joptsimple.OptionException;
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterConfigProperty;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.GroupProtocol;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.RangeAssignor;
 import org.apache.kafka.common.errors.GroupIdNotFoundException;
 import org.apache.kafka.common.errors.GroupNotEmptyException;
+import org.apache.kafka.common.errors.WakeupException;
 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.time.Duration;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static java.util.Collections.singleton;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 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";
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL, brokers = 3, serverProperties = {

Review Comment:
   We reduce the number of partitions/replicas so do we need 3 brokers?



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -17,279 +17,477 @@
 package org.apache.kafka.tools.consumer.group;
 
 import joptsimple.OptionException;
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterConfigProperty;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.GroupProtocol;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.RangeAssignor;
 import org.apache.kafka.common.errors.GroupIdNotFoundException;
 import org.apache.kafka.common.errors.GroupNotEmptyException;
+import org.apache.kafka.common.errors.WakeupException;
 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.time.Duration;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static java.util.Collections.singleton;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 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";
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL, brokers = 3, serverProperties = {
+        @ClusterConfigProperty(key = "offsets.topic.num.partitions", value = 
"1"),
+        @ClusterConfigProperty(key = "offsets.topic.replication.factor", value 
= "1"),
+})
+public class DeleteConsumerGroupsTest {
+    private final ClusterInstance cluster;
+    private static final String TOPIC = "foo";
+    private static final String GROUP = "test.group";
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+    public DeleteConsumerGroupsTest(ClusterInstance cluster) {
+        this.cluster = 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");
+    @ClusterTest
+    public void testDeleteWithTopicOption() {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP, "--topic"};
+            assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
+    @ClusterTest
+    public void testDeleteCmdNonExistingGroup() {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", missingGroup};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            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");
+        }
+    }
 
-        // 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);
+    @ClusterTest
+    public void testDeleteNonExistingGroup() {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
+            // note the group to be deleted is a different (non-existing) group
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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");
+        }
+    }
 
-        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");
+    @ClusterTest
+    public void testDeleteCmdNonEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            // run one consumer in the group
+            ConsumerGroupExecutor consumerGroupExecutor = 
buildConsumerGroupExecutor(GROUP);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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 + ")");
+
+            consumerGroupExecutor.shutdown();
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTest
+    public void testDeleteNonEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
 
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            // run one consumer in the group
+            ConsumerGroupExecutor consumerGroupExecutor = 
buildConsumerGroupExecutor(GROUP);
 
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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 + ")");
 
-        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 + ")");
+            consumerGroupExecutor.shutdown();
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTest
+    public void testDeleteCmdEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
 
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            // run one consumer in the group
+            ConsumerGroupExecutor consumerGroupExecutor = 
buildConsumerGroupExecutor(GROUP);
 
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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."
+            );
+
+            consumerGroupExecutor.shutdown();
+
+            TestUtils.waitForCondition(
+                    () -> 
Objects.equals(service.collectGroupState(GROUP).state, "Empty"),
+                    "The group did not become empty 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 + ")");
+            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");
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTest
+    public void testDeleteCmdAllGroups() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+
+            // Create 3 groups with 1 consumer per each
+            Map<String, ConsumerGroupExecutor> groupNameToExecutor = 
IntStream.rangeClosed(1, 3)
+                    .mapToObj(i -> GROUP + i)
+                    .collect(Collectors.toMap(Function.identity(), 
this::buildConsumerGroupExecutor));
+
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--all-groups"};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+            TestUtils.waitForCondition(() ->
+                            new 
HashSet<>(service.listConsumerGroups()).equals(groupNameToExecutor.keySet()) &&
+                                    
groupNameToExecutor.keySet().stream().allMatch(checkGroupState(service, 
"Stable")),
+                    "The group did not initialize as expected.");
+
+            // Shutdown consumers to empty out groups
+            
groupNameToExecutor.values().forEach(ConsumerGroupExecutor::shutdown);
+
+            TestUtils.waitForCondition(() ->
+                            
groupNameToExecutor.keySet().stream().allMatch(checkGroupState(service, 
"Empty")),
+                    "The group did not become empty as expected.");
+
+            String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups).trim();
+            Set<String> expectedGroupsForDeletion = 
groupNameToExecutor.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");
+        }
+    }
 
-        // 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);
+    @ClusterTest
+    public void testDeleteEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
 
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected."
-        );
+            // run one consumer in the group
+            ConsumerGroupExecutor executor = buildConsumerGroupExecutor(GROUP);
 
-        executor.shutdown();
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected."
-        );
+            TestUtils.waitForCondition(
+                    () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
+                    "The group did not initialize as expected.");
+
+            executor.shutdown();
 
-        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");
+            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");
+        }
     }
 
-    @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);
-                }
-            }),
-            "The group did not become empty as expected.");
+    @ClusterTest
+    public void testDeleteCmdWithMixOfSuccessAndError() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
 
-        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());
+            // run one consumer in the group
+            ConsumerGroupExecutor executor = buildConsumerGroupExecutor(GROUP);
 
-        assertTrue(output.matches("Deletion of requested consumer groups (.*) 
was successful.")
-            && Objects.equals(deletedGroupsGrepped, expectedGroupsForDeletion),
-            "The consumer group(s) could not be deleted as expected");
-    }
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+            TestUtils.waitForCondition(
+                    () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
+                    "The group did not initialize as expected.");
 
-        // 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);
+            executor.shutdown();
 
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
+            TestUtils.waitForCondition(
+                    () -> 
Objects.equals(service.collectGroupState(GROUP).state, "Empty"),
+                    "The group did not become empty as expected.");
 
-        executor.shutdown();
+            cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP, "--group", 
missingGroup};
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
+            ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
 
-        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");
+            String output = 
ToolsTestUtils.grabConsoleOutput(service2::deleteGroups);
+            assertTrue(output.contains("Group '" + missingGroup + "' could not 
be deleted due to:")
+                            && 
output.contains(Errors.GROUP_ID_NOT_FOUND.message())
+                            && output.contains("These consumer groups were 
deleted successfully: '" + GROUP + "'"),
+                    "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";
+    @ClusterTest
+    public void testDeleteWithMixOfSuccessAndError() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
+
+            // run one consumer in the group
+            ConsumerGroupExecutor executor = buildConsumerGroupExecutor(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);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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.");
+            TestUtils.waitForCondition(
+                    () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
+                    "The group did not initialize as expected.");
 
-        executor.shutdown();
+            executor.shutdown();
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
+            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};
+            cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP, "--group", 
missingGroup};
 
-        ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+            ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+            Map<String, Throwable> result = service2.deleteGroups();
+            assertTrue(result.size() == 2 &&
+                            result.containsKey(GROUP) && result.get(GROUP) == 
null &&
+                            result.containsKey(missingGroup) &&
+                            
result.get(missingGroup).getMessage().contains(Errors.GROUP_ID_NOT_FOUND.message()),
+                    "The consumer group deletion did not work as expected");
+        }
+    }
+
+    @ClusterTest
+    public void testDeleteWithUnrecognizedNewConsumerOption() {
+        String[] cgcArgs = new String[]{"--new-consumer", 
"--bootstrap-server", cluster.bootstrapServers(), "--delete", "--group", GROUP};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+    }
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service2::deleteGroups);
-        assertTrue(output.contains("Group '" + missingGroup + "' could not be 
deleted due to:")
-            && output.contains(Errors.GROUP_ID_NOT_FOUND.message())
-            && output.contains("These consumer groups were deleted 
successfully: '" + GROUP + "'"),
-            "The consumer group deletion did not work as expected");
+    private ConsumerGroupExecutor buildConsumerGroupExecutor(String group) {
+        return new ConsumerGroupExecutor(cluster.bootstrapServers(),
+                1,
+                null != group ? group : GROUP,
+                GroupProtocol.CLASSIC.name,
+                TOPIC,
+                RangeAssignor.class.getName(),
+                Optional.empty(),
+                Optional.empty(),
+                false);
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteWithMixOfSuccessAndError(String quorum) throws 
Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
+    private Predicate<String> 
checkGroupState(ConsumerGroupCommand.ConsumerGroupService service, String 
stable) {
+        return groupId -> {
+            try {
+                return 
Objects.equals(service.collectGroupState(groupId).state, stable);
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        };
+    }
 
-        // 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);
+    private Set<NewTopic> buildSingletonTestTopic() {
+        return singleton(new NewTopic(TOPIC, 1, (short) 1));
+    }
 
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
+    ConsumerGroupCommand.ConsumerGroupService getConsumerGroupService(String[] 
args) {
+        ConsumerGroupCommandOptions opts = 
ConsumerGroupCommandOptions.fromArgs(args);
+        return new ConsumerGroupCommand.ConsumerGroupService(
+                opts,
+                Collections.singletonMap(AdminClientConfig.RETRIES_CONFIG, 
Integer.toString(Integer.MAX_VALUE))
+        );
+    }
 
-        executor.shutdown();
+    abstract class AbstractConsumerRunnable implements Runnable {
+        final String broker;
+        final String groupId;
+        final Optional<Properties> customPropsOpt;
+        final boolean syncCommit;
+
+        final Properties props = new Properties();
+        KafkaConsumer<String, String> consumer;
+
+        boolean configured = false;
+
+        public AbstractConsumerRunnable(String broker, String groupId, 
Optional<Properties> customPropsOpt, boolean syncCommit) {
+            this.broker = broker;
+            this.groupId = groupId;
+            this.customPropsOpt = customPropsOpt;
+            this.syncCommit = syncCommit;
+        }
+
+        void configure() {
+            configured = true;
+            configure(props);
+            customPropsOpt.ifPresent(props::putAll);
+            consumer = new KafkaConsumer<>(props);
+        }
+
+        void configure(Properties props) {
+            props.put("bootstrap.servers", broker);
+            props.put("group.id", groupId);
+            props.put("key.deserializer", StringDeserializer.class.getName());
+            props.put("value.deserializer", 
StringDeserializer.class.getName());
+        }
+
+        abstract void subscribe();
+
+        @Override
+        public void run() {
+            assert configured : "Must call configure before use";
+            try {
+                subscribe();
+                while (true) {
+                    consumer.poll(Duration.ofMillis(Long.MAX_VALUE));
+                    if (syncCommit)
+                        consumer.commitSync();
+                }
+            } catch (WakeupException e) {
+                // OK
+            } finally {
+                consumer.close();
+            }
+        }
+
+        void shutdown() {
+            consumer.wakeup();
+        }
+    }
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
+    class ConsumerRunnable extends AbstractConsumerRunnable {
+        final String topic;
+        final String groupProtocol;
+        final String strategy;
+        final Optional<String> remoteAssignor;
+
+        public ConsumerRunnable(String broker, String groupId, String 
groupProtocol, String topic, String strategy,
+                                Optional<String> remoteAssignor, 
Optional<Properties> customPropsOpt, boolean syncCommit) {
+            super(broker, groupId, customPropsOpt, syncCommit);
+
+            this.topic = topic;
+            this.groupProtocol = groupProtocol;
+            this.strategy = strategy;
+            this.remoteAssignor = remoteAssignor;
+        }
+
+        @Override
+        void configure(Properties props) {
+            super.configure(props);
+            props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
+            if 
(groupProtocol.toUpperCase(Locale.ROOT).equals(GroupProtocol.CONSUMER.toString()))
 {
+                remoteAssignor.ifPresent(assignor -> 
props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, assignor));
+            } else {
+                props.put("partition.assignment.strategy", strategy);
+            }
+        }
+
+        @Override
+        void subscribe() {
+            consumer.subscribe(Collections.singleton(topic));
+        }
+    }
 
-        cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--group", 
missingGroup};
 
-        ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
-        Map<String, Throwable> result = service2.deleteGroups();
-        assertTrue(result.size() == 2 &&
-                result.containsKey(GROUP) && result.get(GROUP) == null &&
-                result.containsKey(missingGroup) &&
-                
result.get(missingGroup).getMessage().contains(Errors.GROUP_ID_NOT_FOUND.message()),
-            "The consumer group deletion did not work as expected");
+    class AbstractConsumerGroupExecutor {
+        final int numThreads;
+        final ExecutorService executor;
+        final List<AbstractConsumerRunnable> consumers = new ArrayList<>();
+
+        public AbstractConsumerGroupExecutor(int numThreads) {
+            this.numThreads = numThreads;
+            this.executor = Executors.newFixedThreadPool(numThreads);
+        }
+
+        void submit(AbstractConsumerRunnable consumerThread) {
+            consumers.add(consumerThread);
+            executor.submit(consumerThread);
+        }
+
+        void shutdown() {

Review Comment:
   We can change this to `close` and make `AbstractConsumerGroupExecutor` 
extend `Closeable`. That allow us to use `try-with-resources`



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -17,279 +17,477 @@
 package org.apache.kafka.tools.consumer.group;
 
 import joptsimple.OptionException;
+import kafka.test.ClusterInstance;
+import kafka.test.annotation.ClusterConfigProperty;
+import kafka.test.annotation.ClusterTest;
+import kafka.test.annotation.ClusterTestDefaults;
+import kafka.test.annotation.Type;
+import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.GroupProtocol;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.RangeAssignor;
 import org.apache.kafka.common.errors.GroupIdNotFoundException;
 import org.apache.kafka.common.errors.GroupNotEmptyException;
+import org.apache.kafka.common.errors.WakeupException;
 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.time.Duration;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static java.util.Collections.singleton;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 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";
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL, brokers = 3, serverProperties = {
+        @ClusterConfigProperty(key = "offsets.topic.num.partitions", value = 
"1"),
+        @ClusterConfigProperty(key = "offsets.topic.replication.factor", value 
= "1"),
+})
+public class DeleteConsumerGroupsTest {
+    private final ClusterInstance cluster;
+    private static final String TOPIC = "foo";
+    private static final String GROUP = "test.group";
 
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+    public DeleteConsumerGroupsTest(ClusterInstance cluster) {
+        this.cluster = 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");
+    @ClusterTest
+    public void testDeleteWithTopicOption() {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP, "--topic"};
+            assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonExistingGroup(String quorum) {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
+    @ClusterTest
+    public void testDeleteCmdNonExistingGroup() {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", missingGroup};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            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");
+        }
+    }
 
-        // 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);
+    @ClusterTest
+    public void testDeleteNonExistingGroup() {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
+            // note the group to be deleted is a different (non-existing) group
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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");
+        }
+    }
 
-        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");
+    @ClusterTest
+    public void testDeleteCmdNonEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            // run one consumer in the group
+            ConsumerGroupExecutor consumerGroupExecutor = 
buildConsumerGroupExecutor(GROUP);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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 + ")");
+
+            consumerGroupExecutor.shutdown();
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTest
+    public void testDeleteNonEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
 
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            // run one consumer in the group
+            ConsumerGroupExecutor consumerGroupExecutor = 
buildConsumerGroupExecutor(GROUP);
 
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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 + ")");
 
-        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 + ")");
+            consumerGroupExecutor.shutdown();
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteNonEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTest
+    public void testDeleteCmdEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
 
-        // run one consumer in the group
-        addConsumerGroupExecutor(1);
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
-        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+            // run one consumer in the group
+            ConsumerGroupExecutor consumerGroupExecutor = 
buildConsumerGroupExecutor(GROUP);
 
-        TestUtils.waitForCondition(
-            () -> service.collectGroupMembers(GROUP, 
false).getValue().get().size() == 1,
-            "The group did not initialize as expected."
-        );
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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."
+            );
+
+            consumerGroupExecutor.shutdown();
+
+            TestUtils.waitForCondition(
+                    () -> 
Objects.equals(service.collectGroupState(GROUP).state, "Empty"),
+                    "The group did not become empty 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 + ")");
+            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");
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteCmdEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+    @ClusterTest
+    public void testDeleteCmdAllGroups() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+
+            // Create 3 groups with 1 consumer per each
+            Map<String, ConsumerGroupExecutor> groupNameToExecutor = 
IntStream.rangeClosed(1, 3)
+                    .mapToObj(i -> GROUP + i)
+                    .collect(Collectors.toMap(Function.identity(), 
this::buildConsumerGroupExecutor));
+
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--all-groups"};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+            TestUtils.waitForCondition(() ->
+                            new 
HashSet<>(service.listConsumerGroups()).equals(groupNameToExecutor.keySet()) &&
+                                    
groupNameToExecutor.keySet().stream().allMatch(checkGroupState(service, 
"Stable")),
+                    "The group did not initialize as expected.");
+
+            // Shutdown consumers to empty out groups
+            
groupNameToExecutor.values().forEach(ConsumerGroupExecutor::shutdown);
+
+            TestUtils.waitForCondition(() ->
+                            
groupNameToExecutor.keySet().stream().allMatch(checkGroupState(service, 
"Empty")),
+                    "The group did not become empty as expected.");
+
+            String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteGroups).trim();
+            Set<String> expectedGroupsForDeletion = 
groupNameToExecutor.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");
+        }
+    }
 
-        // 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);
+    @ClusterTest
+    public void testDeleteEmptyGroup() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
 
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected."
-        );
+            // run one consumer in the group
+            ConsumerGroupExecutor executor = buildConsumerGroupExecutor(GROUP);
 
-        executor.shutdown();
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected."
-        );
+            TestUtils.waitForCondition(
+                    () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
+                    "The group did not initialize as expected.");
+
+            executor.shutdown();
 
-        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");
+            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");
+        }
     }
 
-    @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);
-                }
-            }),
-            "The group did not become empty as expected.");
+    @ClusterTest
+    public void testDeleteCmdWithMixOfSuccessAndError() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
 
-        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());
+            // run one consumer in the group
+            ConsumerGroupExecutor executor = buildConsumerGroupExecutor(GROUP);
 
-        assertTrue(output.matches("Deletion of requested consumer groups (.*) 
was successful.")
-            && Objects.equals(deletedGroupsGrepped, expectedGroupsForDeletion),
-            "The consumer group(s) could not be deleted as expected");
-    }
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP};
+            ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteEmptyGroup(String quorum) throws Exception {
-        createOffsetsTopic(listenerName(), new Properties());
+            TestUtils.waitForCondition(
+                    () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
+                    "The group did not initialize as expected.");
 
-        // 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);
+            executor.shutdown();
 
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
+            TestUtils.waitForCondition(
+                    () -> 
Objects.equals(service.collectGroupState(GROUP).state, "Empty"),
+                    "The group did not become empty as expected.");
 
-        executor.shutdown();
+            cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP, "--group", 
missingGroup};
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
+            ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
 
-        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");
+            String output = 
ToolsTestUtils.grabConsoleOutput(service2::deleteGroups);
+            assertTrue(output.contains("Group '" + missingGroup + "' could not 
be deleted due to:")
+                            && 
output.contains(Errors.GROUP_ID_NOT_FOUND.message())
+                            && output.contains("These consumer groups were 
deleted successfully: '" + GROUP + "'"),
+                    "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";
+    @ClusterTest
+    public void testDeleteWithMixOfSuccessAndError() throws Exception {
+        try (Admin admin = cluster.createAdminClient()) {
+            admin.createTopics(buildSingletonTestTopic());
+            String missingGroup = "missing.group";
+
+            // run one consumer in the group
+            ConsumerGroupExecutor executor = buildConsumerGroupExecutor(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);
+            String[] cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--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.");
+            TestUtils.waitForCondition(
+                    () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
+                    "The group did not initialize as expected.");
 
-        executor.shutdown();
+            executor.shutdown();
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
+            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};
+            cgcArgs = new String[]{"--bootstrap-server", 
cluster.bootstrapServers(), "--delete", "--group", GROUP, "--group", 
missingGroup};
 
-        ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+            ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
+            Map<String, Throwable> result = service2.deleteGroups();
+            assertTrue(result.size() == 2 &&
+                            result.containsKey(GROUP) && result.get(GROUP) == 
null &&
+                            result.containsKey(missingGroup) &&
+                            
result.get(missingGroup).getMessage().contains(Errors.GROUP_ID_NOT_FOUND.message()),
+                    "The consumer group deletion did not work as expected");
+        }
+    }
+
+    @ClusterTest
+    public void testDeleteWithUnrecognizedNewConsumerOption() {
+        String[] cgcArgs = new String[]{"--new-consumer", 
"--bootstrap-server", cluster.bootstrapServers(), "--delete", "--group", GROUP};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+    }
 
-        String output = 
ToolsTestUtils.grabConsoleOutput(service2::deleteGroups);
-        assertTrue(output.contains("Group '" + missingGroup + "' could not be 
deleted due to:")
-            && output.contains(Errors.GROUP_ID_NOT_FOUND.message())
-            && output.contains("These consumer groups were deleted 
successfully: '" + GROUP + "'"),
-            "The consumer group deletion did not work as expected");
+    private ConsumerGroupExecutor buildConsumerGroupExecutor(String group) {
+        return new ConsumerGroupExecutor(cluster.bootstrapServers(),
+                1,
+                null != group ? group : GROUP,
+                GroupProtocol.CLASSIC.name,
+                TOPIC,
+                RangeAssignor.class.getName(),
+                Optional.empty(),
+                Optional.empty(),
+                false);
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteWithMixOfSuccessAndError(String quorum) throws 
Exception {
-        createOffsetsTopic(listenerName(), new Properties());
-        String missingGroup = "missing.group";
+    private Predicate<String> 
checkGroupState(ConsumerGroupCommand.ConsumerGroupService service, String 
stable) {
+        return groupId -> {
+            try {
+                return 
Objects.equals(service.collectGroupState(groupId).state, stable);
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        };
+    }
 
-        // 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);
+    private Set<NewTopic> buildSingletonTestTopic() {
+        return singleton(new NewTopic(TOPIC, 1, (short) 1));
+    }
 
-        TestUtils.waitForCondition(
-            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state, "Stable"),
-            "The group did not initialize as expected.");
+    ConsumerGroupCommand.ConsumerGroupService getConsumerGroupService(String[] 
args) {
+        ConsumerGroupCommandOptions opts = 
ConsumerGroupCommandOptions.fromArgs(args);
+        return new ConsumerGroupCommand.ConsumerGroupService(
+                opts,
+                Collections.singletonMap(AdminClientConfig.RETRIES_CONFIG, 
Integer.toString(Integer.MAX_VALUE))
+        );
+    }
 
-        executor.shutdown();
+    abstract class AbstractConsumerRunnable implements Runnable {
+        final String broker;
+        final String groupId;
+        final Optional<Properties> customPropsOpt;
+        final boolean syncCommit;
+
+        final Properties props = new Properties();
+        KafkaConsumer<String, String> consumer;
+
+        boolean configured = false;
+
+        public AbstractConsumerRunnable(String broker, String groupId, 
Optional<Properties> customPropsOpt, boolean syncCommit) {
+            this.broker = broker;
+            this.groupId = groupId;
+            this.customPropsOpt = customPropsOpt;
+            this.syncCommit = syncCommit;
+        }
+
+        void configure() {
+            configured = true;
+            configure(props);
+            customPropsOpt.ifPresent(props::putAll);
+            consumer = new KafkaConsumer<>(props);
+        }
+
+        void configure(Properties props) {
+            props.put("bootstrap.servers", broker);
+            props.put("group.id", groupId);
+            props.put("key.deserializer", StringDeserializer.class.getName());
+            props.put("value.deserializer", 
StringDeserializer.class.getName());
+        }
+
+        abstract void subscribe();
+
+        @Override
+        public void run() {
+            assert configured : "Must call configure before use";
+            try {
+                subscribe();
+                while (true) {
+                    consumer.poll(Duration.ofMillis(Long.MAX_VALUE));
+                    if (syncCommit)
+                        consumer.commitSync();
+                }
+            } catch (WakeupException e) {
+                // OK
+            } finally {
+                consumer.close();
+            }
+        }
+
+        void shutdown() {
+            consumer.wakeup();
+        }
+    }
 
-        TestUtils.waitForCondition(
-            () -> Objects.equals(service.collectGroupState(GROUP).state, 
"Empty"),
-            "The group did not become empty as expected.");
+    class ConsumerRunnable extends AbstractConsumerRunnable {
+        final String topic;
+        final String groupProtocol;
+        final String strategy;
+        final Optional<String> remoteAssignor;
+
+        public ConsumerRunnable(String broker, String groupId, String 
groupProtocol, String topic, String strategy,
+                                Optional<String> remoteAssignor, 
Optional<Properties> customPropsOpt, boolean syncCommit) {
+            super(broker, groupId, customPropsOpt, syncCommit);
+
+            this.topic = topic;
+            this.groupProtocol = groupProtocol;
+            this.strategy = strategy;
+            this.remoteAssignor = remoteAssignor;
+        }
+
+        @Override
+        void configure(Properties props) {
+            super.configure(props);
+            props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol);
+            if 
(groupProtocol.toUpperCase(Locale.ROOT).equals(GroupProtocol.CONSUMER.toString()))
 {
+                remoteAssignor.ifPresent(assignor -> 
props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, assignor));
+            } else {
+                props.put("partition.assignment.strategy", strategy);
+            }
+        }
+
+        @Override
+        void subscribe() {
+            consumer.subscribe(Collections.singleton(topic));
+        }
+    }
 
-        cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--group", 
missingGroup};
 
-        ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs);
-        Map<String, Throwable> result = service2.deleteGroups();
-        assertTrue(result.size() == 2 &&
-                result.containsKey(GROUP) && result.get(GROUP) == null &&
-                result.containsKey(missingGroup) &&
-                
result.get(missingGroup).getMessage().contains(Errors.GROUP_ID_NOT_FOUND.message()),
-            "The consumer group deletion did not work as expected");
+    class AbstractConsumerGroupExecutor {
+        final int numThreads;
+        final ExecutorService executor;
+        final List<AbstractConsumerRunnable> consumers = new ArrayList<>();
+
+        public AbstractConsumerGroupExecutor(int numThreads) {
+            this.numThreads = numThreads;
+            this.executor = Executors.newFixedThreadPool(numThreads);
+        }
+
+        void submit(AbstractConsumerRunnable consumerThread) {
+            consumers.add(consumerThread);
+            executor.submit(consumerThread);
+        }
+
+        void shutdown() {
+            consumers.forEach(AbstractConsumerRunnable::shutdown);
+            executor.shutdown();
+            try {
+                executor.awaitTermination(5000, TimeUnit.MILLISECONDS);
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
     }
 
-    @ParameterizedTest
-    @ValueSource(strings = {"zk", "kraft"})
-    public void testDeleteWithUnrecognizedNewConsumerOption(String quorum) {
-        String[] cgcArgs = new String[]{"--new-consumer", 
"--bootstrap-server", bootstrapServers(listenerName()), "--delete", "--group", 
GROUP};
-        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+    class ConsumerGroupExecutor extends AbstractConsumerGroupExecutor {

Review Comment:
   `AbstractConsumerGroupExecutor` is extended by `ConsumerGroupExecutor` only, 
so we can merge them together.



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