lucasbru commented on code in PR #19893:
URL: https://github.com/apache/kafka/pull/19893#discussion_r2128204458


##########
tools/src/test/java/org/apache/kafka/tools/streams/DeleteStreamsInternalTopicsTest.java:
##########
@@ -0,0 +1,392 @@
+/*
+ * 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.streams;
+
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.GroupState;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
+import org.apache.kafka.streams.GroupProtocol;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValueTimestamp;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.test.TestUtils;
+import org.apache.kafka.tools.ToolsTestUtils;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import joptsimple.OptionException;
+
+import static org.apache.kafka.common.GroupState.EMPTY;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Timeout(600)
+@Tag("integration")
+public class DeleteStreamsInternalTopicsTest {
+    private static final String APP_ID_PREFIX = "delete-internal-topics-test-";
+    private static final String INPUT_TOPIC_PREFIX = "input-topic-";
+    private static final int RECORD_TOTAL = 10;
+    public static EmbeddedKafkaCluster cluster;
+    private static String bootstrapServers;
+
+    @BeforeAll
+    public static void startCluster() {
+        final Properties props = new Properties();
+        
props.setProperty(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG,
 "classic,consumer,streams");

Review Comment:
   THis shouldn't be required anymore, as streams is now in the default.



##########
tools/src/main/java/org/apache/kafka/tools/streams/StreamsGroupCommand.java:
##########
@@ -396,6 +400,171 @@ protected Admin createAdminClient(Map<String, String> 
configOverrides) throws IO
             props.putAll(configOverrides);
             return Admin.create(props);
         }
+
+        public void deleteInternalTopics() {
+            List<String> groupIds = new 
ArrayList<>(opts.options.valuesOf(opts.groupOpt));
+
+            List<GroupListing> streamsGroupIds = listDetailedStreamsGroups();
+            groupIds.removeIf(groupId -> {
+                boolean notFound = streamsGroupIds.stream().noneMatch(item -> 
item.groupId().equals(groupId));
+                if (notFound) {
+                    printError("Group '" + groupId + "' does not exist or is 
not a streams group.", Optional.empty());
+                }
+                return notFound;
+            });
+
+            groupIds.removeIf(groupId -> {
+                try {
+                    GroupState groupState = collectGroupState(groupId);
+                    if (groupState != GroupState.DEAD && groupState != 
GroupState.EMPTY) {
+                        printError("The specified group '" + groupId + "' is 
not EMPTY or DEAD.", Optional.empty());
+                        return true;
+                    }
+                    return false;
+                } catch (Exception e) {
+                    throw new RuntimeException(e);
+                }
+            });
+
+            if (groupIds.isEmpty()) {
+                printError("No valid streams groups specified for deletion.", 
Optional.empty());
+            } else {
+                Set<String> topicsToDelete = new HashSet<>();
+                List<String> allInternalTopics = new ArrayList<>();
+                
retrieveInternalTopics(groupIds).values().forEach(allInternalTopics::addAll);
+
+                if (opts.options.has(opts.internalTopicOpt)) {
+                    List<String> internalTopics = new 
ArrayList<>(opts.options.valuesOf(opts.internalTopicOpt));
+                    if (internalTopics.isEmpty()) {
+                        printError("No internal topics specified for 
deletion.", Optional.empty());
+                        return;
+                    }
+                    topicsToDelete = new HashSet<>(internalTopics);
+                    topicsToDelete.removeIf(topic -> {
+                        if (!allInternalTopics.contains(topic)) {
+                            printError("The specified internal topic '" + 
topic + "' is not associated to the any of the groups ('" +
+                                String.join("', '", groupIds) + "') as an 
internal topic and thus will not be deleted.", Optional.empty());
+                            return true;
+                        }
+                        return false;
+                    });
+                } else if (opts.options.has(opts.allInternalTopicsOpt)) {
+                    topicsToDelete = new HashSet<>(allInternalTopics);
+                }
+                if (topicsToDelete.isEmpty()) {
+                    printError("No internal topics specified for deletion.", 
Optional.empty());
+                } else {
+                    deleteTopics(topicsToDelete);
+                }
+            }
+        }
+
+        private void deleteTopics(Set<String> topicsToDelete) {
+            DeleteTopicsResult deleteTopicsResult = null;
+            try {
+                deleteTopicsResult = adminClient.deleteTopics(topicsToDelete);
+                deleteTopicsResult.all().get();
+                System.out.println("Deletion of requested internal topics ('" 
+ String.join("', '", topicsToDelete) + "') was successful.");
+            } catch (ExecutionException | InterruptedException e) {
+                if (deleteTopicsResult != null) {
+                    deleteTopicsResult.topicNameValues().forEach((topic, 
future) -> {
+                        try {
+                            future.get();
+                        } catch (Exception topicException) {
+                            System.out.println("Failed to delete internal 
topic: " + topic);

Review Comment:
   Shouldn't you use `printError` here, and possiblyh pass the topicException, 
as in all the other error messages?



##########
tools/src/main/java/org/apache/kafka/tools/streams/StreamsGroupCommandOptions.java:
##########
@@ -78,6 +94,13 @@ public StreamsGroupCommandOptions(String[] args) {
             .ofType(String.class);
         listOpt = parser.accepts("list", LIST_DOC);
         describeOpt = parser.accepts("describe", DESCRIBE_DOC);
+        deleteOpt = parser.accepts("delete", DELETE_DOC);
+        internalTopicOpt = parser.accepts("internal-topic", INTERNAL_TOPIC_DOC)
+            .availableIf(deleteOpt)
+            .withRequiredArg()
+            .describedAs("name of internal topic to delete")
+            .ofType(String.class);
+        allInternalTopicsOpt = parser.accepts("all-internal-topics", 
ALL_INTERNAL_TOPICS_DOC);

Review Comment:
   Should we add a:
   ```
               .availableIf(deleteOpt)
   ```
   



##########
tools/src/test/java/org/apache/kafka/tools/streams/DeleteStreamsInternalTopicsTest.java:
##########
@@ -0,0 +1,392 @@
+/*
+ * 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.streams;
+
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.GroupState;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
+import org.apache.kafka.streams.GroupProtocol;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValueTimestamp;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.test.TestUtils;
+import org.apache.kafka.tools.ToolsTestUtils;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import joptsimple.OptionException;
+
+import static org.apache.kafka.common.GroupState.EMPTY;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Timeout(600)
+@Tag("integration")
+public class DeleteStreamsInternalTopicsTest {
+    private static final String APP_ID_PREFIX = "delete-internal-topics-test-";
+    private static final String INPUT_TOPIC_PREFIX = "input-topic-";
+    private static final int RECORD_TOTAL = 10;
+    public static EmbeddedKafkaCluster cluster;
+    private static String bootstrapServers;
+
+    @BeforeAll
+    public static void startCluster() {
+        final Properties props = new Properties();
+        
props.setProperty(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG,
 "classic,consumer,streams");
+        cluster = new EmbeddedKafkaCluster(2, props);
+        cluster.start();
+
+        bootstrapServers = cluster.bootstrapServers();
+    }
+
+    @AfterAll
+    public static void closeCluster() {
+        cluster.stop();
+    }
+
+    @Test
+    public void testDeleteWithUnrecognizedOption() {
+        final String[] args = new String[]{"--unrecognized-option", 
"--bootstrap-server", bootstrapServers, "--delete",
+            "--internal-topic", "foo", "--group", "bar"};
+        assertThrows(OptionException.class, () -> 
getStreamsGroupService(args));
+    }
+
+    @Test
+    public void testDeleteWithoutGroupOption() {
+        final String[] args = new String[]{"--bootstrap-server", 
bootstrapServers, "--delete", "--internal-topic", "foo"};
+        AtomicBoolean exited = new AtomicBoolean(false);
+        Exit.setExitProcedure(((statusCode, message) -> {
+            assertNotEquals(0, statusCode);
+            assertTrue(message.contains("Option [delete] takes the option 
[group]"));
+            exited.set(true);
+        }));
+        try {
+            getStreamsGroupService(args);
+        } finally {
+            assertTrue(exited.get());
+        }
+    }
+
+    @Test
+    public void testDeleteWithoutInternalTopicOption() {
+        final String[] args = new String[]{"--bootstrap-server", 
bootstrapServers, "--delete", "--group", "foo"};
+        AtomicBoolean exited = new AtomicBoolean(false);
+        Exit.setExitProcedure(((statusCode, message) -> {
+            assertNotEquals(0, statusCode);
+            assertTrue(message.contains("Option [delete] takes one of these 
options: [internal-topic], [all-internal-topics]"));
+            exited.set(true);
+        }));
+        try {
+            getStreamsGroupService(args);
+        } finally {
+            assertTrue(exited.get());
+        }
+    }
+
+    @Test
+    public void testDeleteInternalTopicNotExistingGroup() {
+        String[] args = new String[]{"--bootstrap-server", bootstrapServers, 
"--delete", "--internal-topic", "foo", "--group", "bar"};
+        StreamsGroupCommand.StreamsGroupService service = 
getStreamsGroupService(args);
+        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteInternalTopics);
+        assertTrue(output.contains("Group 'bar' does not exist or is not a 
streams group."));
+    }
+
+    @Test
+    public void testDeleteInternalTopicsMixedAssociatedWithTheGroup() throws 
Exception {
+        final String appId = generateGroupAppId();
+        final List<String> internalTopics = Arrays.asList(
+            appId + "-aggregated_value-changelog",
+            appId + "-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition",
+            appId + "-KSTREAM-AGGREGATE-STATE-STORE-0000000003-changelog"
+        );
+        final String wrongInternalTopic = "foo";
+
+        String[] args = new String[]{"--bootstrap-server", bootstrapServers, 
"--delete",
+            "--internal-topic", wrongInternalTopic, "--internal-topic", 
internalTopics.get(0), "--internal-topic", internalTopics.get(1), 
"--internal-topic", internalTopics.get(2),
+            "--group", appId};
+        StreamsGroupCommand.StreamsGroupService service = 
getStreamsGroupService(args);
+        KafkaStreams streams = startKSApp(appId, service);
+        stopKSApp(appId, streams, service);
+        List<String> allTopics = service.collectAllTopics(appId);
+
+        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteInternalTopics);
+
+        assertTrue(output.contains("The specified internal topic 'foo' is not 
associated to the any of the groups ('" + appId + "') " +
+            "as an internal topic and thus will not be deleted."));
+        assertTrue(
+            output.contains("Deletion of requested internal topics (") &&
+                output.contains("was successful.") &&
+                internalTopics.stream().allMatch(output::contains));
+        // Verify that the internal topics are deleted
+        allTopics.addAll(List.of("__consumer_offsets", "__transaction_state"));
+        allTopics.removeAll(internalTopics);
+        cluster.waitForRemainingTopics(30000, allTopics.toArray(new 
String[0]));
+
+        allTopics.removeAll(List.of("__consumer_offsets", 
"__transaction_state"));
+        cluster.deleteTopics(allTopics.toArray(new String[0]));
+    }
+
+    @Test
+    public void testDeleteAllInternalTopicsFromNonEmptyGroup() throws 
Exception {
+        final String appId = generateGroupAppId();
+
+        String[] args = new String[]{"--bootstrap-server", bootstrapServers, 
"--delete",
+            "--all-internal-topics", "--group", appId};
+        StreamsGroupCommand.StreamsGroupService service = 
getStreamsGroupService(args);
+        KafkaStreams streams = startKSApp(appId, service);
+        List<String> allTopics = service.collectAllTopics(appId);
+
+        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteInternalTopics);
+
+        assertTrue(output.contains("The specified group '" + appId + "' is not 
EMPTY or DEAD"));

Review Comment:
   Can we have a more descriptive error message for the user? Since EMPTY and 
DEAD are somewhat internal.
   
   Maybe "The specified group '" + appId + "' still has active members. Please 
terminate the members before retrying the operation" or something.



##########
tools/src/test/java/org/apache/kafka/tools/streams/DeleteStreamsInternalTopicsTest.java:
##########
@@ -0,0 +1,392 @@
+/*
+ * 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.streams;
+
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.GroupState;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
+import org.apache.kafka.streams.GroupProtocol;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValueTimestamp;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.test.TestUtils;
+import org.apache.kafka.tools.ToolsTestUtils;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import joptsimple.OptionException;
+
+import static org.apache.kafka.common.GroupState.EMPTY;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Timeout(600)
+@Tag("integration")
+public class DeleteStreamsInternalTopicsTest {
+    private static final String APP_ID_PREFIX = "delete-internal-topics-test-";
+    private static final String INPUT_TOPIC_PREFIX = "input-topic-";
+    private static final int RECORD_TOTAL = 10;
+    public static EmbeddedKafkaCluster cluster;
+    private static String bootstrapServers;
+
+    @BeforeAll
+    public static void startCluster() {
+        final Properties props = new Properties();
+        
props.setProperty(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG,
 "classic,consumer,streams");
+        cluster = new EmbeddedKafkaCluster(2, props);
+        cluster.start();
+
+        bootstrapServers = cluster.bootstrapServers();
+    }
+
+    @AfterAll
+    public static void closeCluster() {
+        cluster.stop();
+    }
+
+    @Test
+    public void testDeleteWithUnrecognizedOption() {
+        final String[] args = new String[]{"--unrecognized-option", 
"--bootstrap-server", bootstrapServers, "--delete",
+            "--internal-topic", "foo", "--group", "bar"};
+        assertThrows(OptionException.class, () -> 
getStreamsGroupService(args));
+    }
+
+    @Test
+    public void testDeleteWithoutGroupOption() {
+        final String[] args = new String[]{"--bootstrap-server", 
bootstrapServers, "--delete", "--internal-topic", "foo"};
+        AtomicBoolean exited = new AtomicBoolean(false);
+        Exit.setExitProcedure(((statusCode, message) -> {
+            assertNotEquals(0, statusCode);
+            assertTrue(message.contains("Option [delete] takes the option 
[group]"));
+            exited.set(true);
+        }));
+        try {
+            getStreamsGroupService(args);
+        } finally {
+            assertTrue(exited.get());
+        }
+    }
+
+    @Test
+    public void testDeleteWithoutInternalTopicOption() {
+        final String[] args = new String[]{"--bootstrap-server", 
bootstrapServers, "--delete", "--group", "foo"};
+        AtomicBoolean exited = new AtomicBoolean(false);
+        Exit.setExitProcedure(((statusCode, message) -> {
+            assertNotEquals(0, statusCode);
+            assertTrue(message.contains("Option [delete] takes one of these 
options: [internal-topic], [all-internal-topics]"));
+            exited.set(true);
+        }));
+        try {
+            getStreamsGroupService(args);
+        } finally {
+            assertTrue(exited.get());
+        }
+    }
+
+    @Test
+    public void testDeleteInternalTopicNotExistingGroup() {
+        String[] args = new String[]{"--bootstrap-server", bootstrapServers, 
"--delete", "--internal-topic", "foo", "--group", "bar"};
+        StreamsGroupCommand.StreamsGroupService service = 
getStreamsGroupService(args);
+        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteInternalTopics);
+        assertTrue(output.contains("Group 'bar' does not exist or is not a 
streams group."));

Review Comment:
   I understand this PR doesn't delete the group (at least before the rebase). 
After rebase, can we make sure that if something is wrong with the arguments 
(e.g. an internal topic is missing) _nothing_ is done? So, we shouldn't delete 
the group if the internal topic does not exist.



##########
tools/src/test/java/org/apache/kafka/tools/streams/DeleteStreamsInternalTopicsTest.java:
##########
@@ -0,0 +1,392 @@
+/*
+ * 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.streams;
+
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.common.GroupState;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
+import org.apache.kafka.streams.GroupProtocol;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValueTimestamp;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.Consumed;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.Materialized;
+import org.apache.kafka.test.TestUtils;
+import org.apache.kafka.tools.ToolsTestUtils;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import joptsimple.OptionException;
+
+import static org.apache.kafka.common.GroupState.EMPTY;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Timeout(600)
+@Tag("integration")
+public class DeleteStreamsInternalTopicsTest {
+    private static final String APP_ID_PREFIX = "delete-internal-topics-test-";
+    private static final String INPUT_TOPIC_PREFIX = "input-topic-";
+    private static final int RECORD_TOTAL = 10;
+    public static EmbeddedKafkaCluster cluster;
+    private static String bootstrapServers;
+
+    @BeforeAll
+    public static void startCluster() {
+        final Properties props = new Properties();
+        
props.setProperty(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG,
 "classic,consumer,streams");
+        cluster = new EmbeddedKafkaCluster(2, props);
+        cluster.start();
+
+        bootstrapServers = cluster.bootstrapServers();
+    }
+
+    @AfterAll
+    public static void closeCluster() {
+        cluster.stop();
+    }
+
+    @Test
+    public void testDeleteWithUnrecognizedOption() {
+        final String[] args = new String[]{"--unrecognized-option", 
"--bootstrap-server", bootstrapServers, "--delete",
+            "--internal-topic", "foo", "--group", "bar"};
+        assertThrows(OptionException.class, () -> 
getStreamsGroupService(args));
+    }
+
+    @Test
+    public void testDeleteWithoutGroupOption() {
+        final String[] args = new String[]{"--bootstrap-server", 
bootstrapServers, "--delete", "--internal-topic", "foo"};
+        AtomicBoolean exited = new AtomicBoolean(false);
+        Exit.setExitProcedure(((statusCode, message) -> {
+            assertNotEquals(0, statusCode);
+            assertTrue(message.contains("Option [delete] takes the option 
[group]"));
+            exited.set(true);
+        }));
+        try {
+            getStreamsGroupService(args);
+        } finally {
+            assertTrue(exited.get());
+        }
+    }
+
+    @Test
+    public void testDeleteWithoutInternalTopicOption() {
+        final String[] args = new String[]{"--bootstrap-server", 
bootstrapServers, "--delete", "--group", "foo"};
+        AtomicBoolean exited = new AtomicBoolean(false);
+        Exit.setExitProcedure(((statusCode, message) -> {
+            assertNotEquals(0, statusCode);
+            assertTrue(message.contains("Option [delete] takes one of these 
options: [internal-topic], [all-internal-topics]"));
+            exited.set(true);
+        }));
+        try {
+            getStreamsGroupService(args);
+        } finally {
+            assertTrue(exited.get());
+        }
+    }
+
+    @Test
+    public void testDeleteInternalTopicNotExistingGroup() {
+        String[] args = new String[]{"--bootstrap-server", bootstrapServers, 
"--delete", "--internal-topic", "foo", "--group", "bar"};
+        StreamsGroupCommand.StreamsGroupService service = 
getStreamsGroupService(args);
+        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteInternalTopics);
+        assertTrue(output.contains("Group 'bar' does not exist or is not a 
streams group."));
+    }
+
+    @Test
+    public void testDeleteInternalTopicsMixedAssociatedWithTheGroup() throws 
Exception {
+        final String appId = generateGroupAppId();
+        final List<String> internalTopics = Arrays.asList(
+            appId + "-aggregated_value-changelog",
+            appId + "-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition",
+            appId + "-KSTREAM-AGGREGATE-STATE-STORE-0000000003-changelog"
+        );
+        final String wrongInternalTopic = "foo";
+
+        String[] args = new String[]{"--bootstrap-server", bootstrapServers, 
"--delete",
+            "--internal-topic", wrongInternalTopic, "--internal-topic", 
internalTopics.get(0), "--internal-topic", internalTopics.get(1), 
"--internal-topic", internalTopics.get(2),
+            "--group", appId};
+        StreamsGroupCommand.StreamsGroupService service = 
getStreamsGroupService(args);
+        KafkaStreams streams = startKSApp(appId, service);
+        stopKSApp(appId, streams, service);
+        List<String> allTopics = service.collectAllTopics(appId);
+
+        String output = 
ToolsTestUtils.grabConsoleOutput(service::deleteInternalTopics);
+
+        assertTrue(output.contains("The specified internal topic 'foo' is not 
associated to the any of the groups ('" + appId + "') " +
+            "as an internal topic and thus will not be deleted."));
+        assertTrue(
+            output.contains("Deletion of requested internal topics (") &&
+                output.contains("was successful.") &&
+                internalTopics.stream().allMatch(output::contains));
+        // Verify that the internal topics are deleted
+        allTopics.addAll(List.of("__consumer_offsets", "__transaction_state"));
+        allTopics.removeAll(internalTopics);
+        cluster.waitForRemainingTopics(30000, allTopics.toArray(new 
String[0]));
+
+        allTopics.removeAll(List.of("__consumer_offsets", 
"__transaction_state"));
+        cluster.deleteTopics(allTopics.toArray(new String[0]));

Review Comment:
   It's a bit weird that you are using the service itself for cleanup and add 
it to every test.
   
   I would propose you add a @AfterEach method that just deletes all topics:
   ```
           try (final Admin adminClient = createAdminClient()) {
               final Set<String> topics = 
adminClient.listTopics().names().get();
               adminClient.deleteTopics(topics).all().get();
           } catch (final UnknownTopicOrPartitionException ignored) {
           } catch (final ExecutionException | InterruptedException e) {
               if (!(e.getCause() instanceof UnknownTopicOrPartitionException)) 
{
                   throw new RuntimeException(e);
               }
           }
   ```



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