showuon commented on code in PR #15256:
URL: https://github.com/apache/kafka/pull/15256#discussion_r1471122417


##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.tools.consumer.group;
+
+import kafka.admin.ConsumerGroupCommand;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.config.Defaults;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+public class DeleteOffsetsConsumerGroupCommandIntegrationTest extends 
ConsumerGroupCommandTest {
+    String[] getArgs(String group, String topic) {
+        return new String[] {
+            "--bootstrap-server", bootstrapServers(listenerName()),
+            "--delete-offsets",
+            "--group", group,
+            "--topic", topic
+        };
+    }
+
+    @Test

Review Comment:
   Should we add these?
   ```
       @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
       @ValueSource(strings = {"zk", "kraft"})
   ```
   
   Same for below tests.



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.tools.consumer.group;
+
+import joptsimple.OptionException;
+import kafka.admin.ConsumerGroupCommand;
+import org.apache.kafka.clients.admin.ConsumerGroupListing;
+import org.apache.kafka.common.ConsumerGroupState;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.Optional;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class ListConsumerGroupTest extends ConsumerGroupCommandTest {
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListConsumerGroups(String quorum) throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+        scala.collection.Set<String> expectedGroups = set(Arrays.asList(GROUP, 
simpleGroup));
+        final scala.collection.Set[] foundGroups = new scala.collection.Set[1];
+        TestUtils.waitForCondition(() -> {
+            foundGroups[0] = service.listConsumerGroups().toSet();
+            return Objects.equals(expectedGroups, foundGroups[0]);
+        }, "Expected --list to show groups " + expectedGroups + ", but found " 
+ foundGroups[0] + ".");
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListWithUnrecognizedNewConsumerOption() {
+        String[] cgcArgs = new String[]{"--new-consumer", 
"--bootstrap-server", bootstrapServers(listenerName()), "--list"};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListConsumerGroupsWithStates() throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list", "--state"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        scala.collection.Set<ConsumerGroupListing> expectedListing = 
set(Arrays.asList(
+            new ConsumerGroupListing(simpleGroup, true, 
Optional.of(ConsumerGroupState.EMPTY)),
+            new ConsumerGroupListing(GROUP, false, 
Optional.of(ConsumerGroupState.STABLE))));
+
+        final scala.collection.Set[] foundListing = new 
scala.collection.Set[1];

Review Comment:
   Could we use java collection here? 



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.tools.consumer.group;
+
+import joptsimple.OptionException;
+import kafka.admin.ConsumerGroupCommand;
+import org.apache.kafka.clients.admin.ConsumerGroupListing;
+import org.apache.kafka.common.ConsumerGroupState;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.Optional;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class ListConsumerGroupTest extends ConsumerGroupCommandTest {
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListConsumerGroups(String quorum) throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+        scala.collection.Set<String> expectedGroups = set(Arrays.asList(GROUP, 
simpleGroup));
+        final scala.collection.Set[] foundGroups = new scala.collection.Set[1];
+        TestUtils.waitForCondition(() -> {
+            foundGroups[0] = service.listConsumerGroups().toSet();
+            return Objects.equals(expectedGroups, foundGroups[0]);
+        }, "Expected --list to show groups " + expectedGroups + ", but found " 
+ foundGroups[0] + ".");
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListWithUnrecognizedNewConsumerOption() {
+        String[] cgcArgs = new String[]{"--new-consumer", 
"--bootstrap-server", bootstrapServers(listenerName()), "--list"};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListConsumerGroupsWithStates() throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list", "--state"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        scala.collection.Set<ConsumerGroupListing> expectedListing = 
set(Arrays.asList(
+            new ConsumerGroupListing(simpleGroup, true, 
Optional.of(ConsumerGroupState.EMPTY)),
+            new ConsumerGroupListing(GROUP, false, 
Optional.of(ConsumerGroupState.STABLE))));
+
+        final scala.collection.Set[] foundListing = new 
scala.collection.Set[1];
+        TestUtils.waitForCondition(() -> {
+            foundListing[0] = 
service.listConsumerGroupsWithState(set(Arrays.asList(ConsumerGroupState.values()))).toSet();
+            return Objects.equals(expectedListing, foundListing[0]);
+        }, "Expected to show groups " + expectedListing + ", but found " + 
foundListing[0]);
+
+        scala.collection.Set<ConsumerGroupListing> expectedListingStable = 
set(Collections.singleton(
+            new ConsumerGroupListing(GROUP, false, 
Optional.of(ConsumerGroupState.STABLE))));
+
+        foundListing[0] = null;
+
+        TestUtils.waitForCondition(() -> {
+            foundListing[0] = 
service.listConsumerGroupsWithState(set(Collections.singleton(ConsumerGroupState.STABLE))).toSet();
+            return Objects.equals(expectedListingStable, foundListing[0]);
+        }, "Expected to show groups " + expectedListingStable + ", but found " 
+ foundListing[0]);
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testConsumerGroupStatesFromString(String quorum) {
+        scala.collection.Set<ConsumerGroupState> result = 
ConsumerGroupCommand.consumerGroupStatesFromString("Stable");
+        assertEquals(set(Collections.singleton(ConsumerGroupState.STABLE)), 
result);
+
+        result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable, 
PreparingRebalance");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE, 
ConsumerGroupState.PREPARING_REBALANCE)), result);
+
+        result = 
ConsumerGroupCommand.consumerGroupStatesFromString("Dead,CompletingRebalance,");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.DEAD, 
ConsumerGroupState.COMPLETING_REBALANCE)), result);
+
+        result = ConsumerGroupCommand.consumerGroupStatesFromString("stable");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE)), result);
+
+        result = ConsumerGroupCommand.consumerGroupStatesFromString("stable, 
assigning");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE, 
ConsumerGroupState.ASSIGNING)), result);
+
+        result = 
ConsumerGroupCommand.consumerGroupStatesFromString("dead,reconciling,");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.DEAD, 
ConsumerGroupState.RECONCILING)), result);
+
+        assertThrows(IllegalArgumentException.class, () -> 
ConsumerGroupCommand.consumerGroupStatesFromString("bad, wrong"));
+
+        assertThrows(IllegalArgumentException.class, () -> 
ConsumerGroupCommand.consumerGroupStatesFromString("  bad, Stable"));
+
+        assertThrows(IllegalArgumentException.class, () -> 
ConsumerGroupCommand.consumerGroupStatesFromString("   ,   ,"));

