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


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

Review Comment:
   `setName` is a bad naming I feel, since it just add a "tag" to be a part of 
"display name". For example, setting `newGroupCoordinator` will generate a 
display name having following content: "name=newGroupCoordinator".
   
   ```
   Type=Raft-Isolated, Name=newGroupCoordinator, MetadataVersion=3.8-IV0, 
Security=PLAINTEXT PASSED
   
   Type=Raft-Combined, Name=newGroupCoordinator, MetadataVersion=3.8-IV0, 
Security=PLAINTEXT PASSED
   
   ```
   
   We should replace it by "setTags". By that change, it can get more readable 
and enable us to customize more "key" and "value". WDYT



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