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


##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupExecutor.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 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.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.utils.Utils;
+
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static java.util.Collections.singleton;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG;
+import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
+import static org.apache.kafka.common.GroupType.CONSUMER;
+
+class ConsumerGroupExecutor {
+
+    static AutoCloseable buildConsumerGroup(String brokerAddress,
+                                            int numberOfConsumers,
+                                            String groupId,
+                                            String topic,
+                                            String groupProtocol,
+                                            Optional<String> remoteAssignor,
+                                            Map<String, Object> customConfigs,
+                                            boolean syncCommit) {
+        return buildConsumers(
+                brokerAddress,
+                numberOfConsumers,
+                groupId,
+                groupProtocol,
+                topic,
+                RangeAssignor.class.getName(),
+                remoteAssignor,
+                customConfigs,
+                syncCommit
+        );
+    }
+
+    static AutoCloseable buildClassicGroup(String brokerAddress,
+                                           int numberOfConsumers,
+                                           String groupId,
+                                           String topic,
+                                           String assignmentStrategy,
+                                           Map<String, Object> customConfigs,
+                                           boolean syncCommit) {
+        return buildConsumers(
+                brokerAddress,
+                numberOfConsumers,
+                groupId,
+                GroupProtocol.CLASSIC.name,
+                topic,
+                assignmentStrategy,
+                Optional.empty(),
+                customConfigs,
+                syncCommit
+        );
+    }
+
+    private ConsumerGroupExecutor() {
+    }
+
+    private static AutoCloseable buildConsumers(
+            String brokerAddress,
+            int numberOfConsumers,
+            String groupId,
+            String groupProtocol,
+            String topic,
+            String assignmentStrategy,
+            Optional<String> remoteAssignor,
+            Map<String, Object> customConfigs,
+            boolean syncCommit
+    ) {
+        List<Map<String, Object>> allConfigs = IntStream.range(0, 
numberOfConsumers)
+                .mapToObj(ignored ->
+                        composeConfigs(
+                                brokerAddress,
+                                groupId,
+                                groupProtocol,
+                                assignmentStrategy,
+                                remoteAssignor,
+                                customConfigs
+                        )
+                )
+                .collect(Collectors.toList());
+        Queue<KafkaConsumer<String, String>> kafkaConsumers = new 
LinkedList<>();
+        buildConsumersSafely(allConfigs, kafkaConsumers);
+
+        ExecutorService executor = 
Executors.newFixedThreadPool(kafkaConsumers.size());
+
+        AtomicBoolean closed = new AtomicBoolean(false);
+        final AutoCloseable closeable = () -> {
+            closed.set(true);
+            kafkaConsumers.forEach(consumer -> Utils.closeQuietly(consumer, 
"Release Consumer"));
+            executor.shutdownNow();
+            executor.awaitTermination(1, TimeUnit.MINUTES);
+        };
+
+        try {
+            while (!kafkaConsumers.isEmpty()) {
+                KafkaConsumer<String, String> consumer = kafkaConsumers.poll();
+                executor.execute(() -> {
+                    try {
+                        consumer.subscribe(singleton(topic));
+                        while (closed.get()) {

Review Comment:
   `while (!closed.get())`



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java:
##########
@@ -17,279 +17,297 @@
 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.AdminClientConfig;
 import org.apache.kafka.clients.consumer.GroupProtocol;
 import org.apache.kafka.clients.consumer.RangeAssignor;
+import org.apache.kafka.common.ConsumerGroupState;
 import org.apache.kafka.common.errors.GroupIdNotFoundException;
 import org.apache.kafka.common.errors.GroupNotEmptyException;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.test.TestUtils;
 import org.apache.kafka.tools.ToolsTestUtils;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.ValueSource;
+import org.junit.jupiter.api.extension.ExtendWith;
 
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
-import java.util.Properties;
 import java.util.Set;
 import java.util.function.Function;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.singleton;
+import static java.util.Collections.singletonMap;
+import static org.apache.kafka.clients.consumer.GroupProtocol.CLASSIC;
+import static org.apache.kafka.clients.consumer.GroupProtocol.CONSUMER;
+import static org.apache.kafka.common.ConsumerGroupState.EMPTY;
+import static org.apache.kafka.common.ConsumerGroupState.STABLE;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG;
+import static 
org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG;
 import static org.junit.jupiter.api.Assertions.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";
-
-        String[] cgcArgs = new String[]{"--bootstrap-server", 
bootstrapServers(listenerName()), "--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");
+@ExtendWith(value = ClusterTestExtensions.class)
+@ClusterTestDefaults(clusterType = Type.ALL, serverProperties = {
+        @ClusterConfigProperty(key = OFFSETS_TOPIC_PARTITIONS_CONFIG, value = 
"1"),
+        @ClusterConfigProperty(key = OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, 
value = "1"),
+        @ClusterConfigProperty(key = NEW_GROUP_COORDINATOR_ENABLE_CONFIG, 
value = "true")
+})
+public class DeleteConsumerGroupsTest {
+    private static final String TOPIC = "foo";

Review Comment:
   As we run a loop in each test case, it would be nice to create different 
topic/group for them. please take a look at #15802



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