Review Comment:
   It seems 1 test is missing.



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.tools.consumer.group;
+
+import joptsimple.OptionException;
+import kafka.admin.ConsumerGroupCommand;
+import org.apache.kafka.clients.admin.ConsumerGroupListing;
+import org.apache.kafka.common.ConsumerGroupState;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.Optional;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class ListConsumerGroupTest extends ConsumerGroupCommandTest {
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListConsumerGroups(String quorum) throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+        scala.collection.Set<String> expectedGroups = set(Arrays.asList(GROUP, 
simpleGroup));
+        final scala.collection.Set[] foundGroups = new scala.collection.Set[1];
+        TestUtils.waitForCondition(() -> {
+            foundGroups[0] = service.listConsumerGroups().toSet();
+            return Objects.equals(expectedGroups, foundGroups[0]);
+        }, "Expected --list to show groups " + expectedGroups + ", but found " 
+ foundGroups[0] + ".");
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListWithUnrecognizedNewConsumerOption() {
+        String[] cgcArgs = new String[]{"--new-consumer", 
"--bootstrap-server", bootstrapServers(listenerName()), "--list"};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListConsumerGroupsWithStates() throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list", "--state"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        scala.collection.Set<ConsumerGroupListing> expectedListing = 
set(Arrays.asList(
+            new ConsumerGroupListing(simpleGroup, true, 
Optional.of(ConsumerGroupState.EMPTY)),
+            new ConsumerGroupListing(GROUP, false, 
Optional.of(ConsumerGroupState.STABLE))));
+
+        final scala.collection.Set[] foundListing = new 
scala.collection.Set[1];
+        TestUtils.waitForCondition(() -> {
+            foundListing[0] = 
service.listConsumerGroupsWithState(set(Arrays.asList(ConsumerGroupState.values()))).toSet();
+            return Objects.equals(expectedListing, foundListing[0]);
+        }, "Expected to show groups " + expectedListing + ", but found " + 
foundListing[0]);
+
+        scala.collection.Set<ConsumerGroupListing> expectedListingStable = 
set(Collections.singleton(
+            new ConsumerGroupListing(GROUP, false, 
Optional.of(ConsumerGroupState.STABLE))));
+
+        foundListing[0] = null;
+
+        TestUtils.waitForCondition(() -> {
+            foundListing[0] = 
service.listConsumerGroupsWithState(set(Collections.singleton(ConsumerGroupState.STABLE))).toSet();
+            return Objects.equals(expectedListingStable, foundListing[0]);
+        }, "Expected to show groups " + expectedListingStable + ", but found " 
+ foundListing[0]);

Review Comment:
   This is hard to read. `foundListing[0] = null` doesn't make sense to me. 
Please improve it. Thanks.



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.tools.consumer.group;
+
+import joptsimple.OptionException;
+import kafka.admin.ConsumerGroupCommand;
+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.protocol.Errors;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME;
+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(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @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(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteCmdNonExistingGroup(String quorum) {
+        createOffsetsTopic(listenerName(), new Properties());
+        String missingGroup = "missing.group";
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+            service.deleteGroups();
+            return null;
+        });
+        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");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteNonExistingGroup(String quorum) {
+        createOffsetsTopic(listenerName(), new Properties());
+        String missingGroup = "missing.group";
+
+        // note the group to be deleted is a different (non-existing) group
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        scala.collection.Map<String, Throwable> result = 
service.deleteGroups();
+        assertTrue(result.size() == 1 && result.contains(missingGroup) && 
result.get(missingGroup).get().getCause() instanceof GroupIdNotFoundException,
+            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteCmdNonEmptyGroup(String quorum) throws Exception {
+        createOffsetsTopic(listenerName(), new Properties());
+
+        // run one consumer in the group
+        addConsumerGroupExecutor(1);
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        TestUtils.waitForCondition(
+            () -> service.collectGroupMembers(GROUP, false)._2.get().size() == 
1,
+            "The group did not initialize as expected."
+        );
+
+        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+            service.deleteGroups();
+            return null;

Review Comment:
   Will it have compiling failure after removing `return null`?



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.tools.consumer.group;
+
+import joptsimple.OptionException;
+import kafka.admin.ConsumerGroupCommand;
+import org.apache.kafka.clients.admin.ConsumerGroupListing;
+import org.apache.kafka.common.ConsumerGroupState;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.Optional;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class ListConsumerGroupTest extends ConsumerGroupCommandTest {
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListConsumerGroups(String quorum) throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+        scala.collection.Set<String> expectedGroups = set(Arrays.asList(GROUP, 
simpleGroup));
+        final scala.collection.Set[] foundGroups = new scala.collection.Set[1];
+        TestUtils.waitForCondition(() -> {
+            foundGroups[0] = service.listConsumerGroups().toSet();
+            return Objects.equals(expectedGroups, foundGroups[0]);
+        }, "Expected --list to show groups " + expectedGroups + ", but found " 
+ foundGroups[0] + ".");
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListWithUnrecognizedNewConsumerOption() {
+        String[] cgcArgs = new String[]{"--new-consumer", 
"--bootstrap-server", bootstrapServers(listenerName()), "--list"};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListConsumerGroupsWithStates() throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list", "--state"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        scala.collection.Set<ConsumerGroupListing> expectedListing = 
set(Arrays.asList(
+            new ConsumerGroupListing(simpleGroup, true, 
Optional.of(ConsumerGroupState.EMPTY)),
+            new ConsumerGroupListing(GROUP, false, 
Optional.of(ConsumerGroupState.STABLE))));
+
+        final scala.collection.Set[] foundListing = new 
scala.collection.Set[1];
+        TestUtils.waitForCondition(() -> {
+            foundListing[0] = 
service.listConsumerGroupsWithState(set(Arrays.asList(ConsumerGroupState.values()))).toSet();
+            return Objects.equals(expectedListing, foundListing[0]);
+        }, "Expected to show groups " + expectedListing + ", but found " + 
foundListing[0]);
+
+        scala.collection.Set<ConsumerGroupListing> expectedListingStable = 
set(Collections.singleton(
+            new ConsumerGroupListing(GROUP, false, 
Optional.of(ConsumerGroupState.STABLE))));
+
+        foundListing[0] = null;
+
+        TestUtils.waitForCondition(() -> {
+            foundListing[0] = 
service.listConsumerGroupsWithState(set(Collections.singleton(ConsumerGroupState.STABLE))).toSet();
+            return Objects.equals(expectedListingStable, foundListing[0]);
+        }, "Expected to show groups " + expectedListingStable + ", but found " 
+ foundListing[0]);
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testConsumerGroupStatesFromString(String quorum) {
+        scala.collection.Set<ConsumerGroupState> result = 
ConsumerGroupCommand.consumerGroupStatesFromString("Stable");
+        assertEquals(set(Collections.singleton(ConsumerGroupState.STABLE)), 
result);
+
+        result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable, 
PreparingRebalance");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE, 
ConsumerGroupState.PREPARING_REBALANCE)), result);
+
+        result = 
ConsumerGroupCommand.consumerGroupStatesFromString("Dead,CompletingRebalance,");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.DEAD, 
ConsumerGroupState.COMPLETING_REBALANCE)), result);
+
+        result = ConsumerGroupCommand.consumerGroupStatesFromString("stable");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE)), result);
+
+        result = ConsumerGroupCommand.consumerGroupStatesFromString("stable, 
assigning");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE, 
ConsumerGroupState.ASSIGNING)), result);
+
+        result = 
ConsumerGroupCommand.consumerGroupStatesFromString("dead,reconciling,");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.DEAD, 
ConsumerGroupState.RECONCILING)), result);

Review Comment:
   I didn't see these tests in scala. Did you add them?



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.tools.consumer.group;
+
+import joptsimple.OptionException;
+import kafka.admin.ConsumerGroupCommand;
+import org.apache.kafka.clients.admin.ConsumerGroupListing;
+import org.apache.kafka.common.ConsumerGroupState;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.Optional;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class ListConsumerGroupTest extends ConsumerGroupCommandTest {
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListConsumerGroups(String quorum) throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+        scala.collection.Set<String> expectedGroups = set(Arrays.asList(GROUP, 
simpleGroup));
+        final scala.collection.Set[] foundGroups = new scala.collection.Set[1];
+        TestUtils.waitForCondition(() -> {
+            foundGroups[0] = service.listConsumerGroups().toSet();
+            return Objects.equals(expectedGroups, foundGroups[0]);
+        }, "Expected --list to show groups " + expectedGroups + ", but found " 
+ foundGroups[0] + ".");
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListWithUnrecognizedNewConsumerOption() {
+        String[] cgcArgs = new String[]{"--new-consumer", 
"--bootstrap-server", bootstrapServers(listenerName()), "--list"};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListConsumerGroupsWithStates() throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list", "--state"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        scala.collection.Set<ConsumerGroupListing> expectedListing = 
set(Arrays.asList(
+            new ConsumerGroupListing(simpleGroup, true, 
Optional.of(ConsumerGroupState.EMPTY)),
+            new ConsumerGroupListing(GROUP, false, 
Optional.of(ConsumerGroupState.STABLE))));
+
+        final scala.collection.Set[] foundListing = new 
scala.collection.Set[1];
+        TestUtils.waitForCondition(() -> {
+            foundListing[0] = 
service.listConsumerGroupsWithState(set(Arrays.asList(ConsumerGroupState.values()))).toSet();
+            return Objects.equals(expectedListing, foundListing[0]);
+        }, "Expected to show groups " + expectedListing + ", but found " + 
foundListing[0]);
+
+        scala.collection.Set<ConsumerGroupListing> expectedListingStable = 
set(Collections.singleton(
+            new ConsumerGroupListing(GROUP, false, 
Optional.of(ConsumerGroupState.STABLE))));
+
+        foundListing[0] = null;
+
+        TestUtils.waitForCondition(() -> {
+            foundListing[0] = 
service.listConsumerGroupsWithState(set(Collections.singleton(ConsumerGroupState.STABLE))).toSet();
+            return Objects.equals(expectedListingStable, foundListing[0]);
+        }, "Expected to show groups " + expectedListingStable + ", but found " 
+ foundListing[0]);
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testConsumerGroupStatesFromString(String quorum) {
+        scala.collection.Set<ConsumerGroupState> result = 
ConsumerGroupCommand.consumerGroupStatesFromString("Stable");
+        assertEquals(set(Collections.singleton(ConsumerGroupState.STABLE)), 
result);
+
+        result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable, 
PreparingRebalance");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE, 
ConsumerGroupState.PREPARING_REBALANCE)), result);
+
+        result = 
ConsumerGroupCommand.consumerGroupStatesFromString("Dead,CompletingRebalance,");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.DEAD, 
ConsumerGroupState.COMPLETING_REBALANCE)), result);
+
+        result = ConsumerGroupCommand.consumerGroupStatesFromString("stable");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE)), result);
+
+        result = ConsumerGroupCommand.consumerGroupStatesFromString("stable, 
assigning");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE, 
ConsumerGroupState.ASSIGNING)), result);
+
+        result = 
ConsumerGroupCommand.consumerGroupStatesFromString("dead,reconciling,");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.DEAD, 
ConsumerGroupState.RECONCILING)), result);
+
+        assertThrows(IllegalArgumentException.class, () -> 
ConsumerGroupCommand.consumerGroupStatesFromString("bad, wrong"));
+
+        assertThrows(IllegalArgumentException.class, () -> 
ConsumerGroupCommand.consumerGroupStatesFromString("  bad, Stable"));
+
+        assertThrows(IllegalArgumentException.class, () -> 
ConsumerGroupCommand.consumerGroupStatesFromString("   ,   ,"));
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListGroupCommand(String quorum) throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+        final String[] out = {""};

Review Comment:
   Why can't we declare as `String` directly? 
   `final String out = ""`?



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.tools.consumer.group;
+
+import joptsimple.OptionException;
+import kafka.admin.ConsumerGroupCommand;
+import org.apache.kafka.clients.admin.ConsumerGroupListing;
+import org.apache.kafka.common.ConsumerGroupState;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.Optional;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class ListConsumerGroupTest extends ConsumerGroupCommandTest {
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListConsumerGroups(String quorum) throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+        scala.collection.Set<String> expectedGroups = set(Arrays.asList(GROUP, 
simpleGroup));
+        final scala.collection.Set[] foundGroups = new scala.collection.Set[1];

Review Comment:
   Could we use java set? It's weird we compare the 1st element only below. 
That is, you create a Set, and then, you just verify the 1st element of the 
Set. Then, when do you need a set if only 1 element?



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.tools.consumer.group;
+
+import joptsimple.OptionException;
+import kafka.admin.ConsumerGroupCommand;
+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.protocol.Errors;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME;
+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(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @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(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteCmdNonExistingGroup(String quorum) {
+        createOffsetsTopic(listenerName(), new Properties());
+        String missingGroup = "missing.group";
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+            service.deleteGroups();
+            return null;
+        });
+        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");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteNonExistingGroup(String quorum) {
+        createOffsetsTopic(listenerName(), new Properties());
+        String missingGroup = "missing.group";
+
+        // note the group to be deleted is a different (non-existing) group
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        scala.collection.Map<String, Throwable> result = 
service.deleteGroups();
+        assertTrue(result.size() == 1 && result.contains(missingGroup) && 
result.get(missingGroup).get().getCause() instanceof GroupIdNotFoundException,
+            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteCmdNonEmptyGroup(String quorum) throws Exception {
+        createOffsetsTopic(listenerName(), new Properties());
+
+        // run one consumer in the group
+        addConsumerGroupExecutor(1);
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        TestUtils.waitForCondition(
+            () -> service.collectGroupMembers(GROUP, false)._2.get().size() == 
1,
+            "The group did not initialize as expected."
+        );
+
+        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+            service.deleteGroups();
+            return null;
+        });
+        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 + ")");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteNonEmptyGroup(String quorum) throws Exception {
+        createOffsetsTopic(listenerName(), new Properties());
+
+        // run one consumer in the group
+        addConsumerGroupExecutor(1);
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        TestUtils.waitForCondition(
+            () -> service.collectGroupMembers(GROUP, false)._2.get().size() == 
1,
+            "The group did not initialize as expected."
+        );
+
+        scala.collection.Map<String, Throwable> result = 
service.deleteGroups();
+        assertNotNull(result.get(GROUP).get(),
+            "Group was deleted successfully, but it shouldn't have been. 
Result was:(" + result + ")");
+        assertTrue(result.size() == 1 && result.contains(GROUP) && 
result.get(GROUP).get().getCause() instanceof GroupNotEmptyException,
+            "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not 
detected while deleting consumer group. Result was:(" + result + ")");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteCmdEmptyGroup(String quorum) throws Exception {
+        createOffsetsTopic(listenerName(), new Properties());
+
+        // run one consumer in the group
+        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        TestUtils.waitForCondition(
+            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state(), "Stable"),
+            "The group did not initialize as expected."
+        );
+
+        executor.shutdown();
+
+        TestUtils.waitForCondition(
+            () -> Objects.equals(service.collectGroupState(GROUP).state(), 
"Empty"),
+            "The group did not become empty as expected."
+        );
+
+        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+            service.deleteGroups();
+            return null;
+        });
+        assertTrue(output.contains("Deletion of requested consumer groups ('" 
+ GROUP + "') was successful."),
+            "The consumer group could not be deleted as expected");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @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(), false)
+        ));
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--all-groups"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        TestUtils.waitForCondition(() ->
+            Objects.equals(service.listConsumerGroups().toSet(), 
set(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.");
+
+        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+            service.deleteGroups();
+            return null;
+        }).trim();
+        Set<String> expectedGroupsForDeletion = groups.keySet();
+        Set<String> deletedGroupsGrepped = 
Arrays.stream(output.substring(output.indexOf('(') + 1, 
output.indexOf(')')).split(","))
+            .map(str -> str.replaceAll("'", 
"").trim()).collect(Collectors.toSet());
+
+        assertTrue(output.matches("Deletion of requested consumer groups (.*) 
was successful.")
+            && Objects.equals(deletedGroupsGrepped, expectedGroupsForDeletion),
+            "The consumer group(s) could not be deleted as expected");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteEmptyGroup(String quorum) throws Exception {
+        createOffsetsTopic(listenerName(), new Properties());
+
+        // run one consumer in the group
+        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        TestUtils.waitForCondition(
+            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state(), "Stable"),
+            "The group did not initialize as expected.");
+
+        executor.shutdown();
+
+        TestUtils.waitForCondition(
+            () -> Objects.equals(service.collectGroupState(GROUP).state(), 
"Empty"),
+            "The group did not become empty as expected.");
+
+        scala.collection.Map<String, Throwable> result = 
service.deleteGroups();
+        assertTrue(result.size() == 1 && result.contains(GROUP) && 
result.get(GROUP).get() == null,
+            "The consumer group could not be deleted as expected");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteCmdWithMixOfSuccessAndError(String quorum) throws 
Exception {
+        createOffsetsTopic(listenerName(), new Properties());
+        String missingGroup = "missing.group";
+
+        // run one consumer in the group
+        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        TestUtils.waitForCondition(
+            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state(), "Stable"),
+            "The group did not initialize as expected.");
+
+        executor.shutdown();
+
+        TestUtils.waitForCondition(
+            () -> Objects.equals(service.collectGroupState(GROUP).state(), 
"Empty"),
+            "The group did not become empty as expected.");
+
+        String[] cgcArgs2 = new String[cgcArgs.length + 2];
+
+        System.arraycopy(cgcArgs, 0, cgcArgs2, 0, cgcArgs.length);
+        cgcArgs2[cgcArgs2.length - 2] = "--group";
+        cgcArgs2[cgcArgs2.length - 1] = missingGroup;

Review Comment:
   I know we did that in scala like this, but after rewriting in Java it 
becomes more unreadable. Maybe we can just do this:
   ```
   String[] cgcArgs2 = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP, "--group", 
missingGroup};
   ```



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.tools.consumer.group;
+
+import joptsimple.OptionException;
+import kafka.admin.ConsumerGroupCommand;
+import org.apache.kafka.clients.admin.ConsumerGroupListing;
+import org.apache.kafka.common.ConsumerGroupState;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.Optional;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class ListConsumerGroupTest extends ConsumerGroupCommandTest {
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListConsumerGroups(String quorum) throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+        scala.collection.Set<String> expectedGroups = set(Arrays.asList(GROUP, 
simpleGroup));
+        final scala.collection.Set[] foundGroups = new scala.collection.Set[1];
+        TestUtils.waitForCondition(() -> {
+            foundGroups[0] = service.listConsumerGroups().toSet();
+            return Objects.equals(expectedGroups, foundGroups[0]);
+        }, "Expected --list to show groups " + expectedGroups + ", but found " 
+ foundGroups[0] + ".");
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListWithUnrecognizedNewConsumerOption() {
+        String[] cgcArgs = new String[]{"--new-consumer", 
"--bootstrap-server", bootstrapServers(listenerName()), "--list"};
+        assertThrows(OptionException.class, () -> 
getConsumerGroupService(cgcArgs));
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListConsumerGroupsWithStates() throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list", "--state"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        scala.collection.Set<ConsumerGroupListing> expectedListing = 
set(Arrays.asList(
+            new ConsumerGroupListing(simpleGroup, true, 
Optional.of(ConsumerGroupState.EMPTY)),
+            new ConsumerGroupListing(GROUP, false, 
Optional.of(ConsumerGroupState.STABLE))));
+
+        final scala.collection.Set[] foundListing = new 
scala.collection.Set[1];
+        TestUtils.waitForCondition(() -> {
+            foundListing[0] = 
service.listConsumerGroupsWithState(set(Arrays.asList(ConsumerGroupState.values()))).toSet();
+            return Objects.equals(expectedListing, foundListing[0]);
+        }, "Expected to show groups " + expectedListing + ", but found " + 
foundListing[0]);
+
+        scala.collection.Set<ConsumerGroupListing> expectedListingStable = 
set(Collections.singleton(
+            new ConsumerGroupListing(GROUP, false, 
Optional.of(ConsumerGroupState.STABLE))));
+
+        foundListing[0] = null;
+
+        TestUtils.waitForCondition(() -> {
+            foundListing[0] = 
service.listConsumerGroupsWithState(set(Collections.singleton(ConsumerGroupState.STABLE))).toSet();
+            return Objects.equals(expectedListingStable, foundListing[0]);
+        }, "Expected to show groups " + expectedListingStable + ", but found " 
+ foundListing[0]);
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testConsumerGroupStatesFromString(String quorum) {
+        scala.collection.Set<ConsumerGroupState> result = 
ConsumerGroupCommand.consumerGroupStatesFromString("Stable");
+        assertEquals(set(Collections.singleton(ConsumerGroupState.STABLE)), 
result);
+
+        result = ConsumerGroupCommand.consumerGroupStatesFromString("Stable, 
PreparingRebalance");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE, 
ConsumerGroupState.PREPARING_REBALANCE)), result);
+
+        result = 
ConsumerGroupCommand.consumerGroupStatesFromString("Dead,CompletingRebalance,");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.DEAD, 
ConsumerGroupState.COMPLETING_REBALANCE)), result);
+
+        result = ConsumerGroupCommand.consumerGroupStatesFromString("stable");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE)), result);
+
+        result = ConsumerGroupCommand.consumerGroupStatesFromString("stable, 
assigning");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.STABLE, 
ConsumerGroupState.ASSIGNING)), result);
+
+        result = 
ConsumerGroupCommand.consumerGroupStatesFromString("dead,reconciling,");
+        assertEquals(set(Arrays.asList(ConsumerGroupState.DEAD, 
ConsumerGroupState.RECONCILING)), result);
+
+        assertThrows(IllegalArgumentException.class, () -> 
ConsumerGroupCommand.consumerGroupStatesFromString("bad, wrong"));
+
+        assertThrows(IllegalArgumentException.class, () -> 
ConsumerGroupCommand.consumerGroupStatesFromString("  bad, Stable"));
+
+        assertThrows(IllegalArgumentException.class, () -> 
ConsumerGroupCommand.consumerGroupStatesFromString("   ,   ,"));
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListGroupCommand(String quorum) throws Exception {
+        String simpleGroup = "simple-group";
+        addSimpleGroupExecutor(simpleGroup);
+        addConsumerGroupExecutor(1);
+        final String[] out = {""};
+
+        String[] cgcArgs1 = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list"};
+        TestUtils.waitForCondition(() -> {
+            out[0] = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+                ConsumerGroupCommand.main(cgcArgs1);
+                return null;
+            });
+            return !out[0].contains("STATE") && out[0].contains(simpleGroup) 
&& out[0].contains(GROUP);
+        }, "Expected to find " + simpleGroup + ", " + GROUP + " and no header, 
but found " + out[0]);
+
+        String[] cgcArgs2 = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list", "--state"};
+        TestUtils.waitForCondition(() -> {
+            out[0] = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+                ConsumerGroupCommand.main(cgcArgs2);
+                return null;
+            });
+            return out[0].contains("STATE") && out[0].contains(simpleGroup) && 
out[0].contains(GROUP);
+        }, "Expected to find " + simpleGroup + ", " + GROUP + " and the 
header, but found " + out[0]);
+
+        String[] cgcArgs3 = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list", "--state", "Stable"};
+        TestUtils.waitForCondition(() -> {
+            out[0] = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+                ConsumerGroupCommand.main(cgcArgs3);
+                return null;
+            });
+            return out[0].contains("STATE") && out[0].contains(GROUP) && 
out[0].contains("Stable");
+        }, "Expected to find " + GROUP + " in state Stable and the header, but 
found " + out[0]);
+
+        String[] cgcArgs4 = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--list", "--state", "stable"};
+        TestUtils.waitForCondition(() -> {
+            out[0] = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+                ConsumerGroupCommand.main(cgcArgs4);
+                return null;
+            });
+            return out[0].contains("STATE") && out[0].contains(GROUP) && 
out[0].contains("Stable");
+        }, "Expected to find " + GROUP + " in state Stable and the header, but 
found " + out[0]);

Review Comment:
   Duplicate?



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.tools.consumer.group;
+
+import joptsimple.OptionException;
+import kafka.admin.ConsumerGroupCommand;
+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.protocol.Errors;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static 
org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME;
+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(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @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(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteCmdNonExistingGroup(String quorum) {
+        createOffsetsTopic(listenerName(), new Properties());
+        String missingGroup = "missing.group";
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+            service.deleteGroups();
+            return null;
+        });
+        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");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteNonExistingGroup(String quorum) {
+        createOffsetsTopic(listenerName(), new Properties());
+        String missingGroup = "missing.group";
+
+        // note the group to be deleted is a different (non-existing) group
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", missingGroup};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        scala.collection.Map<String, Throwable> result = 
service.deleteGroups();
+        assertTrue(result.size() == 1 && result.contains(missingGroup) && 
result.get(missingGroup).get().getCause() instanceof GroupIdNotFoundException,
+            "The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not 
detected while deleting consumer group");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteCmdNonEmptyGroup(String quorum) throws Exception {
+        createOffsetsTopic(listenerName(), new Properties());
+
+        // run one consumer in the group
+        addConsumerGroupExecutor(1);
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        TestUtils.waitForCondition(
+            () -> service.collectGroupMembers(GROUP, false)._2.get().size() == 
1,
+            "The group did not initialize as expected."
+        );
+
+        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+            service.deleteGroups();
+            return null;
+        });
+        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 + ")");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteNonEmptyGroup(String quorum) throws Exception {
+        createOffsetsTopic(listenerName(), new Properties());
+
+        // run one consumer in the group
+        addConsumerGroupExecutor(1);
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        TestUtils.waitForCondition(
+            () -> service.collectGroupMembers(GROUP, false)._2.get().size() == 
1,
+            "The group did not initialize as expected."
+        );
+
+        scala.collection.Map<String, Throwable> result = 
service.deleteGroups();
+        assertNotNull(result.get(GROUP).get(),
+            "Group was deleted successfully, but it shouldn't have been. 
Result was:(" + result + ")");
+        assertTrue(result.size() == 1 && result.contains(GROUP) && 
result.get(GROUP).get().getCause() instanceof GroupNotEmptyException,
+            "The expected error (" + Errors.NON_EMPTY_GROUP + ") was not 
detected while deleting consumer group. Result was:(" + result + ")");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteCmdEmptyGroup(String quorum) throws Exception {
+        createOffsetsTopic(listenerName(), new Properties());
+
+        // run one consumer in the group
+        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        TestUtils.waitForCondition(
+            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state(), "Stable"),
+            "The group did not initialize as expected."
+        );
+
+        executor.shutdown();
+
+        TestUtils.waitForCondition(
+            () -> Objects.equals(service.collectGroupState(GROUP).state(), 
"Empty"),
+            "The group did not become empty as expected."
+        );
+
+        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+            service.deleteGroups();
+            return null;
+        });
+        assertTrue(output.contains("Deletion of requested consumer groups ('" 
+ GROUP + "') was successful."),
+            "The consumer group could not be deleted as expected");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @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(), false)
+        ));
+
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--all-groups"};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        TestUtils.waitForCondition(() ->
+            Objects.equals(service.listConsumerGroups().toSet(), 
set(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.");
+
+        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+            service.deleteGroups();
+            return null;
+        }).trim();
+        Set<String> expectedGroupsForDeletion = groups.keySet();
+        Set<String> deletedGroupsGrepped = 
Arrays.stream(output.substring(output.indexOf('(') + 1, 
output.indexOf(')')).split(","))
+            .map(str -> str.replaceAll("'", 
"").trim()).collect(Collectors.toSet());
+
+        assertTrue(output.matches("Deletion of requested consumer groups (.*) 
was successful.")
+            && Objects.equals(deletedGroupsGrepped, expectedGroupsForDeletion),
+            "The consumer group(s) could not be deleted as expected");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteEmptyGroup(String quorum) throws Exception {
+        createOffsetsTopic(listenerName(), new Properties());
+
+        // run one consumer in the group
+        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        TestUtils.waitForCondition(
+            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state(), "Stable"),
+            "The group did not initialize as expected.");
+
+        executor.shutdown();
+
+        TestUtils.waitForCondition(
+            () -> Objects.equals(service.collectGroupState(GROUP).state(), 
"Empty"),
+            "The group did not become empty as expected.");
+
+        scala.collection.Map<String, Throwable> result = 
service.deleteGroups();
+        assertTrue(result.size() == 1 && result.contains(GROUP) && 
result.get(GROUP).get() == null,
+            "The consumer group could not be deleted as expected");
+    }
+
+    @ParameterizedTest(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteCmdWithMixOfSuccessAndError(String quorum) throws 
Exception {
+        createOffsetsTopic(listenerName(), new Properties());
+        String missingGroup = "missing.group";
+
+        // run one consumer in the group
+        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        TestUtils.waitForCondition(
+            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state(), "Stable"),
+            "The group did not initialize as expected.");
+
+        executor.shutdown();
+
+        TestUtils.waitForCondition(
+            () -> Objects.equals(service.collectGroupState(GROUP).state(), 
"Empty"),
+            "The group did not become empty as expected.");
+
+        String[] cgcArgs2 = new String[cgcArgs.length + 2];
+
+        System.arraycopy(cgcArgs, 0, cgcArgs2, 0, cgcArgs.length);
+        cgcArgs2[cgcArgs2.length - 2] = "--group";
+        cgcArgs2[cgcArgs2.length - 1] = missingGroup;
+
+        ConsumerGroupCommand.ConsumerGroupService service2 = 
getConsumerGroupService(cgcArgs2);
+
+        String output = kafka.utils.TestUtils.grabConsoleOutput(() -> {
+            service2.deleteGroups();
+            return null;
+        });
+        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(name = TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteWithMixOfSuccessAndError(String quorum) throws 
Exception {
+        createOffsetsTopic(listenerName(), new Properties());
+        String missingGroup = "missing.group";
+
+        // run one consumer in the group
+        ConsumerGroupExecutor executor = addConsumerGroupExecutor(1);
+        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--delete", "--group", GROUP};
+        ConsumerGroupCommand.ConsumerGroupService service = 
getConsumerGroupService(cgcArgs);
+
+        TestUtils.waitForCondition(
+            () -> service.listConsumerGroups().contains(GROUP) && 
Objects.equals(service.collectGroupState(GROUP).state(), "Stable"),
+            "The group did not initialize as expected.");
+
+        executor.shutdown();
+
+        TestUtils.waitForCondition(
+            () -> Objects.equals(service.collectGroupState(GROUP).state(), 
"Empty"),
+            "The group did not become empty as expected.");
+
+        String[] cgcArgs2 = new String[cgcArgs.length + 2];
+
+        System.arraycopy(cgcArgs, 0, cgcArgs2, 0, cgcArgs.length);
+        cgcArgs2[cgcArgs2.length - 2] = "--group";
+        cgcArgs2[cgcArgs2.length - 1] = missingGroup;

Review Comment:
   ditto



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

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

Reply via email to