You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "OmniaGM (via GitHub)" <gi...@apache.org> on 2023/02/06 12:35:22 UTC

[GitHub] [kafka] OmniaGM opened a new pull request, #13201: KAFKA-14596: Move TopicCommand to tools

OmniaGM opened a new pull request, #13201:
URL: https://github.com/apache/kafka/pull/13201

   This pr include 
   - The changes include switching Scala code to java
   - Move TopicCommand and all test cases to tools
   - The PR depends on #13171 to replace the usage of `CoreUtils.duplicate` by `ToolsUtils.findDuplicates` 
   
   Some implementation notes: <Notes to share with upstream>
   - I added `ToolsTestUtils.createBrokerProperties` as a wrapper for `TestUtils.createBrokerConfig` to hide the conversion between Scala and Java types. 
   - Replicated `TestUtils.setReplicationThrottleForPartitions` and `TestUtils.removeReplicationThrottleForPartitions` to `ToolsTestUtils` as the methods are used only `TopicCommandIntegrationTest` and `ReassignPartitionsIntegrationTest`. We need to remove it from `TestUtils` once we migrate `ReassignPartitions`
   - Replicated `TestInfoUtils.TestWithParameterizedQuorumName` to `ToolsTestUtils` as java convert this into a getter function which cannot be used with `ParameterizedTest`
   
   *More detailed description of your change,
   if necessary. The PR title and PR message become
   the squashed commit message, so use a separate
   comment to ping reviewers.*
   
   *Summary of testing strategy (including rationale)
   for the feature or bug fix. Unit and/or integration
   tests are expected for any behaviour change and
   system tests should be considered for larger changes.*
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1719725886

   > > Hi @OmniaGM, thanks. I left some comments.
   > > Additionally, this error message is different from previous implementation:
   > > ```shell
   > > ### OLD
   > > $ bin/kafka-topics.sh --bootstrap-server :9092 --describe --topic my-topic-foo --delete-config sdsad
   > > Option "[delete-config]" can't be used with option "[describe]"
   > > 
   > > ### NEW
   > > $ bin/kafka-topics.sh --bootstrap-server :9092 --describe --topic my-topic-foo --delete-config sdsad
   > > Option "[delete-config]" can't be used with option "[bootstrap-server]"
   > > ```
   > 
   > Good catch @fvaleri , I missed that `immutable.Set` which is used to define invalid options in old scala code does preserve the order of insertion so when we check `bootstrap` option wasn't the first one the list in scala to compare but instead it's `describe`. I updated the code to use `LinkedHashSet` instead of `Set` which should solve the problem now.
   > 
   > I also think the original [scala code](https://github.com/apache/kafka/blob/ac39342d47062c1a9aec9b744b40ad7fd2de27b2/core/src/main/scala/kafka/admin/TopicCommand.scala#L627C1-L627C146) was wrong and shouldn't have bootsrap option to the args to check as this means `deleteConfigOpt` not allowed with bootstrap options which isn't accurate. Not sure if we should remove it or not WDYT?
   
   This is correct, although the error message could be improved (see `alter` option description). The `delete-config` option was used along with the deprecated `zookeeper` option, which was removed in KAFKA-12596. I think that PR missed to remove `delete-config`. Feel free to raise a separate issue for that.
    
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "mimaison (via GitHub)" <gi...@apache.org>.
mimaison commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1744639271

   @OmniaGM Thanks for the PR. Can you rebase on trunk to resolve the conflicts?


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1326292439


##########
build.gradle:
##########
@@ -1880,9 +1880,10 @@ project(':tools:tools-api') {
 
 project(':tools') {
   archivesBaseName = "kafka-tools"
-
   dependencies {
+    api libs.scalaLibrary

Review Comment:
   Do you get some error removing it? For me it works just fine.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1358472785


##########
checkstyle/import-control.xml:
##########
@@ -286,6 +289,8 @@
     <allow pkg="net.sourceforge.argparse4j" />
     <allow pkg="org.apache.log4j" />
     <allow pkg="kafka.test" />
+    <allow pkg="scala.collection" />
+    <allow pkg="scala.compat" />

Review Comment:
   not any more I needed them at some point while converting from scala to java but didn't review the allowed list



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "mimaison (via GitHub)" <gi...@apache.org>.
mimaison commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1360412557


##########
tools/src/main/java/org/apache/kafka/tools/ToolsUtils.java:
##########
@@ -123,4 +125,21 @@ public static void validateBootstrapServer(String hostPort) throws IllegalArgume
             throw new IllegalArgumentException("Please provide valid host:port like host1:9091,host2:9092\n");
         }
     }
+
+    public static <T> List<T> duplicates(List<T> s) {

Review Comment:
   Should the return type be `Set` instead of `List` as each entry in the returned value should only appear once?
   If so this could be simplified into something like, so we only iterate the input list once:
   ```
   Set<T> set = new HashSet<>();
   Set<T> duplicates = new HashSet<>();
   s.forEach(element -> {
       if (!set.add(element)) {
           duplicates.add(element);
       }
   });
   return duplicates;
   ```
   Could we also add a short javadoc explaining the return value only contains each duplicate value once?



##########
core/src/test/scala/unit/kafka/utils/TestUtils.scala:
##########
@@ -2395,6 +2395,8 @@ object TestUtils extends Logging {
     )
 
     def unexpectedThreads: Set[String] = {
+      val x = Thread.getAllStackTraces.asScala.filter(entry => unexpectedThreadNames.exists(s => entry._1.getName.contains(s)))

Review Comment:
   I guess it's a leftover from some debugging?



##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -654,7 +651,7 @@ public List<String> getTopics(Optional<String> topicIncludeList, boolean exclude
         public List<Uuid> getTopicIds(Uuid topicIdIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
             ListTopicsResult allTopics = excludeInternalTopics ? adminClient.listTopics() :
                 adminClient.listTopics(new ListTopicsOptions().listInternal(true));
-            List<Uuid> allTopicIds = null;
+            List<Uuid> allTopicIds;

Review Comment:
   Since we assign it just below, could we just do:
   ```
   List<Uuid> allTopicIds = allTopics.listings().get().stream()
           .map(TopicListing::topicId)
           .sorted()
   ```?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1749188089

   > @OmniaGM Thanks for the PR. Can you rebase on trunk to resolve the conflicts?
   
   Updated the pr waiting now for the pipeline to pass


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1594708038

   @OmniaGM can you please rebase this with trunk and I would be happy to begin a review for this.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on a diff in pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1297476982


##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String[] args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {
+            System.err.println(e.getMessage());
+            return 1;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        opts.checkArgs();
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        log.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")
+    static Map<Integer, List<Integer>> parseReplicaAssignment(String replicaAssignmentList) {
+        String[] partitionList = replicaAssignmentList.split(",");
+        Map<Integer, List<Integer>> ret = new LinkedHashMap<Integer, List<Integer>>();
+        for (int i = 0; i < partitionList.length; i++) {
+            List<Integer> brokerList = Arrays.stream(partitionList[i].split(":"))
+                .map(String::trim)
+                .mapToInt(Integer::parseInt)
+                .boxed()
+                .collect(Collectors.toList());
+            Collection<Integer> duplicateBrokers = ToolsUtils.duplicates(brokerList);
+            if (!duplicateBrokers.isEmpty()) {
+                throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: " +
+                    duplicateBrokers.stream()
+                        .map(Object::toString)
+                        .collect(Collectors.joining(","))
+                );
+            }
+            ret.put(i, brokerList);
+            if (ret.get(i).size() != ret.get(0).size()) {
+                throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList);
+            }
+        }
+        return ret;
+    }
+
+    @SuppressWarnings("deprecation")
+    private static Properties parseTopicConfigsToBeAdded(TopicCommandOptions opts) {
+        List<List<String>> configsToBeAdded = opts.topicConfig().orElse(Collections.emptyList())
+            .stream()
+            .map(s -> Arrays.asList(s.split("\\s*=\\s*")))
+            .collect(Collectors.toList());
+
+        if (!configsToBeAdded.stream().allMatch(config -> config.size() == 2)) {
+            throw new IllegalArgumentException("requirement failed: Invalid topic config: all configs to be added must be in the format \"key=val\".");
+        }
+
+        Properties props = new Properties();
+        configsToBeAdded.stream()
+            .forEach(pair -> props.setProperty(pair.get(0).trim(), pair.get(1).trim()));
+        LogConfig.validate(props);
+        if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) {
+            System.out.println("WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " +
+                "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " +
+                "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.");
+        }
+        return props;
+    }
+
+    // It is possible for a reassignment to complete between the time we have fetched its state and the time
+    // we fetch partition metadata. In this case, we ignore the reassignment when determining replication factor.
+    public static boolean isReassignmentInProgress(TopicPartitionInfo tpi, PartitionReassignment ra) {
+        // Reassignment is still in progress as long as the removing and adding replicas are still present
+        Set<Integer> allReplicaIds = tpi.replicas().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> changingReplicaIds = new HashSet<>();
+        if (ra != null) {
+            changingReplicaIds.addAll(ra.removingReplicas());
+            changingReplicaIds.addAll(ra.addingReplicas());
+        }
+        return allReplicaIds.stream().anyMatch(changingReplicaIds::contains);
+
+    }
+
+    private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionReassignment reassignment) {
+        return isReassignmentInProgress(tpi, reassignment) ?
+            reassignment.replicas().size() - reassignment.addingReplicas().size() :
+            tpi.replicas().size();
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopics        Topics that were found to match the requested topic name.
+     * @param requestedTopic     Name of the topic that was requested.
+     * @param requireTopicExists Indicates if the topic needs to exist for the operation to be successful.
+     *                           If set to true, the command will throw an exception if the topic with the
+     *                           requested name does not exist.
+     */
+    private static void ensureTopicExists(List<String> foundTopics, String requestedTopic, Boolean requireTopicExists) {
+        // If no topic name was mentioned, do not need to throw exception.
+        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
+            // If given topic doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
+        }
+    }
+
+    private static List<String> doGetTopics(List<String> allTopics, Optional<String> topicIncludeList, Boolean excludeInternalTopics) {
+        if (topicIncludeList.isPresent()) {
+            IncludeList topicsFilter = new IncludeList(topicIncludeList.get());
+            return allTopics.stream()
+                .filter(topic -> topicsFilter.isTopicAllowed(topic, excludeInternalTopics))
+                .collect(Collectors.toList());
+        } else {
+            return allTopics.stream()
+                .filter(topic -> !(Topic.isInternal(topic) && excludeInternalTopics))
+                .collect(Collectors.toList());
+        }
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopicIds        Topics that were found to match the requested topic id.
+     * @param requestedTopicId     Id of the topic that was requested.
+     * @param requireTopicIdExists Indicates if the topic needs to exist for the operation to be successful.
+     *                             If set to true, the command will throw an exception if the topic with the
+     *                             requested id does not exist.
+     */
+    private static void ensureTopicIdExists(List<Uuid> foundTopicIds, Uuid requestedTopicId, Boolean requireTopicIdExists) {
+        // If no topic id was mentioned, do not need to throw exception.
+        if (requestedTopicId != null && requireTopicIdExists && foundTopicIds.isEmpty()) {
+            // If given topicId doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("TopicId '%s' does not exist as expected", requestedTopicId));
+        }
+    }
+
+    static class CommandTopicPartition {
+        private final TopicCommandOptions opts;
+        private final Optional<String> name;
+        private final Optional<Integer> replicationFactor;
+        private final Optional<Integer> partitions;
+        private final Map<Integer, List<Integer>> replicaAssignment;
+        private final Properties configsToAdd;
+
+        public CommandTopicPartition(TopicCommandOptions options) {
+            opts = options;
+            name = options.topic();
+            partitions = options.partitions();
+            replicationFactor = options.replicationFactor();
+            replicaAssignment = options.replicaAssignment().orElse(Collections.emptyMap());
+            configsToAdd = parseTopicConfigsToBeAdded(options);
+        }
+
+        public Boolean hasReplicaAssignment() {
+            return !replicaAssignment.isEmpty();
+        }
+
+        public Boolean hasPartitions() {
+            return partitions.isPresent();
+        }
+
+        public Boolean ifTopicDoesntExist() {
+            return opts.ifNotExists();
+        }
+    }
+
+    static class TopicDescription {
+        private final String topic;
+        private final Uuid topicId;
+        private final Integer numPartitions;
+        private final Integer replicationFactor;
+        private final Config config;
+        private final Boolean markedForDeletion;
+
+        public TopicDescription(String topic, Uuid topicId, Integer numPartitions, Integer replicationFactor, Config config, Boolean markedForDeletion) {
+            this.topic = topic;
+            this.topicId = topicId;
+            this.numPartitions = numPartitions;
+            this.replicationFactor = replicationFactor;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+        }
+
+        public void printDescription() {
+            String configsAsString = config.entries().stream()
+                .filter(config -> !config.isDefault())
+                .map(ce -> String.format("%s=%s", ce.name(), ce.value()))
+                .collect(Collectors.joining(","));
+            System.out.print(String.format("Topic: %s", topic));
+            if (topicId != Uuid.ZERO_UUID)
+                System.out.print(String.format("\tTopicId: %s", topicId));
+            System.out.print(String.format("\tPartitionCount: %s", numPartitions));
+            System.out.print(String.format("\tReplicationFactor: %s", replicationFactor));
+            System.out.print(String.format("\tConfigs: %s", configsAsString));
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class PartitionDescription {
+        private final String topic;
+        private final TopicPartitionInfo info;
+        private final Config config;
+        private final Boolean markedForDeletion;
+        private final PartitionReassignment reassignment;
+
+        PartitionDescription(String topic,
+                             TopicPartitionInfo info,
+                             Config config,
+                             Boolean markedForDeletion,
+                             PartitionReassignment reassignment) {
+            this.topic = topic;
+            this.info = info;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+            this.reassignment = reassignment;
+        }
+
+        public Integer minIsrCount() {
+            return Integer.parseInt(config.get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value());
+        }
+
+        public Boolean isUnderReplicated() {
+            return getReplicationFactor(info, reassignment) - info.isr().size() > 0;
+        }
+
+        public boolean hasLeader() {
+            return info.leader() != null;
+        }
+
+        public Boolean isUnderMinIsr() {
+            return !hasLeader() ||  info.isr().size() < minIsrCount();
+        }
+
+        public Boolean isAtMinIsrPartitions() {
+            return minIsrCount() == info.isr().size();
+
+        }
+
+        public Boolean hasUnavailablePartitions(Set<Integer> liveBrokers) {
+            return !hasLeader() || !liveBrokers.contains(info.leader().id());
+        }
+
+        public void printDescription() {
+            System.out.print("\tTopic: " + topic);
+            System.out.print("\tPartition: " + info.partition());
+            System.out.print("\tLeader: " + (hasLeader() ? info.leader().id() : "none"));
+            System.out.print("\tReplicas: " + info.replicas().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            System.out.print("\tIsr: " + info.isr().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            if (reassignment != null) {
+                System.out.print("\tAdding Replicas: " + reassignment.addingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+                System.out.print("\tRemoving Replicas: " + reassignment.removingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+            }
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+
+    }
+
+    static class DescribeOptions {
+        private final TopicCommandOptions opts;
+        private final Set<Integer> liveBrokers;
+        private final boolean describeConfigs;
+        private final boolean describePartitions;
+
+        public DescribeOptions(TopicCommandOptions opts, Set<Integer> liveBrokers) {
+            this.opts = opts;
+            this.liveBrokers = liveBrokers;
+            this.describeConfigs = !opts.reportUnavailablePartitions() &&
+                !opts.reportUnderReplicatedPartitions() &&
+                !opts.reportUnderMinIsrPartitions() &&
+                !opts.reportAtMinIsrPartitions();
+            this.describePartitions = !opts.reportOverriddenConfigs();
+        }
+
+        private boolean shouldPrintUnderReplicatedPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderReplicatedPartitions() && partitionDescription.isUnderReplicated();
+        }
+
+        private boolean shouldPrintUnavailablePartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnavailablePartitions() && partitionDescription.hasUnavailablePartitions(liveBrokers);
+        }
+
+        private boolean shouldPrintUnderMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderMinIsrPartitions() && partitionDescription.isUnderMinIsr();
+        }
+
+        private boolean shouldPrintAtMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportAtMinIsrPartitions() && partitionDescription.isAtMinIsrPartitions();
+        }
+
+        private boolean shouldPrintTopicPartition(PartitionDescription partitionDesc) {
+            return describeConfigs ||
+                shouldPrintUnderReplicatedPartitions(partitionDesc) ||
+                shouldPrintUnavailablePartitions(partitionDesc) ||
+                shouldPrintUnderMinIsrPartitions(partitionDesc) ||
+                shouldPrintAtMinIsrPartitions(partitionDesc);
+        }
+
+        public void maybePrintPartitionDescription(PartitionDescription desc) {
+            if (shouldPrintTopicPartition(desc)) {
+                desc.printDescription();
+            }
+        }
+    }
+
+    public static class TopicService implements AutoCloseable {
+        private Admin adminClient;
+
+        public TopicService(Properties commandConfig, Optional<String> bootstrapServer) {
+            this.adminClient = createAdminClient(commandConfig, bootstrapServer);
+        }
+
+        public TopicService(Admin admin) {
+            this.adminClient = admin;
+        }
+
+        private static Admin createAdminClient(Properties commandConfig, Optional<String> bootstrapServer) {
+            if (bootstrapServer.isPresent()) {
+                commandConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+            }
+            return Admin.create(commandConfig);
+        }
+
+        public void createTopic(TopicCommandOptions opts) throws Exception {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            if (Topic.hasCollisionChars(topic.name.get())) {
+                System.out.println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could " +
+                    "collide. To avoid issues it is best to use either, but not both.");
+            }
+            createTopic(topic);
+        }
+
+        public void createTopic(CommandTopicPartition topic) throws Exception {
+            if (topic.replicationFactor.filter(rf -> rf > Short.MAX_VALUE || rf < 1).isPresent()) {
+                throw new IllegalArgumentException("The replication factor must be between 1 and " + Short.MAX_VALUE + " inclusive");
+            }
+            if (topic.partitions.filter(p -> p < 1).isPresent()) {
+                throw new IllegalArgumentException("The partitions must be greater than 0");
+            }
+
+            try {
+                NewTopic newTopic;
+                if (topic.hasReplicaAssignment()) {
+                    newTopic = new NewTopic(topic.name.get(), topic.replicaAssignment);
+                } else {
+                    newTopic = new NewTopic(topic.name.get(), topic.partitions, topic.replicationFactor.map(Integer::shortValue));
+                }
+
+                Map<String, String> configsMap = topic.configsToAdd.stringPropertyNames().stream()
+                    .collect(Collectors.toMap(name -> name, name -> topic.configsToAdd.getProperty(name)));
+
+                newTopic.configs(configsMap);
+                CreateTopicsResult createResult = adminClient.createTopics(Collections.singleton(newTopic),
+                    new CreateTopicsOptions().retryOnQuotaViolation(false));
+                createResult.all().get();
+                System.out.println("Created topic " + topic.name + ".");
+            } catch (ExecutionException e) {
+                if (e.getCause() == null) {
+                    throw e;
+                }
+                if (!(e.getCause() instanceof TopicExistsException && topic.ifTopicDoesntExist())) {
+                    throw (Exception) e.getCause();
+                }
+            }
+        }
+
+        public void listTopics(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            String results = getTopics(opts.topic(), opts.excludeInternalTopics())
+                .stream()
+                .collect(Collectors.joining("\n"));
+            System.out.println(results);
+        }
+
+        public void alterTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+
+            if (!topics.isEmpty()) {
+                Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo = adminClient.describeTopics(topics).topicNameValues();
+                Map<String, NewPartitions> newPartitions = topics.stream()
+                    .map(topicName -> topicNewPartitions(topic, topicsInfo, topicName))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                adminClient.createPartitions(newPartitions, new CreatePartitionsOptions().retryOnQuotaViolation(false)).all().get();
+            }
+        }
+
+        private AbstractMap.SimpleEntry<String, NewPartitions> topicNewPartitions(
+            CommandTopicPartition topic,
+            Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo,
+            String topicName) {
+            if (topic.hasReplicaAssignment()) {
+                try {
+                    Integer startPartitionId = topicsInfo.get(topicName).get().partitions().size();
+                    Map<Integer, List<Integer>> replicaMap = topic.replicaAssignment.entrySet().stream()
+                        .skip(startPartitionId)
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                    List<List<Integer>> newAssignment = replicaMap.entrySet().stream()
+                        .map(Map.Entry::getValue)
+                        .collect(Collectors.toList());
+                    return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get(), newAssignment));
+                } catch (InterruptedException | ExecutionException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+            return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get()));
+        }
+
+        public Map<TopicPartition, PartitionReassignment> listAllReassignments(Set<TopicPartition> topicPartitions) {
+            try {
+                return adminClient.listPartitionReassignments(topicPartitions).reassignments().get();
+            } catch (ExecutionException e) {
+                Throwable cause = e.getCause();
+                if (cause instanceof UnsupportedVersionException || cause instanceof ClusterAuthorizationException) {
+                    log.debug("Couldn't query reassignments through the AdminClient API: " + cause.getMessage(), cause);
+                    return Collections.emptyMap();
+                } else {
+                    throw new RuntimeException(e);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        public void describeTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            // If topicId is provided and not zero, will use topicId regardless of topic name
+            Optional<Uuid> inputTopicId = opts.topicId()
+                .map(Uuid::fromString).filter(uuid -> uuid != Uuid.ZERO_UUID);
+            Boolean useTopicId = inputTopicId.isPresent();
+
+            List<Uuid> topicIds;
+            List<String> topics;
+            if (useTopicId) {
+                topicIds = getTopicIds(inputTopicId.get(), opts.excludeInternalTopics());
+                topics = Collections.emptyList();
+            } else {
+                topicIds = Collections.emptyList();
+                topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+

Review Comment:
   removed it



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1360418599


##########
core/src/test/scala/unit/kafka/utils/TestUtils.scala:
##########
@@ -2395,6 +2395,8 @@ object TestUtils extends Logging {
     )
 
     def unexpectedThreads: Set[String] = {
+      val x = Thread.getAllStackTraces.asScala.filter(entry => unexpectedThreadNames.exists(s => entry._1.getName.contains(s)))

Review Comment:
   yup :D good catch I was debugging some hanging thread issues 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "mimaison (via GitHub)" <gi...@apache.org>.
mimaison commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1764452479

   With the latest updates it's not failing anymore on my laptop. I'll let the Apache CI run before merging.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1717877198

   > Hi @OmniaGM, thanks. I left some comments.
   > 
   > Additionally, this error message is different from previous implementation:
   > 
   > ```shell
   > ### OLD
   > $ bin/kafka-topics.sh --bootstrap-server :9092 --describe --topic my-topic-foo --delete-config sdsad
   > Option "[delete-config]" can't be used with option "[describe]"
   > 
   > ### NEW
   > $ bin/kafka-topics.sh --bootstrap-server :9092 --describe --topic my-topic-foo --delete-config sdsad
   > Option "[delete-config]" can't be used with option "[bootstrap-server]"
   > ```
   
   Good catch @fvaleri , I missed that `immutable.Set` which is used to define invalid options in old scala code does preserve the order of insertion so when we check `bootstrap` option isn't the first on the list to compare but instead it's `describe`. I updated the code to use `LinkedHashSet` instead of `Set` in Java.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on a diff in pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1327326980


##########
build.gradle:
##########
@@ -1880,9 +1880,10 @@ project(':tools:tools-api') {
 
 project(':tools') {
   archivesBaseName = "kafka-tools"
-
   dependencies {
+    api libs.scalaLibrary

Review Comment:
   You right, my idea was acting up when I remove it but it was wrong state of idea. I remove it along side few other dependencies that we don't need.  



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1358468634


##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1007 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+public abstract class TopicCommand {
+    private static final Logger LOG = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String... args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        LOG.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")
+    static Map<Integer, List<Integer>> parseReplicaAssignment(String replicaAssignmentList) {
+        String[] partitionList = replicaAssignmentList.split(",");
+        Map<Integer, List<Integer>> ret = new LinkedHashMap<Integer, List<Integer>>();
+        for (int i = 0; i < partitionList.length; i++) {
+            List<Integer> brokerList = Arrays.stream(partitionList[i].split(":"))
+                .map(String::trim)
+                .mapToInt(Integer::parseInt)
+                .boxed()
+                .collect(Collectors.toList());
+            Collection<Integer> duplicateBrokers = ToolsUtils.duplicates(brokerList);
+            if (!duplicateBrokers.isEmpty()) {
+                throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: " +
+                    duplicateBrokers.stream()
+                        .map(Object::toString)
+                        .collect(Collectors.joining(","))
+                );
+            }
+            ret.put(i, brokerList);
+            if (ret.get(i).size() != ret.get(0).size()) {
+                throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList);
+            }
+        }
+        return ret;
+    }
+
+    @SuppressWarnings("deprecation")
+    private static Properties parseTopicConfigsToBeAdded(TopicCommandOptions opts) {
+        List<List<String>> configsToBeAdded = opts.topicConfig().orElse(Collections.emptyList())
+            .stream()
+            .map(s -> Arrays.asList(s.split("\\s*=\\s*")))
+            .collect(Collectors.toList());
+
+        if (!configsToBeAdded.stream().allMatch(config -> config.size() == 2)) {
+            throw new IllegalArgumentException("requirement failed: Invalid topic config: all configs to be added must be in the format \"key=val\".");
+        }
+
+        Properties props = new Properties();
+        configsToBeAdded.stream()
+            .forEach(pair -> props.setProperty(pair.get(0).trim(), pair.get(1).trim()));
+        LogConfig.validate(props);
+        if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) {
+            System.out.println("WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " +
+                "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " +
+                "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.");
+        }
+        return props;
+    }
+
+    // It is possible for a reassignment to complete between the time we have fetched its state and the time
+    // we fetch partition metadata. In this case, we ignore the reassignment when determining replication factor.
+    public static boolean isReassignmentInProgress(TopicPartitionInfo tpi, PartitionReassignment ra) {
+        // Reassignment is still in progress as long as the removing and adding replicas are still present
+        Set<Integer> allReplicaIds = tpi.replicas().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> changingReplicaIds = new HashSet<>();
+        if (ra != null) {
+            changingReplicaIds.addAll(ra.removingReplicas());
+            changingReplicaIds.addAll(ra.addingReplicas());
+        }
+        return allReplicaIds.stream().anyMatch(changingReplicaIds::contains);
+
+    }
+
+    private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionReassignment reassignment) {
+        return isReassignmentInProgress(tpi, reassignment) ?
+            reassignment.replicas().size() - reassignment.addingReplicas().size() :
+            tpi.replicas().size();
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopics        Topics that were found to match the requested topic name.
+     * @param requestedTopic     Name of the topic that was requested.
+     * @param requireTopicExists Indicates if the topic needs to exist for the operation to be successful.
+     *                           If set to true, the command will throw an exception if the topic with the
+     *                           requested name does not exist.
+     */
+    private static void ensureTopicExists(List<String> foundTopics, String requestedTopic, Boolean requireTopicExists) {
+        // If no topic name was mentioned, do not need to throw exception.
+        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
+            // If given topic doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
+        }
+    }
+
+    private static List<String> doGetTopics(List<String> allTopics, Optional<String> topicIncludeList, Boolean excludeInternalTopics) {
+        if (topicIncludeList.isPresent()) {
+            IncludeList topicsFilter = new IncludeList(topicIncludeList.get());
+            return allTopics.stream()
+                .filter(topic -> topicsFilter.isTopicAllowed(topic, excludeInternalTopics))
+                .collect(Collectors.toList());
+        } else {
+            return allTopics.stream()
+                .filter(topic -> !(Topic.isInternal(topic) && excludeInternalTopics))
+                .collect(Collectors.toList());
+        }
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopicIds        Topics that were found to match the requested topic id.
+     * @param requestedTopicId     Id of the topic that was requested.
+     * @param requireTopicIdExists Indicates if the topic needs to exist for the operation to be successful.
+     *                             If set to true, the command will throw an exception if the topic with the
+     *                             requested id does not exist.
+     */
+    private static void ensureTopicIdExists(List<Uuid> foundTopicIds, Uuid requestedTopicId, Boolean requireTopicIdExists) {
+        // If no topic id was mentioned, do not need to throw exception.
+        if (requestedTopicId != null && requireTopicIdExists && foundTopicIds.isEmpty()) {
+            // If given topicId doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("TopicId '%s' does not exist as expected", requestedTopicId));
+        }
+    }
+
+    static class CommandTopicPartition {
+        private final Optional<String> name;
+        private final Optional<Integer> partitions;
+        private final Optional<Integer> replicationFactor;
+        private final Map<Integer, List<Integer>> replicaAssignment;
+        private final Properties configsToAdd;
+
+        private final TopicCommandOptions opts;
+
+        public CommandTopicPartition(TopicCommandOptions options) {
+            opts = options;
+            name = options.topic();
+            partitions = options.partitions();
+            replicationFactor = options.replicationFactor();
+            replicaAssignment = options.replicaAssignment().orElse(Collections.emptyMap());
+            configsToAdd = parseTopicConfigsToBeAdded(options);
+        }
+
+        public Boolean hasReplicaAssignment() {
+            return !replicaAssignment.isEmpty();
+        }
+
+        public Boolean ifTopicDoesntExist() {
+            return opts.ifNotExists();
+        }
+    }
+
+    static class TopicDescription {
+        private final String topic;
+        private final Uuid topicId;
+        private final Integer numPartitions;
+        private final Integer replicationFactor;
+        private final Config config;
+        private final Boolean markedForDeletion;
+
+        public TopicDescription(String topic, Uuid topicId, Integer numPartitions, Integer replicationFactor, Config config, Boolean markedForDeletion) {
+            this.topic = topic;
+            this.topicId = topicId;
+            this.numPartitions = numPartitions;
+            this.replicationFactor = replicationFactor;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+        }
+
+        public void printDescription() {
+            String configsAsString = config.entries().stream()
+                .filter(config -> !config.isDefault())
+                .map(ce -> ce.name() + "=" + ce.value())
+                .collect(Collectors.joining(","));
+            System.out.print("Topic: " +  topic);
+            if (topicId != Uuid.ZERO_UUID)
+                System.out.print("\tTopicId: " + topicId);
+            System.out.print("\tPartitionCount: " + numPartitions);
+            System.out.print("\tReplicationFactor: " + replicationFactor);
+            System.out.print("\tConfigs: " + configsAsString);
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class PartitionDescription {
+        private final String topic;
+        private final TopicPartitionInfo info;
+        private final Config config;
+        private final Boolean markedForDeletion;
+        private final PartitionReassignment reassignment;
+
+        PartitionDescription(String topic,
+                             TopicPartitionInfo info,
+                             Config config,
+                             Boolean markedForDeletion,
+                             PartitionReassignment reassignment) {
+            this.topic = topic;
+            this.info = info;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+            this.reassignment = reassignment;
+        }
+
+        public Integer minIsrCount() {
+            return Integer.parseInt(config.get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value());
+        }
+
+        public Boolean isUnderReplicated() {
+            return getReplicationFactor(info, reassignment) - info.isr().size() > 0;
+        }
+
+        public boolean hasLeader() {
+            return info.leader() != null;
+        }
+
+        public Boolean isUnderMinIsr() {
+            return !hasLeader() ||  info.isr().size() < minIsrCount();
+        }
+
+        public Boolean isAtMinIsrPartitions() {
+            return minIsrCount() == info.isr().size();
+        }
+
+        public Boolean hasUnavailablePartitions(Set<Integer> liveBrokers) {
+            return !hasLeader() || !liveBrokers.contains(info.leader().id());
+        }
+
+        public void printDescription() {
+            System.out.print("\tTopic: " + topic);
+            System.out.print("\tPartition: " + info.partition());
+            System.out.print("\tLeader: " + (hasLeader() ? info.leader().id() : "none"));
+            System.out.print("\tReplicas: " + info.replicas().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            System.out.print("\tIsr: " + info.isr().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            if (reassignment != null) {
+                System.out.print("\tAdding Replicas: " + reassignment.addingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+                System.out.print("\tRemoving Replicas: " + reassignment.removingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+            }
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class DescribeOptions {
+        private final TopicCommandOptions opts;
+        private final Set<Integer> liveBrokers;
+        private final boolean describeConfigs;
+        private final boolean describePartitions;
+
+        public DescribeOptions(TopicCommandOptions opts, Set<Integer> liveBrokers) {
+            this.opts = opts;
+            this.liveBrokers = liveBrokers;
+            this.describeConfigs = !opts.reportUnavailablePartitions() &&
+                !opts.reportUnderReplicatedPartitions() &&
+                !opts.reportUnderMinIsrPartitions() &&
+                !opts.reportAtMinIsrPartitions();
+            this.describePartitions = !opts.reportOverriddenConfigs();
+        }
+
+        private boolean shouldPrintUnderReplicatedPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderReplicatedPartitions() && partitionDescription.isUnderReplicated();
+        }
+
+        private boolean shouldPrintUnavailablePartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnavailablePartitions() && partitionDescription.hasUnavailablePartitions(liveBrokers);
+        }
+
+        private boolean shouldPrintUnderMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderMinIsrPartitions() && partitionDescription.isUnderMinIsr();
+        }
+
+        private boolean shouldPrintAtMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportAtMinIsrPartitions() && partitionDescription.isAtMinIsrPartitions();
+        }
+
+        private boolean shouldPrintTopicPartition(PartitionDescription partitionDesc) {
+            return describeConfigs ||
+                shouldPrintUnderReplicatedPartitions(partitionDesc) ||
+                shouldPrintUnavailablePartitions(partitionDesc) ||
+                shouldPrintUnderMinIsrPartitions(partitionDesc) ||
+                shouldPrintAtMinIsrPartitions(partitionDesc);
+        }
+
+        public void maybePrintPartitionDescription(PartitionDescription desc) {
+            if (shouldPrintTopicPartition(desc)) {
+                desc.printDescription();
+            }
+        }
+    }
+
+    public static class TopicService implements AutoCloseable {
+        private Admin adminClient;
+
+        public TopicService(Properties commandConfig, Optional<String> bootstrapServer) {
+            this.adminClient = createAdminClient(commandConfig, bootstrapServer);
+        }
+
+        public TopicService(Admin admin) {
+            this.adminClient = admin;
+        }
+
+        private static Admin createAdminClient(Properties commandConfig, Optional<String> bootstrapServer) {
+            if (bootstrapServer.isPresent()) {
+                commandConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer.get());
+            }
+            return Admin.create(commandConfig);
+        }
+
+        public void createTopic(TopicCommandOptions opts) throws Exception {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            if (Topic.hasCollisionChars(topic.name.get())) {
+                System.out.println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could " +
+                    "collide. To avoid issues it is best to use either, but not both.");
+            }
+            createTopic(topic);
+        }
+
+        public void createTopic(CommandTopicPartition topic) throws Exception {
+            if (topic.replicationFactor.filter(rf -> rf > Short.MAX_VALUE || rf < 1).isPresent()) {
+                throw new IllegalArgumentException("The replication factor must be between 1 and " + Short.MAX_VALUE + " inclusive");
+            }
+            if (topic.partitions.filter(p -> p < 1).isPresent()) {
+                throw new IllegalArgumentException("The partitions must be greater than 0");
+            }
+
+            try {
+                NewTopic newTopic;
+                if (topic.hasReplicaAssignment()) {
+                    newTopic = new NewTopic(topic.name.get(), topic.replicaAssignment);
+                } else {
+                    newTopic = new NewTopic(topic.name.get(), topic.partitions, topic.replicationFactor.map(Integer::shortValue));
+                }
+
+                Map<String, String> configsMap = topic.configsToAdd.stringPropertyNames().stream()
+                    .collect(Collectors.toMap(name -> name, name -> topic.configsToAdd.getProperty(name)));
+
+                newTopic.configs(configsMap);
+                CreateTopicsResult createResult = adminClient.createTopics(Collections.singleton(newTopic),
+                    new CreateTopicsOptions().retryOnQuotaViolation(false));
+                createResult.all().get();
+                System.out.println("Created topic " + topic.name + ".");
+            } catch (ExecutionException e) {
+                if (e.getCause() == null) {
+                    throw e;
+                }
+                if (!(e.getCause() instanceof TopicExistsException && topic.ifTopicDoesntExist())) {
+                    throw (Exception) e.getCause();
+                }
+            }
+        }
+
+        public void listTopics(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            String results = getTopics(opts.topic(), opts.excludeInternalTopics())
+                .stream()
+                .collect(Collectors.joining("\n"));
+            System.out.println(results);
+        }
+
+        public void alterTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+
+            if (!topics.isEmpty()) {
+                Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo = adminClient.describeTopics(topics).topicNameValues();
+                Map<String, NewPartitions> newPartitions = topics.stream()
+                    .map(topicName -> topicNewPartitions(topic, topicsInfo, topicName))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                adminClient.createPartitions(newPartitions, new CreatePartitionsOptions().retryOnQuotaViolation(false)).all().get();
+            }
+        }
+
+        private AbstractMap.SimpleEntry<String, NewPartitions> topicNewPartitions(
+            CommandTopicPartition topic,
+            Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo,
+            String topicName) {
+            if (topic.hasReplicaAssignment()) {
+                try {
+                    Integer startPartitionId = topicsInfo.get(topicName).get().partitions().size();
+                    Map<Integer, List<Integer>> replicaMap = topic.replicaAssignment.entrySet().stream()
+                        .skip(startPartitionId)
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                    List<List<Integer>> newAssignment = replicaMap.entrySet().stream()

Review Comment:
   good shout 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1358471763


##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1007 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+public abstract class TopicCommand {
+    private static final Logger LOG = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String... args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        LOG.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")

Review Comment:
   not anymore I think I needed it when I was converting some scala to java at some point and forgot to remove this one.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1627731940

   > Hi @OmniaGM.
   > 
   > > Move TopicFilter out of core. (This is part of https://issues.apache.org/jira/browse/KAFKA-14647 which seems not moving for a while )
   > 
   > I think #13158 is ready. I'll try to drive some attention to it.
   > 
   > > Move CoreUtils.duplicate to ToolsUtils
   > > Duplicate AdminCommandFailedException and AdminOperationException out of core.
   > 
   > I looks like `AdminCommandFailedException` and `AdminOperationException` are being moved in #13278. This also seems to be almost ready.
   
   Will update this PR once these prs are merged. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1358474706


##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import kafka.utils.Exit;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientTestUtils;
+import org.apache.kafka.clients.admin.CreatePartitionsResult;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.DeleteTopicsResult;
+import org.apache.kafka.clients.admin.DescribeTopicsResult;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.errors.ThrottlingQuotaExceededException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyCollection;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.verify;
+
+@Timeout(value = 60)
+public class TopicCommandTest {
+    private String bootstrapServer = "localhost:9092";
+    private String topicName = "topicName";
+
+    @Test
+    public void testIsNotUnderReplicatedWhenAdding() {
+        List<Integer> replicaIds = Arrays.asList(1, 2);
+        List<Node> replicas = new ArrayList<>();
+        for (int id : replicaIds) {
+            replicas.add(new Node(id, "localhost", 9090 + id));
+        }
+
+        TopicCommand.PartitionDescription partitionDescription = new TopicCommand.PartitionDescription("test-topic",
+            new TopicPartitionInfo(0, new Node(1, "localhost", 9091), replicas,
+                Collections.singletonList(new Node(1, "localhost", 9091))),
+            null, false,
+                new PartitionReassignment(replicaIds, Arrays.asList(2), Collections.emptyList())
+        );
+
+        assertFalse(partitionDescription.isUnderReplicated());
+    }
+
+    @Test
+    public void testAlterWithUnspecifiedPartitionCount() {
+        String[] options = new String[] {" --bootstrap-server", bootstrapServer, "--alter", "--topic", topicName};
+        assertInitializeInvalidOptionsExitCode(1, options);
+    }
+
+    @Test
+    public void testConfigOptWithBootstrapServers() {
+        assertInitializeInvalidOptionsExitCode(1,
+            new String[] {"--bootstrap-server", bootstrapServer, "--alter", "--topic", topicName,
+                "--partitions", "3", "--config", "cleanup.policy=compact"});
+        assertInitializeInvalidOptionsExitCode(1,
+            new String[] {"--bootstrap-server", bootstrapServer, "--alter", "--topic", topicName,
+                "--partitions", "3", "--delete-config", "cleanup.policy"});
+        TopicCommand.TopicCommandOptions opts =
+            new TopicCommand.TopicCommandOptions(
+                new String[] {"--bootstrap-server", bootstrapServer, "--create", "--topic", topicName, "--partitions", "3",
+                    "--replication-factor", "3", "--config", "cleanup.policy=compact"});
+        assertTrue(opts.hasCreateOption());
+        assertEquals(bootstrapServer, opts.bootstrapServer().get());
+        assertEquals("cleanup.policy=compact", opts.topicConfig().get().get(0));
+    }
+
+    @Test
+    public void testCreateWithPartitionCountWithoutReplicationFactorShouldSucceed() {
+        TopicCommand.TopicCommandOptions opts = new TopicCommand.TopicCommandOptions(
+            new String[] {"--bootstrap-server", bootstrapServer,
+                "--create",
+                "--partitions", "2",
+                "--topic", topicName});
+        assertTrue(opts.hasCreateOption());
+        assertEquals(topicName, opts.topic().get());
+        assertEquals(2, opts.partitions().get());

Review Comment:
   `checkArgs()` is called in the constructor of  `TopicCommandOptions`. It is redundant to call  `opts.checkArgs` 



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import kafka.utils.Exit;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientTestUtils;
+import org.apache.kafka.clients.admin.CreatePartitionsResult;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.DeleteTopicsResult;
+import org.apache.kafka.clients.admin.DescribeTopicsResult;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.errors.ThrottlingQuotaExceededException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyCollection;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.verify;
+
+@Timeout(value = 60)
+public class TopicCommandTest {
+    private String bootstrapServer = "localhost:9092";
+    private String topicName = "topicName";
+
+    @Test
+    public void testIsNotUnderReplicatedWhenAdding() {
+        List<Integer> replicaIds = Arrays.asList(1, 2);
+        List<Node> replicas = new ArrayList<>();
+        for (int id : replicaIds) {
+            replicas.add(new Node(id, "localhost", 9090 + id));
+        }
+
+        TopicCommand.PartitionDescription partitionDescription = new TopicCommand.PartitionDescription("test-topic",
+            new TopicPartitionInfo(0, new Node(1, "localhost", 9091), replicas,
+                Collections.singletonList(new Node(1, "localhost", 9091))),
+            null, false,
+                new PartitionReassignment(replicaIds, Arrays.asList(2), Collections.emptyList())
+        );
+
+        assertFalse(partitionDescription.isUnderReplicated());
+    }
+
+    @Test
+    public void testAlterWithUnspecifiedPartitionCount() {
+        String[] options = new String[] {" --bootstrap-server", bootstrapServer, "--alter", "--topic", topicName};
+        assertInitializeInvalidOptionsExitCode(1, options);
+    }
+
+    @Test
+    public void testConfigOptWithBootstrapServers() {
+        assertInitializeInvalidOptionsExitCode(1,
+            new String[] {"--bootstrap-server", bootstrapServer, "--alter", "--topic", topicName,
+                "--partitions", "3", "--config", "cleanup.policy=compact"});
+        assertInitializeInvalidOptionsExitCode(1,
+            new String[] {"--bootstrap-server", bootstrapServer, "--alter", "--topic", topicName,
+                "--partitions", "3", "--delete-config", "cleanup.policy"});
+        TopicCommand.TopicCommandOptions opts =
+            new TopicCommand.TopicCommandOptions(
+                new String[] {"--bootstrap-server", bootstrapServer, "--create", "--topic", topicName, "--partitions", "3",
+                    "--replication-factor", "3", "--config", "cleanup.policy=compact"});
+        assertTrue(opts.hasCreateOption());
+        assertEquals(bootstrapServer, opts.bootstrapServer().get());
+        assertEquals("cleanup.policy=compact", opts.topicConfig().get().get(0));
+    }
+
+    @Test
+    public void testCreateWithPartitionCountWithoutReplicationFactorShouldSucceed() {
+        TopicCommand.TopicCommandOptions opts = new TopicCommand.TopicCommandOptions(
+            new String[] {"--bootstrap-server", bootstrapServer,
+                "--create",
+                "--partitions", "2",
+                "--topic", topicName});
+        assertTrue(opts.hasCreateOption());
+        assertEquals(topicName, opts.topic().get());
+        assertEquals(2, opts.partitions().get());
+    }
+
+    @Test
+    public void testCreateWithReplicationFactorWithoutPartitionCountShouldSucceed() {
+        TopicCommand.TopicCommandOptions opts = new TopicCommand.TopicCommandOptions(
+            new String[] {"--bootstrap-server", bootstrapServer,
+                "--create",
+                "--replication-factor", "3",
+                "--topic", topicName});
+        assertTrue(opts.hasCreateOption());
+        assertEquals(topicName, opts.topic().get());
+        assertEquals(3, opts.replicationFactor().get());

Review Comment:
   `checkArgs()` is called in the constructor of  `TopicCommandOptions`. It is redundant to call  `opts.checkArgs` 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1764360877

   > @OmniaGM thanks for the updates. I'm getting a test failure in TopicCommandIntegrationTest:
   > 
   > ```
   > Gradle Test Run :tools:test > Gradle Test Executor 31 > TopicCommandIntegrationTest > testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized(String) > testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized(String).quorum=kraft FAILED
   >     java.lang.RuntimeException: java.lang.IllegalArgumentException: Topic 'testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized-HbRT1wjRyV' does not exist as expected
   >         at org.apache.kafka.tools.TopicCommandIntegrationTest.lambda$captureDescribeTopicStandardOut$46(TopicCommandIntegrationTest.java:1006)
   >         at org.apache.kafka.tools.ToolsTestUtils.captureStandardStream(ToolsTestUtils.java:66)
   >         at org.apache.kafka.tools.ToolsTestUtils.captureStandardOut(ToolsTestUtils.java:50)
   >         at org.apache.kafka.tools.TopicCommandIntegrationTest.captureDescribeTopicStandardOut(TopicCommandIntegrationTest.java:1009)
   >         at org.apache.kafka.tools.TopicCommandIntegrationTest.testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized(TopicCommandIntegrationTest.java:943)
   > 
   >         Caused by:
   >         java.lang.IllegalArgumentException: Topic 'testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized-HbRT1wjRyV' does not exist as expected
   >             at org.apache.kafka.tools.TopicCommand.ensureTopicExists(TopicCommand.java:215)
   >             at org.apache.kafka.tools.TopicCommand.access$700(TopicCommand.java:78)
   >             at org.apache.kafka.tools.TopicCommand$TopicService.describeTopic(TopicCommand.java:559)
   >             at org.apache.kafka.tools.TopicCommandIntegrationTest.lambda$captureDescribeTopicStandardOut$46(TopicCommandIntegrationTest.java:1004)
   >             ... 4 more
   > ```
   
   Hi @mimaison I think this is flaky test I added `waitForTopicCreated(testTopicName);`. The test isn't failing locally with me so will wait for the pipeline to finish.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "mimaison (via GitHub)" <gi...@apache.org>.
mimaison merged PR #13201:
URL: https://github.com/apache/kafka/pull/13201


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on a diff in pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1297480256


##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String[] args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {
+            System.err.println(e.getMessage());
+            return 1;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        opts.checkArgs();
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        log.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")
+    static Map<Integer, List<Integer>> parseReplicaAssignment(String replicaAssignmentList) {
+        String[] partitionList = replicaAssignmentList.split(",");
+        Map<Integer, List<Integer>> ret = new LinkedHashMap<Integer, List<Integer>>();
+        for (int i = 0; i < partitionList.length; i++) {
+            List<Integer> brokerList = Arrays.stream(partitionList[i].split(":"))
+                .map(String::trim)
+                .mapToInt(Integer::parseInt)
+                .boxed()
+                .collect(Collectors.toList());
+            Collection<Integer> duplicateBrokers = ToolsUtils.duplicates(brokerList);
+            if (!duplicateBrokers.isEmpty()) {
+                throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: " +
+                    duplicateBrokers.stream()
+                        .map(Object::toString)
+                        .collect(Collectors.joining(","))
+                );
+            }
+            ret.put(i, brokerList);
+            if (ret.get(i).size() != ret.get(0).size()) {
+                throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList);
+            }
+        }
+        return ret;
+    }
+
+    @SuppressWarnings("deprecation")
+    private static Properties parseTopicConfigsToBeAdded(TopicCommandOptions opts) {
+        List<List<String>> configsToBeAdded = opts.topicConfig().orElse(Collections.emptyList())
+            .stream()
+            .map(s -> Arrays.asList(s.split("\\s*=\\s*")))
+            .collect(Collectors.toList());
+
+        if (!configsToBeAdded.stream().allMatch(config -> config.size() == 2)) {
+            throw new IllegalArgumentException("requirement failed: Invalid topic config: all configs to be added must be in the format \"key=val\".");
+        }
+
+        Properties props = new Properties();
+        configsToBeAdded.stream()
+            .forEach(pair -> props.setProperty(pair.get(0).trim(), pair.get(1).trim()));
+        LogConfig.validate(props);
+        if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) {
+            System.out.println("WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " +
+                "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " +
+                "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.");
+        }
+        return props;
+    }
+
+    // It is possible for a reassignment to complete between the time we have fetched its state and the time
+    // we fetch partition metadata. In this case, we ignore the reassignment when determining replication factor.
+    public static boolean isReassignmentInProgress(TopicPartitionInfo tpi, PartitionReassignment ra) {
+        // Reassignment is still in progress as long as the removing and adding replicas are still present
+        Set<Integer> allReplicaIds = tpi.replicas().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> changingReplicaIds = new HashSet<>();
+        if (ra != null) {
+            changingReplicaIds.addAll(ra.removingReplicas());
+            changingReplicaIds.addAll(ra.addingReplicas());
+        }
+        return allReplicaIds.stream().anyMatch(changingReplicaIds::contains);
+
+    }
+
+    private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionReassignment reassignment) {
+        return isReassignmentInProgress(tpi, reassignment) ?
+            reassignment.replicas().size() - reassignment.addingReplicas().size() :
+            tpi.replicas().size();
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopics        Topics that were found to match the requested topic name.
+     * @param requestedTopic     Name of the topic that was requested.
+     * @param requireTopicExists Indicates if the topic needs to exist for the operation to be successful.
+     *                           If set to true, the command will throw an exception if the topic with the
+     *                           requested name does not exist.
+     */
+    private static void ensureTopicExists(List<String> foundTopics, String requestedTopic, Boolean requireTopicExists) {
+        // If no topic name was mentioned, do not need to throw exception.
+        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
+            // If given topic doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
+        }
+    }
+
+    private static List<String> doGetTopics(List<String> allTopics, Optional<String> topicIncludeList, Boolean excludeInternalTopics) {
+        if (topicIncludeList.isPresent()) {
+            IncludeList topicsFilter = new IncludeList(topicIncludeList.get());
+            return allTopics.stream()
+                .filter(topic -> topicsFilter.isTopicAllowed(topic, excludeInternalTopics))
+                .collect(Collectors.toList());
+        } else {
+            return allTopics.stream()
+                .filter(topic -> !(Topic.isInternal(topic) && excludeInternalTopics))
+                .collect(Collectors.toList());
+        }
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopicIds        Topics that were found to match the requested topic id.
+     * @param requestedTopicId     Id of the topic that was requested.
+     * @param requireTopicIdExists Indicates if the topic needs to exist for the operation to be successful.
+     *                             If set to true, the command will throw an exception if the topic with the
+     *                             requested id does not exist.
+     */
+    private static void ensureTopicIdExists(List<Uuid> foundTopicIds, Uuid requestedTopicId, Boolean requireTopicIdExists) {
+        // If no topic id was mentioned, do not need to throw exception.
+        if (requestedTopicId != null && requireTopicIdExists && foundTopicIds.isEmpty()) {
+            // If given topicId doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("TopicId '%s' does not exist as expected", requestedTopicId));
+        }
+    }
+
+    static class CommandTopicPartition {
+        private final TopicCommandOptions opts;
+        private final Optional<String> name;
+        private final Optional<Integer> replicationFactor;
+        private final Optional<Integer> partitions;
+        private final Map<Integer, List<Integer>> replicaAssignment;
+        private final Properties configsToAdd;
+
+        public CommandTopicPartition(TopicCommandOptions options) {
+            opts = options;
+            name = options.topic();
+            partitions = options.partitions();
+            replicationFactor = options.replicationFactor();
+            replicaAssignment = options.replicaAssignment().orElse(Collections.emptyMap());
+            configsToAdd = parseTopicConfigsToBeAdded(options);
+        }
+
+        public Boolean hasReplicaAssignment() {
+            return !replicaAssignment.isEmpty();
+        }
+
+        public Boolean hasPartitions() {
+            return partitions.isPresent();
+        }
+
+        public Boolean ifTopicDoesntExist() {
+            return opts.ifNotExists();
+        }
+    }
+
+    static class TopicDescription {
+        private final String topic;
+        private final Uuid topicId;
+        private final Integer numPartitions;
+        private final Integer replicationFactor;
+        private final Config config;
+        private final Boolean markedForDeletion;
+
+        public TopicDescription(String topic, Uuid topicId, Integer numPartitions, Integer replicationFactor, Config config, Boolean markedForDeletion) {
+            this.topic = topic;
+            this.topicId = topicId;
+            this.numPartitions = numPartitions;
+            this.replicationFactor = replicationFactor;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+        }
+
+        public void printDescription() {
+            String configsAsString = config.entries().stream()
+                .filter(config -> !config.isDefault())
+                .map(ce -> String.format("%s=%s", ce.name(), ce.value()))
+                .collect(Collectors.joining(","));
+            System.out.print(String.format("Topic: %s", topic));
+            if (topicId != Uuid.ZERO_UUID)
+                System.out.print(String.format("\tTopicId: %s", topicId));
+            System.out.print(String.format("\tPartitionCount: %s", numPartitions));
+            System.out.print(String.format("\tReplicationFactor: %s", replicationFactor));
+            System.out.print(String.format("\tConfigs: %s", configsAsString));
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class PartitionDescription {
+        private final String topic;
+        private final TopicPartitionInfo info;
+        private final Config config;
+        private final Boolean markedForDeletion;
+        private final PartitionReassignment reassignment;
+
+        PartitionDescription(String topic,
+                             TopicPartitionInfo info,
+                             Config config,
+                             Boolean markedForDeletion,
+                             PartitionReassignment reassignment) {
+            this.topic = topic;
+            this.info = info;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+            this.reassignment = reassignment;
+        }
+
+        public Integer minIsrCount() {
+            return Integer.parseInt(config.get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value());
+        }
+
+        public Boolean isUnderReplicated() {
+            return getReplicationFactor(info, reassignment) - info.isr().size() > 0;
+        }
+
+        public boolean hasLeader() {
+            return info.leader() != null;
+        }
+
+        public Boolean isUnderMinIsr() {
+            return !hasLeader() ||  info.isr().size() < minIsrCount();
+        }
+
+        public Boolean isAtMinIsrPartitions() {
+            return minIsrCount() == info.isr().size();
+
+        }
+
+        public Boolean hasUnavailablePartitions(Set<Integer> liveBrokers) {
+            return !hasLeader() || !liveBrokers.contains(info.leader().id());
+        }
+
+        public void printDescription() {
+            System.out.print("\tTopic: " + topic);
+            System.out.print("\tPartition: " + info.partition());
+            System.out.print("\tLeader: " + (hasLeader() ? info.leader().id() : "none"));
+            System.out.print("\tReplicas: " + info.replicas().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            System.out.print("\tIsr: " + info.isr().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            if (reassignment != null) {
+                System.out.print("\tAdding Replicas: " + reassignment.addingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+                System.out.print("\tRemoving Replicas: " + reassignment.removingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+            }
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+
+    }
+
+    static class DescribeOptions {
+        private final TopicCommandOptions opts;
+        private final Set<Integer> liveBrokers;
+        private final boolean describeConfigs;
+        private final boolean describePartitions;
+
+        public DescribeOptions(TopicCommandOptions opts, Set<Integer> liveBrokers) {
+            this.opts = opts;
+            this.liveBrokers = liveBrokers;
+            this.describeConfigs = !opts.reportUnavailablePartitions() &&
+                !opts.reportUnderReplicatedPartitions() &&
+                !opts.reportUnderMinIsrPartitions() &&
+                !opts.reportAtMinIsrPartitions();
+            this.describePartitions = !opts.reportOverriddenConfigs();
+        }
+
+        private boolean shouldPrintUnderReplicatedPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderReplicatedPartitions() && partitionDescription.isUnderReplicated();
+        }
+
+        private boolean shouldPrintUnavailablePartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnavailablePartitions() && partitionDescription.hasUnavailablePartitions(liveBrokers);
+        }
+
+        private boolean shouldPrintUnderMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderMinIsrPartitions() && partitionDescription.isUnderMinIsr();
+        }
+
+        private boolean shouldPrintAtMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportAtMinIsrPartitions() && partitionDescription.isAtMinIsrPartitions();
+        }
+
+        private boolean shouldPrintTopicPartition(PartitionDescription partitionDesc) {
+            return describeConfigs ||
+                shouldPrintUnderReplicatedPartitions(partitionDesc) ||
+                shouldPrintUnavailablePartitions(partitionDesc) ||
+                shouldPrintUnderMinIsrPartitions(partitionDesc) ||
+                shouldPrintAtMinIsrPartitions(partitionDesc);
+        }
+
+        public void maybePrintPartitionDescription(PartitionDescription desc) {
+            if (shouldPrintTopicPartition(desc)) {
+                desc.printDescription();
+            }
+        }
+    }
+
+    public static class TopicService implements AutoCloseable {
+        private Admin adminClient;
+
+        public TopicService(Properties commandConfig, Optional<String> bootstrapServer) {
+            this.adminClient = createAdminClient(commandConfig, bootstrapServer);
+        }
+
+        public TopicService(Admin admin) {
+            this.adminClient = admin;
+        }
+
+        private static Admin createAdminClient(Properties commandConfig, Optional<String> bootstrapServer) {
+            if (bootstrapServer.isPresent()) {
+                commandConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+            }
+            return Admin.create(commandConfig);
+        }
+
+        public void createTopic(TopicCommandOptions opts) throws Exception {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            if (Topic.hasCollisionChars(topic.name.get())) {
+                System.out.println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could " +
+                    "collide. To avoid issues it is best to use either, but not both.");
+            }
+            createTopic(topic);
+        }
+
+        public void createTopic(CommandTopicPartition topic) throws Exception {
+            if (topic.replicationFactor.filter(rf -> rf > Short.MAX_VALUE || rf < 1).isPresent()) {
+                throw new IllegalArgumentException("The replication factor must be between 1 and " + Short.MAX_VALUE + " inclusive");
+            }
+            if (topic.partitions.filter(p -> p < 1).isPresent()) {
+                throw new IllegalArgumentException("The partitions must be greater than 0");
+            }
+
+            try {
+                NewTopic newTopic;
+                if (topic.hasReplicaAssignment()) {
+                    newTopic = new NewTopic(topic.name.get(), topic.replicaAssignment);
+                } else {
+                    newTopic = new NewTopic(topic.name.get(), topic.partitions, topic.replicationFactor.map(Integer::shortValue));
+                }
+
+                Map<String, String> configsMap = topic.configsToAdd.stringPropertyNames().stream()
+                    .collect(Collectors.toMap(name -> name, name -> topic.configsToAdd.getProperty(name)));
+
+                newTopic.configs(configsMap);
+                CreateTopicsResult createResult = adminClient.createTopics(Collections.singleton(newTopic),
+                    new CreateTopicsOptions().retryOnQuotaViolation(false));
+                createResult.all().get();
+                System.out.println("Created topic " + topic.name + ".");
+            } catch (ExecutionException e) {
+                if (e.getCause() == null) {
+                    throw e;
+                }
+                if (!(e.getCause() instanceof TopicExistsException && topic.ifTopicDoesntExist())) {
+                    throw (Exception) e.getCause();
+                }
+            }
+        }
+
+        public void listTopics(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            String results = getTopics(opts.topic(), opts.excludeInternalTopics())
+                .stream()
+                .collect(Collectors.joining("\n"));
+            System.out.println(results);
+        }
+
+        public void alterTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+
+            if (!topics.isEmpty()) {
+                Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo = adminClient.describeTopics(topics).topicNameValues();
+                Map<String, NewPartitions> newPartitions = topics.stream()
+                    .map(topicName -> topicNewPartitions(topic, topicsInfo, topicName))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                adminClient.createPartitions(newPartitions, new CreatePartitionsOptions().retryOnQuotaViolation(false)).all().get();
+            }
+        }
+
+        private AbstractMap.SimpleEntry<String, NewPartitions> topicNewPartitions(
+            CommandTopicPartition topic,
+            Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo,
+            String topicName) {
+            if (topic.hasReplicaAssignment()) {
+                try {
+                    Integer startPartitionId = topicsInfo.get(topicName).get().partitions().size();
+                    Map<Integer, List<Integer>> replicaMap = topic.replicaAssignment.entrySet().stream()
+                        .skip(startPartitionId)
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                    List<List<Integer>> newAssignment = replicaMap.entrySet().stream()
+                        .map(Map.Entry::getValue)
+                        .collect(Collectors.toList());
+                    return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get(), newAssignment));
+                } catch (InterruptedException | ExecutionException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+            return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get()));
+        }
+
+        public Map<TopicPartition, PartitionReassignment> listAllReassignments(Set<TopicPartition> topicPartitions) {
+            try {
+                return adminClient.listPartitionReassignments(topicPartitions).reassignments().get();
+            } catch (ExecutionException e) {
+                Throwable cause = e.getCause();
+                if (cause instanceof UnsupportedVersionException || cause instanceof ClusterAuthorizationException) {
+                    log.debug("Couldn't query reassignments through the AdminClient API: " + cause.getMessage(), cause);
+                    return Collections.emptyMap();
+                } else {
+                    throw new RuntimeException(e);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        public void describeTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            // If topicId is provided and not zero, will use topicId regardless of topic name
+            Optional<Uuid> inputTopicId = opts.topicId()
+                .map(Uuid::fromString).filter(uuid -> uuid != Uuid.ZERO_UUID);
+            Boolean useTopicId = inputTopicId.isPresent();
+
+            List<Uuid> topicIds;
+            List<String> topics;
+            if (useTopicId) {
+                topicIds = getTopicIds(inputTopicId.get(), opts.excludeInternalTopics());
+                topics = Collections.emptyList();
+            } else {
+                topicIds = Collections.emptyList();
+                topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+
+            }
+
+            // Only check topic name when topicId is not provided
+            if (useTopicId) {
+                ensureTopicIdExists(topicIds, inputTopicId.get(), !opts.ifExists());
+            } else {
+                ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            }
+            List<org.apache.kafka.clients.admin.TopicDescription> topicDescriptions = new ArrayList<>();
+
+            if (!topicIds.isEmpty()) {
+                Map<Uuid, org.apache.kafka.clients.admin.TopicDescription> descTopics =
+                    adminClient.describeTopics(TopicCollection.ofTopicIds(topicIds)).allTopicIds().get();
+                topicDescriptions = new ArrayList<>(descTopics.values());
+            }
+
+            if (!topics.isEmpty()) {
+                Map<String, org.apache.kafka.clients.admin.TopicDescription> descTopics =
+                    adminClient.describeTopics(TopicCollection.ofTopicNames(topics)).allTopicNames().get();
+                topicDescriptions = new ArrayList<>(descTopics.values());
+            }
+
+            List<String> topicNames = topicDescriptions.stream()
+                .map(org.apache.kafka.clients.admin.TopicDescription::name)
+                .collect(Collectors.toList());
+            Map<ConfigResource, KafkaFuture<Config>> allConfigs = adminClient.describeConfigs(
+                topicNames.stream()
+                    .map(name -> new ConfigResource(ConfigResource.Type.TOPIC, name))
+                    .collect(Collectors.toList())
+            ).values();
+            List<Integer> liveBrokers = adminClient.describeCluster().nodes().get().stream()
+                .map(Node::id)
+                .collect(Collectors.toList());
+            DescribeOptions describeOptions = new DescribeOptions(opts, new HashSet<>(liveBrokers));
+            Set<TopicPartition> topicPartitions = topicDescriptions
+                .stream()
+                .flatMap(td -> td.partitions().stream()
+                    .map(p -> new TopicPartition(td.name(), p.partition())))
+                .collect(Collectors.toSet());
+            Map<TopicPartition, PartitionReassignment> reassignments = listAllReassignments(topicPartitions);
+            for (org.apache.kafka.clients.admin.TopicDescription td : topicDescriptions) {
+                String topicName = td.name();
+                Uuid topicId = td.topicId();
+                Config config = allConfigs.get(new ConfigResource(ConfigResource.Type.TOPIC, topicName)).get();
+                ArrayList<TopicPartitionInfo> sortedPartitions = new ArrayList<>(td.partitions());
+                sortedPartitions.sort(Comparator.comparingInt(TopicPartitionInfo::partition));
+                printDescribeConfig(opts, describeOptions, reassignments, td, topicName, topicId, config, sortedPartitions);
+                printPartitionDescription(describeOptions, reassignments, td, topicName, config, sortedPartitions);
+            }
+        }
+
+        private void printPartitionDescription(DescribeOptions describeOptions, Map<TopicPartition, PartitionReassignment> reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Config config, ArrayList<TopicPartitionInfo> sortedPartitions) {
+            if (describeOptions.describePartitions) {
+                for (TopicPartitionInfo partition : sortedPartitions) {
+                    PartitionReassignment reassignment =
+                        reassignments.get(new TopicPartition(td.name(), partition.partition()));
+                    PartitionDescription partitionDesc = new PartitionDescription(topicName,
+                        partition, config, false, reassignment);
+                    describeOptions.maybePrintPartitionDescription(partitionDesc);
+                }
+            }
+        }
+
+        private void printDescribeConfig(TopicCommandOptions opts, DescribeOptions describeOptions, Map<TopicPartition, PartitionReassignment> reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Uuid topicId, Config config, ArrayList<TopicPartitionInfo> sortedPartitions) {
+            if (describeOptions.describeConfigs) {
+                List<ConfigEntry> entries = new ArrayList<>(config.entries());
+                boolean hasNonDefault = entries.stream().anyMatch(e -> !e.isDefault());
+                if (!opts.reportOverriddenConfigs() || hasNonDefault) {
+                    int numPartitions = td.partitions().size();
+                    TopicPartitionInfo firstPartition = sortedPartitions.get(0);
+                    PartitionReassignment reassignment =
+                        reassignments.get(new TopicPartition(td.name(), firstPartition.partition()));
+                    TopicDescription topicDesc = new TopicDescription(topicName, topicId,
+                        numPartitions, getReplicationFactor(firstPartition, reassignment),
+                        config, false);
+                    topicDesc.printDescription();
+                }
+            }
+        }
+
+        public void deleteTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            adminClient.deleteTopics(Collections.unmodifiableList(topics),
+                new DeleteTopicsOptions().retryOnQuotaViolation(false)
+            ).all().get();
+        }
+
+        public List<String> getTopics(Optional<String> topicIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
+            ListTopicsOptions listTopicsOptions = new ListTopicsOptions();
+            if (!excludeInternalTopics) {
+                listTopicsOptions.listInternal(true);
+            }
+
+            Set<String> allTopics = adminClient.listTopics(listTopicsOptions).names().get();
+            return doGetTopics(allTopics.stream().sorted().collect(Collectors.toList()), topicIncludeList, excludeInternalTopics);
+        }
+
+        public List<Uuid> getTopicIds(Uuid topicIdIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
+            ListTopicsResult allTopics = excludeInternalTopics ? adminClient.listTopics() :
+                adminClient.listTopics(new ListTopicsOptions().listInternal(true));
+            List<Uuid> allTopicIds = null;
+            allTopicIds = allTopics.listings().get().stream()
+                .map(TopicListing::topicId)
+                .sorted()
+                .collect(Collectors.toList());
+            return allTopicIds.contains(topicIdIncludeList) ?
+                Collections.singletonList(topicIdIncludeList) :
+                Collections.emptyList();
+        }
+
+        @Override
+        public void close() throws Exception {
+            adminClient.close();
+        }
+    }
+
+    public static class TopicCommandOptions extends CommandDefaultOptions {
+        private final ArgumentAcceptingOptionSpec<String> bootstrapServerOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> commandConfigOpt;
+
+        private final OptionSpecBuilder listOpt;
+
+        private final OptionSpecBuilder createOpt;
+
+        private final OptionSpecBuilder deleteOpt;
+
+        private final OptionSpecBuilder alterOpt;
+
+        private final OptionSpecBuilder describeOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> topicOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> topicIdOpt;
+
+        private final String nl;
+
+        private static final String KAFKA_CONFIGS_CLI_SUPPORTS_ALTERING_TOPIC_CONFIGS_WITH_A_BOOTSTRAP_SERVER =
+                " (the kafka-configs CLI supports altering topic configs with a --bootstrap-server option)";
+
+        private final ArgumentAcceptingOptionSpec<String> configOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> deleteConfigOpt;
+
+        private final ArgumentAcceptingOptionSpec<Integer> partitionsOpt;
+
+        private final ArgumentAcceptingOptionSpec<Integer> replicationFactorOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> replicaAssignmentOpt;
+
+        private final OptionSpecBuilder reportUnderReplicatedPartitionsOpt;
+
+        private final OptionSpecBuilder reportUnavailablePartitionsOpt;
+
+        private final OptionSpecBuilder reportUnderMinIsrPartitionsOpt;
+
+        private final OptionSpecBuilder reportAtMinIsrPartitionsOpt;
+
+        private final OptionSpecBuilder topicsWithOverridesOpt;
+
+        private final OptionSpecBuilder ifExistsOpt;
+
+        private final OptionSpecBuilder ifNotExistsOpt;
+
+        private final OptionSpecBuilder excludeInternalTopicOpt;
+
+        private final Set<OptionSpec<?>> allTopicLevelOpts;
+
+        private final Set<OptionSpecBuilder> allReplicationReportOpts;
+
+        public TopicCommandOptions(String[] args) {
+            super(args);
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The Kafka server to connect to.")
+                .withRequiredArg()
+                .describedAs("server to connect to")
+                .ofType(String.class);
+            commandConfigOpt = parser.accepts("command-config", "Property file containing configs to be passed to Admin Client. " +
+                    "This is used only with --bootstrap-server option for describing and altering broker configs.")
+                .withRequiredArg()
+                .describedAs("command config property file")
+                .ofType(String.class);
+
+
+            listOpt = parser.accepts("list", "List all available topics.");
+            createOpt = parser.accepts("create", "Create a new topic.");
+            deleteOpt = parser.accepts("delete", "Delete a topic");
+            alterOpt = parser.accepts("alter", "Alter the number of partitions, replica assignment, and/or configuration for the topic.");
+            describeOpt = parser.accepts("describe", "List details for the given topics.");
+            topicOpt = parser.accepts("topic", "Alter the number of partitions and replica assignment. " +
+                            "Update the configuration of an existing topic via --alter is no longer supported here" +
+                            KAFKA_CONFIGS_CLI_SUPPORTS_ALTERING_TOPIC_CONFIGS_WITH_A_BOOTSTRAP_SERVER + ".")
+                .withRequiredArg()
+                .describedAs("topic")
+                .ofType(String.class);
+            topicIdOpt = parser.accepts("topic-id", "The topic-id to describe." +
+                    "This is used only with --bootstrap-server option for describing topics.")
+                .withRequiredArg()
+                .describedAs("topic-id")
+                .ofType(String.class);
+            nl = System.getProperty("line.separator");
+
+            String configNames = LogConfig.configNames().stream()
+                .map(name -> "\t" + name)
+                .collect(Collectors.joining("nl"));

Review Comment:
   the original scala code was like this 
   ```
   
       private val nl = System.getProperty("line.separator")
       private val configOpt = parser.accepts("config", "A topic configuration override for the topic being created or altered." +
                                                " The following is a list of valid configurations: " + nl + LogConfig.configNames.asScala.map("\t" + _).mkString(nl) + nl +
                                                "See the Kafka documentation for full details on the topic configs." +
                                                " It is supported only in combination with --create if --bootstrap-server option is used" +
                                                kafkaConfigsCanAlterTopicConfigsViaBootstrapServer + ".")
    ```
    I removed this double quote as it was done by mistake. Now the code refer to `nl` variable which is `nl  = System.getProperty("line.separator");`



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1283070227


##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String[] args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {
+            System.err.println(e.getMessage());
+            return 1;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        opts.checkArgs();

Review Comment:
   We could move this one inside the TopicCommandOptions' constructor.



##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String[] args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {
+            System.err.println(e.getMessage());
+            return 1;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        opts.checkArgs();
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        log.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")
+    static Map<Integer, List<Integer>> parseReplicaAssignment(String replicaAssignmentList) {
+        String[] partitionList = replicaAssignmentList.split(",");
+        Map<Integer, List<Integer>> ret = new LinkedHashMap<Integer, List<Integer>>();
+        for (int i = 0; i < partitionList.length; i++) {
+            List<Integer> brokerList = Arrays.stream(partitionList[i].split(":"))
+                .map(String::trim)
+                .mapToInt(Integer::parseInt)
+                .boxed()
+                .collect(Collectors.toList());
+            Collection<Integer> duplicateBrokers = ToolsUtils.duplicates(brokerList);
+            if (!duplicateBrokers.isEmpty()) {
+                throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: " +
+                    duplicateBrokers.stream()
+                        .map(Object::toString)
+                        .collect(Collectors.joining(","))
+                );
+            }
+            ret.put(i, brokerList);
+            if (ret.get(i).size() != ret.get(0).size()) {
+                throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList);
+            }
+        }
+        return ret;
+    }
+
+    @SuppressWarnings("deprecation")
+    private static Properties parseTopicConfigsToBeAdded(TopicCommandOptions opts) {
+        List<List<String>> configsToBeAdded = opts.topicConfig().orElse(Collections.emptyList())
+            .stream()
+            .map(s -> Arrays.asList(s.split("\\s*=\\s*")))
+            .collect(Collectors.toList());
+
+        if (!configsToBeAdded.stream().allMatch(config -> config.size() == 2)) {
+            throw new IllegalArgumentException("requirement failed: Invalid topic config: all configs to be added must be in the format \"key=val\".");
+        }
+
+        Properties props = new Properties();
+        configsToBeAdded.stream()
+            .forEach(pair -> props.setProperty(pair.get(0).trim(), pair.get(1).trim()));
+        LogConfig.validate(props);
+        if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) {
+            System.out.println("WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " +
+                "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " +
+                "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.");
+        }
+        return props;
+    }
+
+    // It is possible for a reassignment to complete between the time we have fetched its state and the time
+    // we fetch partition metadata. In this case, we ignore the reassignment when determining replication factor.
+    public static boolean isReassignmentInProgress(TopicPartitionInfo tpi, PartitionReassignment ra) {
+        // Reassignment is still in progress as long as the removing and adding replicas are still present
+        Set<Integer> allReplicaIds = tpi.replicas().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> changingReplicaIds = new HashSet<>();
+        if (ra != null) {
+            changingReplicaIds.addAll(ra.removingReplicas());
+            changingReplicaIds.addAll(ra.addingReplicas());
+        }
+        return allReplicaIds.stream().anyMatch(changingReplicaIds::contains);
+
+    }
+
+    private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionReassignment reassignment) {
+        return isReassignmentInProgress(tpi, reassignment) ?
+            reassignment.replicas().size() - reassignment.addingReplicas().size() :
+            tpi.replicas().size();
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopics        Topics that were found to match the requested topic name.
+     * @param requestedTopic     Name of the topic that was requested.
+     * @param requireTopicExists Indicates if the topic needs to exist for the operation to be successful.
+     *                           If set to true, the command will throw an exception if the topic with the
+     *                           requested name does not exist.
+     */
+    private static void ensureTopicExists(List<String> foundTopics, String requestedTopic, Boolean requireTopicExists) {
+        // If no topic name was mentioned, do not need to throw exception.
+        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
+            // If given topic doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
+        }
+    }
+
+    private static List<String> doGetTopics(List<String> allTopics, Optional<String> topicIncludeList, Boolean excludeInternalTopics) {
+        if (topicIncludeList.isPresent()) {
+            IncludeList topicsFilter = new IncludeList(topicIncludeList.get());
+            return allTopics.stream()
+                .filter(topic -> topicsFilter.isTopicAllowed(topic, excludeInternalTopics))
+                .collect(Collectors.toList());
+        } else {
+            return allTopics.stream()
+                .filter(topic -> !(Topic.isInternal(topic) && excludeInternalTopics))
+                .collect(Collectors.toList());
+        }
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopicIds        Topics that were found to match the requested topic id.
+     * @param requestedTopicId     Id of the topic that was requested.
+     * @param requireTopicIdExists Indicates if the topic needs to exist for the operation to be successful.
+     *                             If set to true, the command will throw an exception if the topic with the
+     *                             requested id does not exist.
+     */
+    private static void ensureTopicIdExists(List<Uuid> foundTopicIds, Uuid requestedTopicId, Boolean requireTopicIdExists) {
+        // If no topic id was mentioned, do not need to throw exception.
+        if (requestedTopicId != null && requireTopicIdExists && foundTopicIds.isEmpty()) {
+            // If given topicId doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("TopicId '%s' does not exist as expected", requestedTopicId));
+        }
+    }
+
+    static class CommandTopicPartition {
+        private final TopicCommandOptions opts;
+        private final Optional<String> name;
+        private final Optional<Integer> replicationFactor;
+        private final Optional<Integer> partitions;
+        private final Map<Integer, List<Integer>> replicaAssignment;
+        private final Properties configsToAdd;
+
+        public CommandTopicPartition(TopicCommandOptions options) {
+            opts = options;
+            name = options.topic();
+            partitions = options.partitions();
+            replicationFactor = options.replicationFactor();
+            replicaAssignment = options.replicaAssignment().orElse(Collections.emptyMap());
+            configsToAdd = parseTopicConfigsToBeAdded(options);
+        }
+
+        public Boolean hasReplicaAssignment() {
+            return !replicaAssignment.isEmpty();
+        }
+
+        public Boolean hasPartitions() {
+            return partitions.isPresent();
+        }
+
+        public Boolean ifTopicDoesntExist() {
+            return opts.ifNotExists();
+        }
+    }
+
+    static class TopicDescription {
+        private final String topic;
+        private final Uuid topicId;
+        private final Integer numPartitions;
+        private final Integer replicationFactor;
+        private final Config config;
+        private final Boolean markedForDeletion;
+
+        public TopicDescription(String topic, Uuid topicId, Integer numPartitions, Integer replicationFactor, Config config, Boolean markedForDeletion) {
+            this.topic = topic;
+            this.topicId = topicId;
+            this.numPartitions = numPartitions;
+            this.replicationFactor = replicationFactor;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+        }
+
+        public void printDescription() {
+            String configsAsString = config.entries().stream()
+                .filter(config -> !config.isDefault())
+                .map(ce -> String.format("%s=%s", ce.name(), ce.value()))
+                .collect(Collectors.joining(","));
+            System.out.print(String.format("Topic: %s", topic));
+            if (topicId != Uuid.ZERO_UUID)
+                System.out.print(String.format("\tTopicId: %s", topicId));
+            System.out.print(String.format("\tPartitionCount: %s", numPartitions));
+            System.out.print(String.format("\tReplicationFactor: %s", replicationFactor));
+            System.out.print(String.format("\tConfigs: %s", configsAsString));
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class PartitionDescription {
+        private final String topic;
+        private final TopicPartitionInfo info;
+        private final Config config;
+        private final Boolean markedForDeletion;
+        private final PartitionReassignment reassignment;
+
+        PartitionDescription(String topic,
+                             TopicPartitionInfo info,
+                             Config config,
+                             Boolean markedForDeletion,
+                             PartitionReassignment reassignment) {
+            this.topic = topic;
+            this.info = info;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+            this.reassignment = reassignment;
+        }
+
+        public Integer minIsrCount() {
+            return Integer.parseInt(config.get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value());
+        }
+
+        public Boolean isUnderReplicated() {
+            return getReplicationFactor(info, reassignment) - info.isr().size() > 0;
+        }
+
+        public boolean hasLeader() {
+            return info.leader() != null;
+        }
+
+        public Boolean isUnderMinIsr() {
+            return !hasLeader() ||  info.isr().size() < minIsrCount();
+        }
+
+        public Boolean isAtMinIsrPartitions() {
+            return minIsrCount() == info.isr().size();
+
+        }
+
+        public Boolean hasUnavailablePartitions(Set<Integer> liveBrokers) {
+            return !hasLeader() || !liveBrokers.contains(info.leader().id());
+        }
+
+        public void printDescription() {
+            System.out.print("\tTopic: " + topic);
+            System.out.print("\tPartition: " + info.partition());
+            System.out.print("\tLeader: " + (hasLeader() ? info.leader().id() : "none"));
+            System.out.print("\tReplicas: " + info.replicas().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            System.out.print("\tIsr: " + info.isr().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            if (reassignment != null) {
+                System.out.print("\tAdding Replicas: " + reassignment.addingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+                System.out.print("\tRemoving Replicas: " + reassignment.removingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+            }
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+
+    }
+
+    static class DescribeOptions {
+        private final TopicCommandOptions opts;
+        private final Set<Integer> liveBrokers;
+        private final boolean describeConfigs;
+        private final boolean describePartitions;
+
+        public DescribeOptions(TopicCommandOptions opts, Set<Integer> liveBrokers) {
+            this.opts = opts;
+            this.liveBrokers = liveBrokers;
+            this.describeConfigs = !opts.reportUnavailablePartitions() &&
+                !opts.reportUnderReplicatedPartitions() &&
+                !opts.reportUnderMinIsrPartitions() &&
+                !opts.reportAtMinIsrPartitions();
+            this.describePartitions = !opts.reportOverriddenConfigs();
+        }
+
+        private boolean shouldPrintUnderReplicatedPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderReplicatedPartitions() && partitionDescription.isUnderReplicated();
+        }
+
+        private boolean shouldPrintUnavailablePartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnavailablePartitions() && partitionDescription.hasUnavailablePartitions(liveBrokers);
+        }
+
+        private boolean shouldPrintUnderMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderMinIsrPartitions() && partitionDescription.isUnderMinIsr();
+        }
+
+        private boolean shouldPrintAtMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportAtMinIsrPartitions() && partitionDescription.isAtMinIsrPartitions();
+        }
+
+        private boolean shouldPrintTopicPartition(PartitionDescription partitionDesc) {
+            return describeConfigs ||
+                shouldPrintUnderReplicatedPartitions(partitionDesc) ||
+                shouldPrintUnavailablePartitions(partitionDesc) ||
+                shouldPrintUnderMinIsrPartitions(partitionDesc) ||
+                shouldPrintAtMinIsrPartitions(partitionDesc);
+        }
+
+        public void maybePrintPartitionDescription(PartitionDescription desc) {
+            if (shouldPrintTopicPartition(desc)) {
+                desc.printDescription();
+            }
+        }
+    }
+
+    public static class TopicService implements AutoCloseable {
+        private Admin adminClient;
+
+        public TopicService(Properties commandConfig, Optional<String> bootstrapServer) {
+            this.adminClient = createAdminClient(commandConfig, bootstrapServer);
+        }
+
+        public TopicService(Admin admin) {
+            this.adminClient = admin;
+        }
+
+        private static Admin createAdminClient(Properties commandConfig, Optional<String> bootstrapServer) {
+            if (bootstrapServer.isPresent()) {
+                commandConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+            }
+            return Admin.create(commandConfig);
+        }
+
+        public void createTopic(TopicCommandOptions opts) throws Exception {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            if (Topic.hasCollisionChars(topic.name.get())) {
+                System.out.println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could " +
+                    "collide. To avoid issues it is best to use either, but not both.");
+            }
+            createTopic(topic);
+        }
+
+        public void createTopic(CommandTopicPartition topic) throws Exception {
+            if (topic.replicationFactor.filter(rf -> rf > Short.MAX_VALUE || rf < 1).isPresent()) {
+                throw new IllegalArgumentException("The replication factor must be between 1 and " + Short.MAX_VALUE + " inclusive");
+            }
+            if (topic.partitions.filter(p -> p < 1).isPresent()) {
+                throw new IllegalArgumentException("The partitions must be greater than 0");
+            }
+
+            try {
+                NewTopic newTopic;
+                if (topic.hasReplicaAssignment()) {
+                    newTopic = new NewTopic(topic.name.get(), topic.replicaAssignment);
+                } else {
+                    newTopic = new NewTopic(topic.name.get(), topic.partitions, topic.replicationFactor.map(Integer::shortValue));
+                }
+
+                Map<String, String> configsMap = topic.configsToAdd.stringPropertyNames().stream()
+                    .collect(Collectors.toMap(name -> name, name -> topic.configsToAdd.getProperty(name)));
+
+                newTopic.configs(configsMap);
+                CreateTopicsResult createResult = adminClient.createTopics(Collections.singleton(newTopic),
+                    new CreateTopicsOptions().retryOnQuotaViolation(false));
+                createResult.all().get();
+                System.out.println("Created topic " + topic.name + ".");
+            } catch (ExecutionException e) {
+                if (e.getCause() == null) {
+                    throw e;
+                }
+                if (!(e.getCause() instanceof TopicExistsException && topic.ifTopicDoesntExist())) {
+                    throw (Exception) e.getCause();
+                }
+            }
+        }
+
+        public void listTopics(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            String results = getTopics(opts.topic(), opts.excludeInternalTopics())
+                .stream()
+                .collect(Collectors.joining("\n"));
+            System.out.println(results);
+        }
+
+        public void alterTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+
+            if (!topics.isEmpty()) {
+                Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo = adminClient.describeTopics(topics).topicNameValues();
+                Map<String, NewPartitions> newPartitions = topics.stream()
+                    .map(topicName -> topicNewPartitions(topic, topicsInfo, topicName))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                adminClient.createPartitions(newPartitions, new CreatePartitionsOptions().retryOnQuotaViolation(false)).all().get();
+            }
+        }
+
+        private AbstractMap.SimpleEntry<String, NewPartitions> topicNewPartitions(
+            CommandTopicPartition topic,
+            Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo,
+            String topicName) {
+            if (topic.hasReplicaAssignment()) {
+                try {
+                    Integer startPartitionId = topicsInfo.get(topicName).get().partitions().size();
+                    Map<Integer, List<Integer>> replicaMap = topic.replicaAssignment.entrySet().stream()
+                        .skip(startPartitionId)
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                    List<List<Integer>> newAssignment = replicaMap.entrySet().stream()
+                        .map(Map.Entry::getValue)
+                        .collect(Collectors.toList());
+                    return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get(), newAssignment));
+                } catch (InterruptedException | ExecutionException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+            return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get()));
+        }
+
+        public Map<TopicPartition, PartitionReassignment> listAllReassignments(Set<TopicPartition> topicPartitions) {
+            try {
+                return adminClient.listPartitionReassignments(topicPartitions).reassignments().get();
+            } catch (ExecutionException e) {
+                Throwable cause = e.getCause();
+                if (cause instanceof UnsupportedVersionException || cause instanceof ClusterAuthorizationException) {
+                    log.debug("Couldn't query reassignments through the AdminClient API: " + cause.getMessage(), cause);
+                    return Collections.emptyMap();
+                } else {
+                    throw new RuntimeException(e);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        public void describeTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            // If topicId is provided and not zero, will use topicId regardless of topic name
+            Optional<Uuid> inputTopicId = opts.topicId()
+                .map(Uuid::fromString).filter(uuid -> uuid != Uuid.ZERO_UUID);
+            Boolean useTopicId = inputTopicId.isPresent();
+
+            List<Uuid> topicIds;
+            List<String> topics;
+            if (useTopicId) {
+                topicIds = getTopicIds(inputTopicId.get(), opts.excludeInternalTopics());
+                topics = Collections.emptyList();
+            } else {
+                topicIds = Collections.emptyList();
+                topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+
+            }
+
+            // Only check topic name when topicId is not provided
+            if (useTopicId) {
+                ensureTopicIdExists(topicIds, inputTopicId.get(), !opts.ifExists());
+            } else {
+                ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            }
+            List<org.apache.kafka.clients.admin.TopicDescription> topicDescriptions = new ArrayList<>();
+
+            if (!topicIds.isEmpty()) {
+                Map<Uuid, org.apache.kafka.clients.admin.TopicDescription> descTopics =
+                    adminClient.describeTopics(TopicCollection.ofTopicIds(topicIds)).allTopicIds().get();
+                topicDescriptions = new ArrayList<>(descTopics.values());
+            }
+
+            if (!topics.isEmpty()) {
+                Map<String, org.apache.kafka.clients.admin.TopicDescription> descTopics =
+                    adminClient.describeTopics(TopicCollection.ofTopicNames(topics)).allTopicNames().get();
+                topicDescriptions = new ArrayList<>(descTopics.values());
+            }
+
+            List<String> topicNames = topicDescriptions.stream()
+                .map(org.apache.kafka.clients.admin.TopicDescription::name)
+                .collect(Collectors.toList());
+            Map<ConfigResource, KafkaFuture<Config>> allConfigs = adminClient.describeConfigs(
+                topicNames.stream()
+                    .map(name -> new ConfigResource(ConfigResource.Type.TOPIC, name))
+                    .collect(Collectors.toList())
+            ).values();
+            List<Integer> liveBrokers = adminClient.describeCluster().nodes().get().stream()
+                .map(Node::id)
+                .collect(Collectors.toList());
+            DescribeOptions describeOptions = new DescribeOptions(opts, new HashSet<>(liveBrokers));
+            Set<TopicPartition> topicPartitions = topicDescriptions
+                .stream()
+                .flatMap(td -> td.partitions().stream()
+                    .map(p -> new TopicPartition(td.name(), p.partition())))
+                .collect(Collectors.toSet());
+            Map<TopicPartition, PartitionReassignment> reassignments = listAllReassignments(topicPartitions);
+            for (org.apache.kafka.clients.admin.TopicDescription td : topicDescriptions) {
+                String topicName = td.name();
+                Uuid topicId = td.topicId();
+                Config config = allConfigs.get(new ConfigResource(ConfigResource.Type.TOPIC, topicName)).get();
+                ArrayList<TopicPartitionInfo> sortedPartitions = new ArrayList<>(td.partitions());
+                sortedPartitions.sort(Comparator.comparingInt(TopicPartitionInfo::partition));
+                printDescribeConfig(opts, describeOptions, reassignments, td, topicName, topicId, config, sortedPartitions);
+                printPartitionDescription(describeOptions, reassignments, td, topicName, config, sortedPartitions);
+            }
+        }
+
+        private void printPartitionDescription(DescribeOptions describeOptions, Map<TopicPartition, PartitionReassignment> reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Config config, ArrayList<TopicPartitionInfo> sortedPartitions) {
+            if (describeOptions.describePartitions) {
+                for (TopicPartitionInfo partition : sortedPartitions) {
+                    PartitionReassignment reassignment =
+                        reassignments.get(new TopicPartition(td.name(), partition.partition()));
+                    PartitionDescription partitionDesc = new PartitionDescription(topicName,
+                        partition, config, false, reassignment);
+                    describeOptions.maybePrintPartitionDescription(partitionDesc);
+                }
+            }
+        }
+
+        private void printDescribeConfig(TopicCommandOptions opts, DescribeOptions describeOptions, Map<TopicPartition, PartitionReassignment> reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Uuid topicId, Config config, ArrayList<TopicPartitionInfo> sortedPartitions) {
+            if (describeOptions.describeConfigs) {
+                List<ConfigEntry> entries = new ArrayList<>(config.entries());
+                boolean hasNonDefault = entries.stream().anyMatch(e -> !e.isDefault());
+                if (!opts.reportOverriddenConfigs() || hasNonDefault) {
+                    int numPartitions = td.partitions().size();
+                    TopicPartitionInfo firstPartition = sortedPartitions.get(0);
+                    PartitionReassignment reassignment =
+                        reassignments.get(new TopicPartition(td.name(), firstPartition.partition()));
+                    TopicDescription topicDesc = new TopicDescription(topicName, topicId,
+                        numPartitions, getReplicationFactor(firstPartition, reassignment),
+                        config, false);
+                    topicDesc.printDescription();
+                }
+            }
+        }
+
+        public void deleteTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            adminClient.deleteTopics(Collections.unmodifiableList(topics),
+                new DeleteTopicsOptions().retryOnQuotaViolation(false)
+            ).all().get();
+        }
+
+        public List<String> getTopics(Optional<String> topicIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
+            ListTopicsOptions listTopicsOptions = new ListTopicsOptions();
+            if (!excludeInternalTopics) {
+                listTopicsOptions.listInternal(true);
+            }
+
+            Set<String> allTopics = adminClient.listTopics(listTopicsOptions).names().get();
+            return doGetTopics(allTopics.stream().sorted().collect(Collectors.toList()), topicIncludeList, excludeInternalTopics);
+        }
+
+        public List<Uuid> getTopicIds(Uuid topicIdIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
+            ListTopicsResult allTopics = excludeInternalTopics ? adminClient.listTopics() :
+                adminClient.listTopics(new ListTopicsOptions().listInternal(true));
+            List<Uuid> allTopicIds = null;
+            allTopicIds = allTopics.listings().get().stream()
+                .map(TopicListing::topicId)
+                .sorted()
+                .collect(Collectors.toList());
+            return allTopicIds.contains(topicIdIncludeList) ?
+                Collections.singletonList(topicIdIncludeList) :
+                Collections.emptyList();
+        }
+
+        @Override
+        public void close() throws Exception {
+            adminClient.close();
+        }
+    }
+
+    public static class TopicCommandOptions extends CommandDefaultOptions {
+        private final ArgumentAcceptingOptionSpec<String> bootstrapServerOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> commandConfigOpt;
+
+        private final OptionSpecBuilder listOpt;
+
+        private final OptionSpecBuilder createOpt;
+
+        private final OptionSpecBuilder deleteOpt;
+
+        private final OptionSpecBuilder alterOpt;
+
+        private final OptionSpecBuilder describeOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> topicOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> topicIdOpt;
+
+        private final String nl;
+
+        private static final String KAFKA_CONFIGS_CLI_SUPPORTS_ALTERING_TOPIC_CONFIGS_WITH_A_BOOTSTRAP_SERVER =
+                " (the kafka-configs CLI supports altering topic configs with a --bootstrap-server option)";
+
+        private final ArgumentAcceptingOptionSpec<String> configOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> deleteConfigOpt;
+
+        private final ArgumentAcceptingOptionSpec<Integer> partitionsOpt;
+
+        private final ArgumentAcceptingOptionSpec<Integer> replicationFactorOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> replicaAssignmentOpt;
+
+        private final OptionSpecBuilder reportUnderReplicatedPartitionsOpt;
+
+        private final OptionSpecBuilder reportUnavailablePartitionsOpt;
+
+        private final OptionSpecBuilder reportUnderMinIsrPartitionsOpt;
+
+        private final OptionSpecBuilder reportAtMinIsrPartitionsOpt;
+
+        private final OptionSpecBuilder topicsWithOverridesOpt;
+
+        private final OptionSpecBuilder ifExistsOpt;
+
+        private final OptionSpecBuilder ifNotExistsOpt;
+
+        private final OptionSpecBuilder excludeInternalTopicOpt;
+
+        private final Set<OptionSpec<?>> allTopicLevelOpts;
+
+        private final Set<OptionSpecBuilder> allReplicationReportOpts;
+
+        public TopicCommandOptions(String[] args) {
+            super(args);
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The Kafka server to connect to.")
+                .withRequiredArg()
+                .describedAs("server to connect to")
+                .ofType(String.class);
+            commandConfigOpt = parser.accepts("command-config", "Property file containing configs to be passed to Admin Client. " +
+                    "This is used only with --bootstrap-server option for describing and altering broker configs.")
+                .withRequiredArg()
+                .describedAs("command config property file")
+                .ofType(String.class);
+
+
+            listOpt = parser.accepts("list", "List all available topics.");
+            createOpt = parser.accepts("create", "Create a new topic.");
+            deleteOpt = parser.accepts("delete", "Delete a topic");
+            alterOpt = parser.accepts("alter", "Alter the number of partitions, replica assignment, and/or configuration for the topic.");
+            describeOpt = parser.accepts("describe", "List details for the given topics.");
+            topicOpt = parser.accepts("topic", "Alter the number of partitions and replica assignment. " +
+                            "Update the configuration of an existing topic via --alter is no longer supported here" +
+                            KAFKA_CONFIGS_CLI_SUPPORTS_ALTERING_TOPIC_CONFIGS_WITH_A_BOOTSTRAP_SERVER + ".")
+                .withRequiredArg()
+                .describedAs("topic")
+                .ofType(String.class);
+            topicIdOpt = parser.accepts("topic-id", "The topic-id to describe." +
+                    "This is used only with --bootstrap-server option for describing topics.")
+                .withRequiredArg()
+                .describedAs("topic-id")
+                .ofType(String.class);
+            nl = System.getProperty("line.separator");
+
+            String configNames = LogConfig.configNames().stream()
+                .map(name -> "\t" + name)
+                .collect(Collectors.joining("nl"));
+            configOpt = parser.accepts("config", "A topic configuration override for the topic being created or altered." +

Review Comment:
   Shouldn't we remove "or altered" here, given it is not supported anymore?



##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String[] args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {
+            System.err.println(e.getMessage());
+            return 1;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        opts.checkArgs();
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        log.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")
+    static Map<Integer, List<Integer>> parseReplicaAssignment(String replicaAssignmentList) {
+        String[] partitionList = replicaAssignmentList.split(",");
+        Map<Integer, List<Integer>> ret = new LinkedHashMap<Integer, List<Integer>>();
+        for (int i = 0; i < partitionList.length; i++) {
+            List<Integer> brokerList = Arrays.stream(partitionList[i].split(":"))
+                .map(String::trim)
+                .mapToInt(Integer::parseInt)
+                .boxed()
+                .collect(Collectors.toList());
+            Collection<Integer> duplicateBrokers = ToolsUtils.duplicates(brokerList);
+            if (!duplicateBrokers.isEmpty()) {
+                throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: " +
+                    duplicateBrokers.stream()
+                        .map(Object::toString)
+                        .collect(Collectors.joining(","))
+                );
+            }
+            ret.put(i, brokerList);
+            if (ret.get(i).size() != ret.get(0).size()) {
+                throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList);
+            }
+        }
+        return ret;
+    }
+
+    @SuppressWarnings("deprecation")
+    private static Properties parseTopicConfigsToBeAdded(TopicCommandOptions opts) {
+        List<List<String>> configsToBeAdded = opts.topicConfig().orElse(Collections.emptyList())
+            .stream()
+            .map(s -> Arrays.asList(s.split("\\s*=\\s*")))
+            .collect(Collectors.toList());
+
+        if (!configsToBeAdded.stream().allMatch(config -> config.size() == 2)) {
+            throw new IllegalArgumentException("requirement failed: Invalid topic config: all configs to be added must be in the format \"key=val\".");
+        }
+
+        Properties props = new Properties();
+        configsToBeAdded.stream()
+            .forEach(pair -> props.setProperty(pair.get(0).trim(), pair.get(1).trim()));
+        LogConfig.validate(props);
+        if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) {
+            System.out.println("WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " +
+                "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " +
+                "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.");
+        }
+        return props;
+    }
+
+    // It is possible for a reassignment to complete between the time we have fetched its state and the time
+    // we fetch partition metadata. In this case, we ignore the reassignment when determining replication factor.
+    public static boolean isReassignmentInProgress(TopicPartitionInfo tpi, PartitionReassignment ra) {
+        // Reassignment is still in progress as long as the removing and adding replicas are still present
+        Set<Integer> allReplicaIds = tpi.replicas().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> changingReplicaIds = new HashSet<>();
+        if (ra != null) {
+            changingReplicaIds.addAll(ra.removingReplicas());
+            changingReplicaIds.addAll(ra.addingReplicas());
+        }
+        return allReplicaIds.stream().anyMatch(changingReplicaIds::contains);
+
+    }
+
+    private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionReassignment reassignment) {
+        return isReassignmentInProgress(tpi, reassignment) ?
+            reassignment.replicas().size() - reassignment.addingReplicas().size() :
+            tpi.replicas().size();
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopics        Topics that were found to match the requested topic name.
+     * @param requestedTopic     Name of the topic that was requested.
+     * @param requireTopicExists Indicates if the topic needs to exist for the operation to be successful.
+     *                           If set to true, the command will throw an exception if the topic with the
+     *                           requested name does not exist.
+     */
+    private static void ensureTopicExists(List<String> foundTopics, String requestedTopic, Boolean requireTopicExists) {
+        // If no topic name was mentioned, do not need to throw exception.
+        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
+            // If given topic doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
+        }
+    }
+
+    private static List<String> doGetTopics(List<String> allTopics, Optional<String> topicIncludeList, Boolean excludeInternalTopics) {
+        if (topicIncludeList.isPresent()) {
+            IncludeList topicsFilter = new IncludeList(topicIncludeList.get());
+            return allTopics.stream()
+                .filter(topic -> topicsFilter.isTopicAllowed(topic, excludeInternalTopics))
+                .collect(Collectors.toList());
+        } else {
+            return allTopics.stream()
+                .filter(topic -> !(Topic.isInternal(topic) && excludeInternalTopics))
+                .collect(Collectors.toList());
+        }
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopicIds        Topics that were found to match the requested topic id.
+     * @param requestedTopicId     Id of the topic that was requested.
+     * @param requireTopicIdExists Indicates if the topic needs to exist for the operation to be successful.
+     *                             If set to true, the command will throw an exception if the topic with the
+     *                             requested id does not exist.
+     */
+    private static void ensureTopicIdExists(List<Uuid> foundTopicIds, Uuid requestedTopicId, Boolean requireTopicIdExists) {
+        // If no topic id was mentioned, do not need to throw exception.
+        if (requestedTopicId != null && requireTopicIdExists && foundTopicIds.isEmpty()) {
+            // If given topicId doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("TopicId '%s' does not exist as expected", requestedTopicId));
+        }
+    }
+
+    static class CommandTopicPartition {
+        private final TopicCommandOptions opts;
+        private final Optional<String> name;
+        private final Optional<Integer> replicationFactor;
+        private final Optional<Integer> partitions;
+        private final Map<Integer, List<Integer>> replicaAssignment;
+        private final Properties configsToAdd;
+
+        public CommandTopicPartition(TopicCommandOptions options) {
+            opts = options;
+            name = options.topic();
+            partitions = options.partitions();
+            replicationFactor = options.replicationFactor();
+            replicaAssignment = options.replicaAssignment().orElse(Collections.emptyMap());
+            configsToAdd = parseTopicConfigsToBeAdded(options);
+        }
+
+        public Boolean hasReplicaAssignment() {
+            return !replicaAssignment.isEmpty();
+        }
+
+        public Boolean hasPartitions() {
+            return partitions.isPresent();
+        }
+
+        public Boolean ifTopicDoesntExist() {
+            return opts.ifNotExists();
+        }
+    }
+
+    static class TopicDescription {
+        private final String topic;
+        private final Uuid topicId;
+        private final Integer numPartitions;
+        private final Integer replicationFactor;
+        private final Config config;
+        private final Boolean markedForDeletion;
+
+        public TopicDescription(String topic, Uuid topicId, Integer numPartitions, Integer replicationFactor, Config config, Boolean markedForDeletion) {
+            this.topic = topic;
+            this.topicId = topicId;
+            this.numPartitions = numPartitions;
+            this.replicationFactor = replicationFactor;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+        }
+
+        public void printDescription() {
+            String configsAsString = config.entries().stream()
+                .filter(config -> !config.isDefault())
+                .map(ce -> String.format("%s=%s", ce.name(), ce.value()))
+                .collect(Collectors.joining(","));
+            System.out.print(String.format("Topic: %s", topic));
+            if (topicId != Uuid.ZERO_UUID)
+                System.out.print(String.format("\tTopicId: %s", topicId));
+            System.out.print(String.format("\tPartitionCount: %s", numPartitions));
+            System.out.print(String.format("\tReplicationFactor: %s", replicationFactor));
+            System.out.print(String.format("\tConfigs: %s", configsAsString));
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class PartitionDescription {
+        private final String topic;
+        private final TopicPartitionInfo info;
+        private final Config config;
+        private final Boolean markedForDeletion;
+        private final PartitionReassignment reassignment;
+
+        PartitionDescription(String topic,
+                             TopicPartitionInfo info,
+                             Config config,
+                             Boolean markedForDeletion,
+                             PartitionReassignment reassignment) {
+            this.topic = topic;
+            this.info = info;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+            this.reassignment = reassignment;
+        }
+
+        public Integer minIsrCount() {
+            return Integer.parseInt(config.get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value());
+        }
+
+        public Boolean isUnderReplicated() {
+            return getReplicationFactor(info, reassignment) - info.isr().size() > 0;
+        }
+
+        public boolean hasLeader() {
+            return info.leader() != null;
+        }
+
+        public Boolean isUnderMinIsr() {
+            return !hasLeader() ||  info.isr().size() < minIsrCount();
+        }
+
+        public Boolean isAtMinIsrPartitions() {
+            return minIsrCount() == info.isr().size();
+
+        }
+
+        public Boolean hasUnavailablePartitions(Set<Integer> liveBrokers) {
+            return !hasLeader() || !liveBrokers.contains(info.leader().id());
+        }
+
+        public void printDescription() {
+            System.out.print("\tTopic: " + topic);
+            System.out.print("\tPartition: " + info.partition());
+            System.out.print("\tLeader: " + (hasLeader() ? info.leader().id() : "none"));
+            System.out.print("\tReplicas: " + info.replicas().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            System.out.print("\tIsr: " + info.isr().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            if (reassignment != null) {
+                System.out.print("\tAdding Replicas: " + reassignment.addingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+                System.out.print("\tRemoving Replicas: " + reassignment.removingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+            }
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+
+    }
+
+    static class DescribeOptions {
+        private final TopicCommandOptions opts;
+        private final Set<Integer> liveBrokers;
+        private final boolean describeConfigs;
+        private final boolean describePartitions;
+
+        public DescribeOptions(TopicCommandOptions opts, Set<Integer> liveBrokers) {
+            this.opts = opts;
+            this.liveBrokers = liveBrokers;
+            this.describeConfigs = !opts.reportUnavailablePartitions() &&
+                !opts.reportUnderReplicatedPartitions() &&
+                !opts.reportUnderMinIsrPartitions() &&
+                !opts.reportAtMinIsrPartitions();
+            this.describePartitions = !opts.reportOverriddenConfigs();
+        }
+
+        private boolean shouldPrintUnderReplicatedPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderReplicatedPartitions() && partitionDescription.isUnderReplicated();
+        }
+
+        private boolean shouldPrintUnavailablePartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnavailablePartitions() && partitionDescription.hasUnavailablePartitions(liveBrokers);
+        }
+
+        private boolean shouldPrintUnderMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderMinIsrPartitions() && partitionDescription.isUnderMinIsr();
+        }
+
+        private boolean shouldPrintAtMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportAtMinIsrPartitions() && partitionDescription.isAtMinIsrPartitions();
+        }
+
+        private boolean shouldPrintTopicPartition(PartitionDescription partitionDesc) {
+            return describeConfigs ||
+                shouldPrintUnderReplicatedPartitions(partitionDesc) ||
+                shouldPrintUnavailablePartitions(partitionDesc) ||
+                shouldPrintUnderMinIsrPartitions(partitionDesc) ||
+                shouldPrintAtMinIsrPartitions(partitionDesc);
+        }
+
+        public void maybePrintPartitionDescription(PartitionDescription desc) {
+            if (shouldPrintTopicPartition(desc)) {
+                desc.printDescription();
+            }
+        }
+    }
+
+    public static class TopicService implements AutoCloseable {
+        private Admin adminClient;
+
+        public TopicService(Properties commandConfig, Optional<String> bootstrapServer) {
+            this.adminClient = createAdminClient(commandConfig, bootstrapServer);
+        }
+
+        public TopicService(Admin admin) {
+            this.adminClient = admin;
+        }
+
+        private static Admin createAdminClient(Properties commandConfig, Optional<String> bootstrapServer) {
+            if (bootstrapServer.isPresent()) {
+                commandConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+            }
+            return Admin.create(commandConfig);
+        }
+
+        public void createTopic(TopicCommandOptions opts) throws Exception {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            if (Topic.hasCollisionChars(topic.name.get())) {
+                System.out.println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could " +
+                    "collide. To avoid issues it is best to use either, but not both.");
+            }
+            createTopic(topic);
+        }
+
+        public void createTopic(CommandTopicPartition topic) throws Exception {
+            if (topic.replicationFactor.filter(rf -> rf > Short.MAX_VALUE || rf < 1).isPresent()) {
+                throw new IllegalArgumentException("The replication factor must be between 1 and " + Short.MAX_VALUE + " inclusive");
+            }
+            if (topic.partitions.filter(p -> p < 1).isPresent()) {
+                throw new IllegalArgumentException("The partitions must be greater than 0");
+            }
+
+            try {
+                NewTopic newTopic;
+                if (topic.hasReplicaAssignment()) {
+                    newTopic = new NewTopic(topic.name.get(), topic.replicaAssignment);
+                } else {
+                    newTopic = new NewTopic(topic.name.get(), topic.partitions, topic.replicationFactor.map(Integer::shortValue));
+                }
+
+                Map<String, String> configsMap = topic.configsToAdd.stringPropertyNames().stream()
+                    .collect(Collectors.toMap(name -> name, name -> topic.configsToAdd.getProperty(name)));
+
+                newTopic.configs(configsMap);
+                CreateTopicsResult createResult = adminClient.createTopics(Collections.singleton(newTopic),
+                    new CreateTopicsOptions().retryOnQuotaViolation(false));
+                createResult.all().get();
+                System.out.println("Created topic " + topic.name + ".");
+            } catch (ExecutionException e) {
+                if (e.getCause() == null) {
+                    throw e;
+                }
+                if (!(e.getCause() instanceof TopicExistsException && topic.ifTopicDoesntExist())) {
+                    throw (Exception) e.getCause();
+                }
+            }
+        }
+
+        public void listTopics(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            String results = getTopics(opts.topic(), opts.excludeInternalTopics())
+                .stream()
+                .collect(Collectors.joining("\n"));
+            System.out.println(results);
+        }
+
+        public void alterTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+
+            if (!topics.isEmpty()) {
+                Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo = adminClient.describeTopics(topics).topicNameValues();
+                Map<String, NewPartitions> newPartitions = topics.stream()
+                    .map(topicName -> topicNewPartitions(topic, topicsInfo, topicName))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                adminClient.createPartitions(newPartitions, new CreatePartitionsOptions().retryOnQuotaViolation(false)).all().get();
+            }
+        }
+
+        private AbstractMap.SimpleEntry<String, NewPartitions> topicNewPartitions(
+            CommandTopicPartition topic,
+            Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo,
+            String topicName) {
+            if (topic.hasReplicaAssignment()) {
+                try {
+                    Integer startPartitionId = topicsInfo.get(topicName).get().partitions().size();
+                    Map<Integer, List<Integer>> replicaMap = topic.replicaAssignment.entrySet().stream()
+                        .skip(startPartitionId)
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                    List<List<Integer>> newAssignment = replicaMap.entrySet().stream()
+                        .map(Map.Entry::getValue)
+                        .collect(Collectors.toList());
+                    return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get(), newAssignment));
+                } catch (InterruptedException | ExecutionException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+            return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get()));
+        }
+
+        public Map<TopicPartition, PartitionReassignment> listAllReassignments(Set<TopicPartition> topicPartitions) {
+            try {
+                return adminClient.listPartitionReassignments(topicPartitions).reassignments().get();
+            } catch (ExecutionException e) {
+                Throwable cause = e.getCause();
+                if (cause instanceof UnsupportedVersionException || cause instanceof ClusterAuthorizationException) {
+                    log.debug("Couldn't query reassignments through the AdminClient API: " + cause.getMessage(), cause);
+                    return Collections.emptyMap();
+                } else {
+                    throw new RuntimeException(e);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        public void describeTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            // If topicId is provided and not zero, will use topicId regardless of topic name
+            Optional<Uuid> inputTopicId = opts.topicId()
+                .map(Uuid::fromString).filter(uuid -> uuid != Uuid.ZERO_UUID);
+            Boolean useTopicId = inputTopicId.isPresent();
+
+            List<Uuid> topicIds;
+            List<String> topics;
+            if (useTopicId) {
+                topicIds = getTopicIds(inputTopicId.get(), opts.excludeInternalTopics());
+                topics = Collections.emptyList();
+            } else {
+                topicIds = Collections.emptyList();
+                topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+
+            }
+
+            // Only check topic name when topicId is not provided
+            if (useTopicId) {
+                ensureTopicIdExists(topicIds, inputTopicId.get(), !opts.ifExists());
+            } else {
+                ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            }
+            List<org.apache.kafka.clients.admin.TopicDescription> topicDescriptions = new ArrayList<>();
+
+            if (!topicIds.isEmpty()) {
+                Map<Uuid, org.apache.kafka.clients.admin.TopicDescription> descTopics =
+                    adminClient.describeTopics(TopicCollection.ofTopicIds(topicIds)).allTopicIds().get();
+                topicDescriptions = new ArrayList<>(descTopics.values());
+            }
+
+            if (!topics.isEmpty()) {
+                Map<String, org.apache.kafka.clients.admin.TopicDescription> descTopics =
+                    adminClient.describeTopics(TopicCollection.ofTopicNames(topics)).allTopicNames().get();
+                topicDescriptions = new ArrayList<>(descTopics.values());
+            }
+
+            List<String> topicNames = topicDescriptions.stream()
+                .map(org.apache.kafka.clients.admin.TopicDescription::name)
+                .collect(Collectors.toList());
+            Map<ConfigResource, KafkaFuture<Config>> allConfigs = adminClient.describeConfigs(
+                topicNames.stream()
+                    .map(name -> new ConfigResource(ConfigResource.Type.TOPIC, name))
+                    .collect(Collectors.toList())
+            ).values();
+            List<Integer> liveBrokers = adminClient.describeCluster().nodes().get().stream()
+                .map(Node::id)
+                .collect(Collectors.toList());
+            DescribeOptions describeOptions = new DescribeOptions(opts, new HashSet<>(liveBrokers));
+            Set<TopicPartition> topicPartitions = topicDescriptions
+                .stream()
+                .flatMap(td -> td.partitions().stream()
+                    .map(p -> new TopicPartition(td.name(), p.partition())))
+                .collect(Collectors.toSet());
+            Map<TopicPartition, PartitionReassignment> reassignments = listAllReassignments(topicPartitions);
+            for (org.apache.kafka.clients.admin.TopicDescription td : topicDescriptions) {
+                String topicName = td.name();
+                Uuid topicId = td.topicId();
+                Config config = allConfigs.get(new ConfigResource(ConfigResource.Type.TOPIC, topicName)).get();
+                ArrayList<TopicPartitionInfo> sortedPartitions = new ArrayList<>(td.partitions());
+                sortedPartitions.sort(Comparator.comparingInt(TopicPartitionInfo::partition));
+                printDescribeConfig(opts, describeOptions, reassignments, td, topicName, topicId, config, sortedPartitions);
+                printPartitionDescription(describeOptions, reassignments, td, topicName, config, sortedPartitions);
+            }
+        }
+
+        private void printPartitionDescription(DescribeOptions describeOptions, Map<TopicPartition, PartitionReassignment> reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Config config, ArrayList<TopicPartitionInfo> sortedPartitions) {
+            if (describeOptions.describePartitions) {
+                for (TopicPartitionInfo partition : sortedPartitions) {
+                    PartitionReassignment reassignment =
+                        reassignments.get(new TopicPartition(td.name(), partition.partition()));
+                    PartitionDescription partitionDesc = new PartitionDescription(topicName,
+                        partition, config, false, reassignment);
+                    describeOptions.maybePrintPartitionDescription(partitionDesc);
+                }
+            }
+        }
+
+        private void printDescribeConfig(TopicCommandOptions opts, DescribeOptions describeOptions, Map<TopicPartition, PartitionReassignment> reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Uuid topicId, Config config, ArrayList<TopicPartitionInfo> sortedPartitions) {
+            if (describeOptions.describeConfigs) {
+                List<ConfigEntry> entries = new ArrayList<>(config.entries());
+                boolean hasNonDefault = entries.stream().anyMatch(e -> !e.isDefault());
+                if (!opts.reportOverriddenConfigs() || hasNonDefault) {
+                    int numPartitions = td.partitions().size();
+                    TopicPartitionInfo firstPartition = sortedPartitions.get(0);
+                    PartitionReassignment reassignment =
+                        reassignments.get(new TopicPartition(td.name(), firstPartition.partition()));
+                    TopicDescription topicDesc = new TopicDescription(topicName, topicId,
+                        numPartitions, getReplicationFactor(firstPartition, reassignment),
+                        config, false);
+                    topicDesc.printDescription();
+                }
+            }
+        }
+
+        public void deleteTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            adminClient.deleteTopics(Collections.unmodifiableList(topics),
+                new DeleteTopicsOptions().retryOnQuotaViolation(false)
+            ).all().get();
+        }
+
+        public List<String> getTopics(Optional<String> topicIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
+            ListTopicsOptions listTopicsOptions = new ListTopicsOptions();
+            if (!excludeInternalTopics) {
+                listTopicsOptions.listInternal(true);
+            }
+
+            Set<String> allTopics = adminClient.listTopics(listTopicsOptions).names().get();
+            return doGetTopics(allTopics.stream().sorted().collect(Collectors.toList()), topicIncludeList, excludeInternalTopics);
+        }
+
+        public List<Uuid> getTopicIds(Uuid topicIdIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
+            ListTopicsResult allTopics = excludeInternalTopics ? adminClient.listTopics() :
+                adminClient.listTopics(new ListTopicsOptions().listInternal(true));
+            List<Uuid> allTopicIds = null;
+            allTopicIds = allTopics.listings().get().stream()
+                .map(TopicListing::topicId)
+                .sorted()
+                .collect(Collectors.toList());
+            return allTopicIds.contains(topicIdIncludeList) ?
+                Collections.singletonList(topicIdIncludeList) :
+                Collections.emptyList();
+        }
+
+        @Override
+        public void close() throws Exception {
+            adminClient.close();
+        }
+    }
+
+    public static class TopicCommandOptions extends CommandDefaultOptions {
+        private final ArgumentAcceptingOptionSpec<String> bootstrapServerOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> commandConfigOpt;
+
+        private final OptionSpecBuilder listOpt;
+
+        private final OptionSpecBuilder createOpt;
+
+        private final OptionSpecBuilder deleteOpt;
+
+        private final OptionSpecBuilder alterOpt;
+
+        private final OptionSpecBuilder describeOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> topicOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> topicIdOpt;
+
+        private final String nl;
+
+        private static final String KAFKA_CONFIGS_CLI_SUPPORTS_ALTERING_TOPIC_CONFIGS_WITH_A_BOOTSTRAP_SERVER =
+                " (the kafka-configs CLI supports altering topic configs with a --bootstrap-server option)";
+
+        private final ArgumentAcceptingOptionSpec<String> configOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> deleteConfigOpt;
+
+        private final ArgumentAcceptingOptionSpec<Integer> partitionsOpt;
+
+        private final ArgumentAcceptingOptionSpec<Integer> replicationFactorOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> replicaAssignmentOpt;
+
+        private final OptionSpecBuilder reportUnderReplicatedPartitionsOpt;
+
+        private final OptionSpecBuilder reportUnavailablePartitionsOpt;
+
+        private final OptionSpecBuilder reportUnderMinIsrPartitionsOpt;
+
+        private final OptionSpecBuilder reportAtMinIsrPartitionsOpt;
+
+        private final OptionSpecBuilder topicsWithOverridesOpt;
+
+        private final OptionSpecBuilder ifExistsOpt;
+
+        private final OptionSpecBuilder ifNotExistsOpt;
+
+        private final OptionSpecBuilder excludeInternalTopicOpt;
+
+        private final Set<OptionSpec<?>> allTopicLevelOpts;
+
+        private final Set<OptionSpecBuilder> allReplicationReportOpts;
+
+        public TopicCommandOptions(String[] args) {
+            super(args);
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The Kafka server to connect to.")
+                .withRequiredArg()
+                .describedAs("server to connect to")
+                .ofType(String.class);
+            commandConfigOpt = parser.accepts("command-config", "Property file containing configs to be passed to Admin Client. " +
+                    "This is used only with --bootstrap-server option for describing and altering broker configs.")
+                .withRequiredArg()
+                .describedAs("command config property file")
+                .ofType(String.class);
+
+
+            listOpt = parser.accepts("list", "List all available topics.");
+            createOpt = parser.accepts("create", "Create a new topic.");
+            deleteOpt = parser.accepts("delete", "Delete a topic");
+            alterOpt = parser.accepts("alter", "Alter the number of partitions, replica assignment, and/or configuration for the topic.");
+            describeOpt = parser.accepts("describe", "List details for the given topics.");
+            topicOpt = parser.accepts("topic", "Alter the number of partitions and replica assignment. " +
+                            "Update the configuration of an existing topic via --alter is no longer supported here" +
+                            KAFKA_CONFIGS_CLI_SUPPORTS_ALTERING_TOPIC_CONFIGS_WITH_A_BOOTSTRAP_SERVER + ".")
+                .withRequiredArg()
+                .describedAs("topic")
+                .ofType(String.class);
+            topicIdOpt = parser.accepts("topic-id", "The topic-id to describe." +
+                    "This is used only with --bootstrap-server option for describing topics.")
+                .withRequiredArg()
+                .describedAs("topic-id")
+                .ofType(String.class);
+            nl = System.getProperty("line.separator");
+
+            String configNames = LogConfig.configNames().stream()
+                .map(name -> "\t" + name)
+                .collect(Collectors.joining("nl"));

Review Comment:
   Shouldn't we use `System.getProperty("line.separator")` instead of `nl`?



##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);

Review Comment:
   Nit: the convention here is to use uppercase for constants.



##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String[] args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {
+            System.err.println(e.getMessage());
+            return 1;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        opts.checkArgs();
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        log.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")
+    static Map<Integer, List<Integer>> parseReplicaAssignment(String replicaAssignmentList) {
+        String[] partitionList = replicaAssignmentList.split(",");
+        Map<Integer, List<Integer>> ret = new LinkedHashMap<Integer, List<Integer>>();
+        for (int i = 0; i < partitionList.length; i++) {
+            List<Integer> brokerList = Arrays.stream(partitionList[i].split(":"))
+                .map(String::trim)
+                .mapToInt(Integer::parseInt)
+                .boxed()
+                .collect(Collectors.toList());
+            Collection<Integer> duplicateBrokers = ToolsUtils.duplicates(brokerList);
+            if (!duplicateBrokers.isEmpty()) {
+                throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: " +
+                    duplicateBrokers.stream()
+                        .map(Object::toString)
+                        .collect(Collectors.joining(","))
+                );
+            }
+            ret.put(i, brokerList);
+            if (ret.get(i).size() != ret.get(0).size()) {
+                throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList);
+            }
+        }
+        return ret;
+    }
+
+    @SuppressWarnings("deprecation")
+    private static Properties parseTopicConfigsToBeAdded(TopicCommandOptions opts) {
+        List<List<String>> configsToBeAdded = opts.topicConfig().orElse(Collections.emptyList())
+            .stream()
+            .map(s -> Arrays.asList(s.split("\\s*=\\s*")))
+            .collect(Collectors.toList());
+
+        if (!configsToBeAdded.stream().allMatch(config -> config.size() == 2)) {
+            throw new IllegalArgumentException("requirement failed: Invalid topic config: all configs to be added must be in the format \"key=val\".");
+        }
+
+        Properties props = new Properties();
+        configsToBeAdded.stream()
+            .forEach(pair -> props.setProperty(pair.get(0).trim(), pair.get(1).trim()));
+        LogConfig.validate(props);
+        if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) {
+            System.out.println("WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " +
+                "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " +
+                "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.");
+        }
+        return props;
+    }
+
+    // It is possible for a reassignment to complete between the time we have fetched its state and the time
+    // we fetch partition metadata. In this case, we ignore the reassignment when determining replication factor.
+    public static boolean isReassignmentInProgress(TopicPartitionInfo tpi, PartitionReassignment ra) {
+        // Reassignment is still in progress as long as the removing and adding replicas are still present
+        Set<Integer> allReplicaIds = tpi.replicas().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> changingReplicaIds = new HashSet<>();
+        if (ra != null) {
+            changingReplicaIds.addAll(ra.removingReplicas());
+            changingReplicaIds.addAll(ra.addingReplicas());
+        }
+        return allReplicaIds.stream().anyMatch(changingReplicaIds::contains);
+
+    }
+
+    private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionReassignment reassignment) {
+        return isReassignmentInProgress(tpi, reassignment) ?
+            reassignment.replicas().size() - reassignment.addingReplicas().size() :
+            tpi.replicas().size();
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopics        Topics that were found to match the requested topic name.
+     * @param requestedTopic     Name of the topic that was requested.
+     * @param requireTopicExists Indicates if the topic needs to exist for the operation to be successful.
+     *                           If set to true, the command will throw an exception if the topic with the
+     *                           requested name does not exist.
+     */
+    private static void ensureTopicExists(List<String> foundTopics, String requestedTopic, Boolean requireTopicExists) {
+        // If no topic name was mentioned, do not need to throw exception.
+        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
+            // If given topic doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
+        }
+    }
+
+    private static List<String> doGetTopics(List<String> allTopics, Optional<String> topicIncludeList, Boolean excludeInternalTopics) {
+        if (topicIncludeList.isPresent()) {
+            IncludeList topicsFilter = new IncludeList(topicIncludeList.get());
+            return allTopics.stream()
+                .filter(topic -> topicsFilter.isTopicAllowed(topic, excludeInternalTopics))
+                .collect(Collectors.toList());
+        } else {
+            return allTopics.stream()
+                .filter(topic -> !(Topic.isInternal(topic) && excludeInternalTopics))
+                .collect(Collectors.toList());
+        }
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopicIds        Topics that were found to match the requested topic id.
+     * @param requestedTopicId     Id of the topic that was requested.
+     * @param requireTopicIdExists Indicates if the topic needs to exist for the operation to be successful.
+     *                             If set to true, the command will throw an exception if the topic with the
+     *                             requested id does not exist.
+     */
+    private static void ensureTopicIdExists(List<Uuid> foundTopicIds, Uuid requestedTopicId, Boolean requireTopicIdExists) {
+        // If no topic id was mentioned, do not need to throw exception.
+        if (requestedTopicId != null && requireTopicIdExists && foundTopicIds.isEmpty()) {
+            // If given topicId doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("TopicId '%s' does not exist as expected", requestedTopicId));
+        }
+    }
+
+    static class CommandTopicPartition {
+        private final TopicCommandOptions opts;
+        private final Optional<String> name;
+        private final Optional<Integer> replicationFactor;
+        private final Optional<Integer> partitions;
+        private final Map<Integer, List<Integer>> replicaAssignment;
+        private final Properties configsToAdd;
+
+        public CommandTopicPartition(TopicCommandOptions options) {
+            opts = options;
+            name = options.topic();
+            partitions = options.partitions();
+            replicationFactor = options.replicationFactor();
+            replicaAssignment = options.replicaAssignment().orElse(Collections.emptyMap());
+            configsToAdd = parseTopicConfigsToBeAdded(options);
+        }
+
+        public Boolean hasReplicaAssignment() {
+            return !replicaAssignment.isEmpty();
+        }
+
+        public Boolean hasPartitions() {
+            return partitions.isPresent();
+        }
+
+        public Boolean ifTopicDoesntExist() {
+            return opts.ifNotExists();
+        }
+    }
+
+    static class TopicDescription {
+        private final String topic;
+        private final Uuid topicId;
+        private final Integer numPartitions;
+        private final Integer replicationFactor;
+        private final Config config;
+        private final Boolean markedForDeletion;
+
+        public TopicDescription(String topic, Uuid topicId, Integer numPartitions, Integer replicationFactor, Config config, Boolean markedForDeletion) {
+            this.topic = topic;
+            this.topicId = topicId;
+            this.numPartitions = numPartitions;
+            this.replicationFactor = replicationFactor;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+        }
+
+        public void printDescription() {
+            String configsAsString = config.entries().stream()
+                .filter(config -> !config.isDefault())
+                .map(ce -> String.format("%s=%s", ce.name(), ce.value()))
+                .collect(Collectors.joining(","));
+            System.out.print(String.format("Topic: %s", topic));
+            if (topicId != Uuid.ZERO_UUID)
+                System.out.print(String.format("\tTopicId: %s", topicId));
+            System.out.print(String.format("\tPartitionCount: %s", numPartitions));
+            System.out.print(String.format("\tReplicationFactor: %s", replicationFactor));
+            System.out.print(String.format("\tConfigs: %s", configsAsString));
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class PartitionDescription {
+        private final String topic;
+        private final TopicPartitionInfo info;
+        private final Config config;
+        private final Boolean markedForDeletion;
+        private final PartitionReassignment reassignment;
+
+        PartitionDescription(String topic,
+                             TopicPartitionInfo info,
+                             Config config,
+                             Boolean markedForDeletion,
+                             PartitionReassignment reassignment) {
+            this.topic = topic;
+            this.info = info;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+            this.reassignment = reassignment;
+        }
+
+        public Integer minIsrCount() {
+            return Integer.parseInt(config.get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value());
+        }
+
+        public Boolean isUnderReplicated() {
+            return getReplicationFactor(info, reassignment) - info.isr().size() > 0;
+        }
+
+        public boolean hasLeader() {
+            return info.leader() != null;
+        }
+
+        public Boolean isUnderMinIsr() {
+            return !hasLeader() ||  info.isr().size() < minIsrCount();
+        }
+
+        public Boolean isAtMinIsrPartitions() {
+            return minIsrCount() == info.isr().size();
+
+        }
+
+        public Boolean hasUnavailablePartitions(Set<Integer> liveBrokers) {
+            return !hasLeader() || !liveBrokers.contains(info.leader().id());
+        }
+
+        public void printDescription() {
+            System.out.print("\tTopic: " + topic);
+            System.out.print("\tPartition: " + info.partition());
+            System.out.print("\tLeader: " + (hasLeader() ? info.leader().id() : "none"));
+            System.out.print("\tReplicas: " + info.replicas().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            System.out.print("\tIsr: " + info.isr().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            if (reassignment != null) {
+                System.out.print("\tAdding Replicas: " + reassignment.addingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+                System.out.print("\tRemoving Replicas: " + reassignment.removingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+            }
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+
+    }
+
+    static class DescribeOptions {
+        private final TopicCommandOptions opts;
+        private final Set<Integer> liveBrokers;
+        private final boolean describeConfigs;
+        private final boolean describePartitions;
+
+        public DescribeOptions(TopicCommandOptions opts, Set<Integer> liveBrokers) {
+            this.opts = opts;
+            this.liveBrokers = liveBrokers;
+            this.describeConfigs = !opts.reportUnavailablePartitions() &&
+                !opts.reportUnderReplicatedPartitions() &&
+                !opts.reportUnderMinIsrPartitions() &&
+                !opts.reportAtMinIsrPartitions();
+            this.describePartitions = !opts.reportOverriddenConfigs();
+        }
+
+        private boolean shouldPrintUnderReplicatedPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderReplicatedPartitions() && partitionDescription.isUnderReplicated();
+        }
+
+        private boolean shouldPrintUnavailablePartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnavailablePartitions() && partitionDescription.hasUnavailablePartitions(liveBrokers);
+        }
+
+        private boolean shouldPrintUnderMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderMinIsrPartitions() && partitionDescription.isUnderMinIsr();
+        }
+
+        private boolean shouldPrintAtMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportAtMinIsrPartitions() && partitionDescription.isAtMinIsrPartitions();
+        }
+
+        private boolean shouldPrintTopicPartition(PartitionDescription partitionDesc) {
+            return describeConfigs ||
+                shouldPrintUnderReplicatedPartitions(partitionDesc) ||
+                shouldPrintUnavailablePartitions(partitionDesc) ||
+                shouldPrintUnderMinIsrPartitions(partitionDesc) ||
+                shouldPrintAtMinIsrPartitions(partitionDesc);
+        }
+
+        public void maybePrintPartitionDescription(PartitionDescription desc) {
+            if (shouldPrintTopicPartition(desc)) {
+                desc.printDescription();
+            }
+        }
+    }
+
+    public static class TopicService implements AutoCloseable {
+        private Admin adminClient;
+
+        public TopicService(Properties commandConfig, Optional<String> bootstrapServer) {
+            this.adminClient = createAdminClient(commandConfig, bootstrapServer);
+        }
+
+        public TopicService(Admin admin) {
+            this.adminClient = admin;
+        }
+
+        private static Admin createAdminClient(Properties commandConfig, Optional<String> bootstrapServer) {
+            if (bootstrapServer.isPresent()) {
+                commandConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);

Review Comment:
   Should be `bootstrapServer.get()`, otherwise you get a ConfigException. I think we should update the integration test to use --bootstrap-server instead of adminClient, so that we can capture errors like this.



##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String[] args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {
+            System.err.println(e.getMessage());
+            return 1;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        opts.checkArgs();
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        log.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")
+    static Map<Integer, List<Integer>> parseReplicaAssignment(String replicaAssignmentList) {
+        String[] partitionList = replicaAssignmentList.split(",");
+        Map<Integer, List<Integer>> ret = new LinkedHashMap<Integer, List<Integer>>();
+        for (int i = 0; i < partitionList.length; i++) {
+            List<Integer> brokerList = Arrays.stream(partitionList[i].split(":"))
+                .map(String::trim)
+                .mapToInt(Integer::parseInt)
+                .boxed()
+                .collect(Collectors.toList());
+            Collection<Integer> duplicateBrokers = ToolsUtils.duplicates(brokerList);
+            if (!duplicateBrokers.isEmpty()) {
+                throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: " +
+                    duplicateBrokers.stream()
+                        .map(Object::toString)
+                        .collect(Collectors.joining(","))
+                );
+            }
+            ret.put(i, brokerList);
+            if (ret.get(i).size() != ret.get(0).size()) {
+                throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList);
+            }
+        }
+        return ret;
+    }
+
+    @SuppressWarnings("deprecation")
+    private static Properties parseTopicConfigsToBeAdded(TopicCommandOptions opts) {
+        List<List<String>> configsToBeAdded = opts.topicConfig().orElse(Collections.emptyList())
+            .stream()
+            .map(s -> Arrays.asList(s.split("\\s*=\\s*")))
+            .collect(Collectors.toList());
+
+        if (!configsToBeAdded.stream().allMatch(config -> config.size() == 2)) {
+            throw new IllegalArgumentException("requirement failed: Invalid topic config: all configs to be added must be in the format \"key=val\".");
+        }
+
+        Properties props = new Properties();
+        configsToBeAdded.stream()
+            .forEach(pair -> props.setProperty(pair.get(0).trim(), pair.get(1).trim()));
+        LogConfig.validate(props);
+        if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) {
+            System.out.println("WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " +
+                "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " +
+                "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.");
+        }
+        return props;
+    }
+
+    // It is possible for a reassignment to complete between the time we have fetched its state and the time
+    // we fetch partition metadata. In this case, we ignore the reassignment when determining replication factor.
+    public static boolean isReassignmentInProgress(TopicPartitionInfo tpi, PartitionReassignment ra) {
+        // Reassignment is still in progress as long as the removing and adding replicas are still present
+        Set<Integer> allReplicaIds = tpi.replicas().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> changingReplicaIds = new HashSet<>();
+        if (ra != null) {
+            changingReplicaIds.addAll(ra.removingReplicas());
+            changingReplicaIds.addAll(ra.addingReplicas());
+        }
+        return allReplicaIds.stream().anyMatch(changingReplicaIds::contains);
+
+    }
+
+    private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionReassignment reassignment) {
+        return isReassignmentInProgress(tpi, reassignment) ?
+            reassignment.replicas().size() - reassignment.addingReplicas().size() :
+            tpi.replicas().size();
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopics        Topics that were found to match the requested topic name.
+     * @param requestedTopic     Name of the topic that was requested.
+     * @param requireTopicExists Indicates if the topic needs to exist for the operation to be successful.
+     *                           If set to true, the command will throw an exception if the topic with the
+     *                           requested name does not exist.
+     */
+    private static void ensureTopicExists(List<String> foundTopics, String requestedTopic, Boolean requireTopicExists) {
+        // If no topic name was mentioned, do not need to throw exception.
+        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
+            // If given topic doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
+        }
+    }
+
+    private static List<String> doGetTopics(List<String> allTopics, Optional<String> topicIncludeList, Boolean excludeInternalTopics) {
+        if (topicIncludeList.isPresent()) {
+            IncludeList topicsFilter = new IncludeList(topicIncludeList.get());
+            return allTopics.stream()
+                .filter(topic -> topicsFilter.isTopicAllowed(topic, excludeInternalTopics))
+                .collect(Collectors.toList());
+        } else {
+            return allTopics.stream()
+                .filter(topic -> !(Topic.isInternal(topic) && excludeInternalTopics))
+                .collect(Collectors.toList());
+        }
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopicIds        Topics that were found to match the requested topic id.
+     * @param requestedTopicId     Id of the topic that was requested.
+     * @param requireTopicIdExists Indicates if the topic needs to exist for the operation to be successful.
+     *                             If set to true, the command will throw an exception if the topic with the
+     *                             requested id does not exist.
+     */
+    private static void ensureTopicIdExists(List<Uuid> foundTopicIds, Uuid requestedTopicId, Boolean requireTopicIdExists) {
+        // If no topic id was mentioned, do not need to throw exception.
+        if (requestedTopicId != null && requireTopicIdExists && foundTopicIds.isEmpty()) {
+            // If given topicId doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("TopicId '%s' does not exist as expected", requestedTopicId));
+        }
+    }
+
+    static class CommandTopicPartition {
+        private final TopicCommandOptions opts;
+        private final Optional<String> name;
+        private final Optional<Integer> replicationFactor;
+        private final Optional<Integer> partitions;
+        private final Map<Integer, List<Integer>> replicaAssignment;
+        private final Properties configsToAdd;
+
+        public CommandTopicPartition(TopicCommandOptions options) {
+            opts = options;
+            name = options.topic();
+            partitions = options.partitions();
+            replicationFactor = options.replicationFactor();
+            replicaAssignment = options.replicaAssignment().orElse(Collections.emptyMap());
+            configsToAdd = parseTopicConfigsToBeAdded(options);
+        }
+
+        public Boolean hasReplicaAssignment() {
+            return !replicaAssignment.isEmpty();
+        }
+
+        public Boolean hasPartitions() {
+            return partitions.isPresent();
+        }
+
+        public Boolean ifTopicDoesntExist() {
+            return opts.ifNotExists();
+        }
+    }
+
+    static class TopicDescription {
+        private final String topic;
+        private final Uuid topicId;
+        private final Integer numPartitions;
+        private final Integer replicationFactor;
+        private final Config config;
+        private final Boolean markedForDeletion;
+
+        public TopicDescription(String topic, Uuid topicId, Integer numPartitions, Integer replicationFactor, Config config, Boolean markedForDeletion) {
+            this.topic = topic;
+            this.topicId = topicId;
+            this.numPartitions = numPartitions;
+            this.replicationFactor = replicationFactor;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+        }
+
+        public void printDescription() {
+            String configsAsString = config.entries().stream()
+                .filter(config -> !config.isDefault())
+                .map(ce -> String.format("%s=%s", ce.name(), ce.value()))
+                .collect(Collectors.joining(","));
+            System.out.print(String.format("Topic: %s", topic));
+            if (topicId != Uuid.ZERO_UUID)
+                System.out.print(String.format("\tTopicId: %s", topicId));
+            System.out.print(String.format("\tPartitionCount: %s", numPartitions));
+            System.out.print(String.format("\tReplicationFactor: %s", replicationFactor));
+            System.out.print(String.format("\tConfigs: %s", configsAsString));
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class PartitionDescription {
+        private final String topic;
+        private final TopicPartitionInfo info;
+        private final Config config;
+        private final Boolean markedForDeletion;
+        private final PartitionReassignment reassignment;
+
+        PartitionDescription(String topic,
+                             TopicPartitionInfo info,
+                             Config config,
+                             Boolean markedForDeletion,
+                             PartitionReassignment reassignment) {
+            this.topic = topic;
+            this.info = info;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+            this.reassignment = reassignment;
+        }
+
+        public Integer minIsrCount() {
+            return Integer.parseInt(config.get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value());
+        }
+
+        public Boolean isUnderReplicated() {
+            return getReplicationFactor(info, reassignment) - info.isr().size() > 0;
+        }
+
+        public boolean hasLeader() {
+            return info.leader() != null;
+        }
+
+        public Boolean isUnderMinIsr() {
+            return !hasLeader() ||  info.isr().size() < minIsrCount();
+        }
+
+        public Boolean isAtMinIsrPartitions() {
+            return minIsrCount() == info.isr().size();
+
+        }
+
+        public Boolean hasUnavailablePartitions(Set<Integer> liveBrokers) {
+            return !hasLeader() || !liveBrokers.contains(info.leader().id());
+        }
+
+        public void printDescription() {
+            System.out.print("\tTopic: " + topic);
+            System.out.print("\tPartition: " + info.partition());
+            System.out.print("\tLeader: " + (hasLeader() ? info.leader().id() : "none"));
+            System.out.print("\tReplicas: " + info.replicas().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            System.out.print("\tIsr: " + info.isr().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            if (reassignment != null) {
+                System.out.print("\tAdding Replicas: " + reassignment.addingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+                System.out.print("\tRemoving Replicas: " + reassignment.removingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+            }
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+
+    }
+
+    static class DescribeOptions {
+        private final TopicCommandOptions opts;
+        private final Set<Integer> liveBrokers;
+        private final boolean describeConfigs;
+        private final boolean describePartitions;
+
+        public DescribeOptions(TopicCommandOptions opts, Set<Integer> liveBrokers) {
+            this.opts = opts;
+            this.liveBrokers = liveBrokers;
+            this.describeConfigs = !opts.reportUnavailablePartitions() &&
+                !opts.reportUnderReplicatedPartitions() &&
+                !opts.reportUnderMinIsrPartitions() &&
+                !opts.reportAtMinIsrPartitions();
+            this.describePartitions = !opts.reportOverriddenConfigs();
+        }
+
+        private boolean shouldPrintUnderReplicatedPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderReplicatedPartitions() && partitionDescription.isUnderReplicated();
+        }
+
+        private boolean shouldPrintUnavailablePartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnavailablePartitions() && partitionDescription.hasUnavailablePartitions(liveBrokers);
+        }
+
+        private boolean shouldPrintUnderMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderMinIsrPartitions() && partitionDescription.isUnderMinIsr();
+        }
+
+        private boolean shouldPrintAtMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportAtMinIsrPartitions() && partitionDescription.isAtMinIsrPartitions();
+        }
+
+        private boolean shouldPrintTopicPartition(PartitionDescription partitionDesc) {
+            return describeConfigs ||
+                shouldPrintUnderReplicatedPartitions(partitionDesc) ||
+                shouldPrintUnavailablePartitions(partitionDesc) ||
+                shouldPrintUnderMinIsrPartitions(partitionDesc) ||
+                shouldPrintAtMinIsrPartitions(partitionDesc);
+        }
+
+        public void maybePrintPartitionDescription(PartitionDescription desc) {
+            if (shouldPrintTopicPartition(desc)) {
+                desc.printDescription();
+            }
+        }
+    }
+
+    public static class TopicService implements AutoCloseable {
+        private Admin adminClient;
+
+        public TopicService(Properties commandConfig, Optional<String> bootstrapServer) {
+            this.adminClient = createAdminClient(commandConfig, bootstrapServer);
+        }
+
+        public TopicService(Admin admin) {
+            this.adminClient = admin;
+        }
+
+        private static Admin createAdminClient(Properties commandConfig, Optional<String> bootstrapServer) {
+            if (bootstrapServer.isPresent()) {
+                commandConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+            }
+            return Admin.create(commandConfig);
+        }
+
+        public void createTopic(TopicCommandOptions opts) throws Exception {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            if (Topic.hasCollisionChars(topic.name.get())) {
+                System.out.println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could " +
+                    "collide. To avoid issues it is best to use either, but not both.");
+            }
+            createTopic(topic);
+        }
+
+        public void createTopic(CommandTopicPartition topic) throws Exception {
+            if (topic.replicationFactor.filter(rf -> rf > Short.MAX_VALUE || rf < 1).isPresent()) {
+                throw new IllegalArgumentException("The replication factor must be between 1 and " + Short.MAX_VALUE + " inclusive");
+            }
+            if (topic.partitions.filter(p -> p < 1).isPresent()) {
+                throw new IllegalArgumentException("The partitions must be greater than 0");
+            }
+
+            try {
+                NewTopic newTopic;
+                if (topic.hasReplicaAssignment()) {
+                    newTopic = new NewTopic(topic.name.get(), topic.replicaAssignment);
+                } else {
+                    newTopic = new NewTopic(topic.name.get(), topic.partitions, topic.replicationFactor.map(Integer::shortValue));
+                }
+
+                Map<String, String> configsMap = topic.configsToAdd.stringPropertyNames().stream()
+                    .collect(Collectors.toMap(name -> name, name -> topic.configsToAdd.getProperty(name)));
+
+                newTopic.configs(configsMap);
+                CreateTopicsResult createResult = adminClient.createTopics(Collections.singleton(newTopic),
+                    new CreateTopicsOptions().retryOnQuotaViolation(false));
+                createResult.all().get();
+                System.out.println("Created topic " + topic.name + ".");
+            } catch (ExecutionException e) {
+                if (e.getCause() == null) {
+                    throw e;
+                }
+                if (!(e.getCause() instanceof TopicExistsException && topic.ifTopicDoesntExist())) {
+                    throw (Exception) e.getCause();
+                }
+            }
+        }
+
+        public void listTopics(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            String results = getTopics(opts.topic(), opts.excludeInternalTopics())
+                .stream()
+                .collect(Collectors.joining("\n"));
+            System.out.println(results);
+        }
+
+        public void alterTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+
+            if (!topics.isEmpty()) {
+                Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo = adminClient.describeTopics(topics).topicNameValues();
+                Map<String, NewPartitions> newPartitions = topics.stream()
+                    .map(topicName -> topicNewPartitions(topic, topicsInfo, topicName))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                adminClient.createPartitions(newPartitions, new CreatePartitionsOptions().retryOnQuotaViolation(false)).all().get();
+            }
+        }
+
+        private AbstractMap.SimpleEntry<String, NewPartitions> topicNewPartitions(
+            CommandTopicPartition topic,
+            Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo,
+            String topicName) {
+            if (topic.hasReplicaAssignment()) {
+                try {
+                    Integer startPartitionId = topicsInfo.get(topicName).get().partitions().size();
+                    Map<Integer, List<Integer>> replicaMap = topic.replicaAssignment.entrySet().stream()
+                        .skip(startPartitionId)
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                    List<List<Integer>> newAssignment = replicaMap.entrySet().stream()
+                        .map(Map.Entry::getValue)
+                        .collect(Collectors.toList());
+                    return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get(), newAssignment));
+                } catch (InterruptedException | ExecutionException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+            return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get()));
+        }
+
+        public Map<TopicPartition, PartitionReassignment> listAllReassignments(Set<TopicPartition> topicPartitions) {
+            try {
+                return adminClient.listPartitionReassignments(topicPartitions).reassignments().get();
+            } catch (ExecutionException e) {
+                Throwable cause = e.getCause();
+                if (cause instanceof UnsupportedVersionException || cause instanceof ClusterAuthorizationException) {
+                    log.debug("Couldn't query reassignments through the AdminClient API: " + cause.getMessage(), cause);
+                    return Collections.emptyMap();
+                } else {
+                    throw new RuntimeException(e);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        public void describeTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            // If topicId is provided and not zero, will use topicId regardless of topic name
+            Optional<Uuid> inputTopicId = opts.topicId()
+                .map(Uuid::fromString).filter(uuid -> uuid != Uuid.ZERO_UUID);
+            Boolean useTopicId = inputTopicId.isPresent();
+
+            List<Uuid> topicIds;
+            List<String> topics;
+            if (useTopicId) {
+                topicIds = getTopicIds(inputTopicId.get(), opts.excludeInternalTopics());
+                topics = Collections.emptyList();
+            } else {
+                topicIds = Collections.emptyList();
+                topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+
+            }
+
+            // Only check topic name when topicId is not provided
+            if (useTopicId) {
+                ensureTopicIdExists(topicIds, inputTopicId.get(), !opts.ifExists());
+            } else {
+                ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            }
+            List<org.apache.kafka.clients.admin.TopicDescription> topicDescriptions = new ArrayList<>();
+
+            if (!topicIds.isEmpty()) {
+                Map<Uuid, org.apache.kafka.clients.admin.TopicDescription> descTopics =
+                    adminClient.describeTopics(TopicCollection.ofTopicIds(topicIds)).allTopicIds().get();
+                topicDescriptions = new ArrayList<>(descTopics.values());
+            }
+
+            if (!topics.isEmpty()) {
+                Map<String, org.apache.kafka.clients.admin.TopicDescription> descTopics =
+                    adminClient.describeTopics(TopicCollection.ofTopicNames(topics)).allTopicNames().get();
+                topicDescriptions = new ArrayList<>(descTopics.values());
+            }
+
+            List<String> topicNames = topicDescriptions.stream()
+                .map(org.apache.kafka.clients.admin.TopicDescription::name)
+                .collect(Collectors.toList());
+            Map<ConfigResource, KafkaFuture<Config>> allConfigs = adminClient.describeConfigs(
+                topicNames.stream()
+                    .map(name -> new ConfigResource(ConfigResource.Type.TOPIC, name))
+                    .collect(Collectors.toList())
+            ).values();
+            List<Integer> liveBrokers = adminClient.describeCluster().nodes().get().stream()
+                .map(Node::id)
+                .collect(Collectors.toList());
+            DescribeOptions describeOptions = new DescribeOptions(opts, new HashSet<>(liveBrokers));
+            Set<TopicPartition> topicPartitions = topicDescriptions
+                .stream()
+                .flatMap(td -> td.partitions().stream()
+                    .map(p -> new TopicPartition(td.name(), p.partition())))
+                .collect(Collectors.toSet());
+            Map<TopicPartition, PartitionReassignment> reassignments = listAllReassignments(topicPartitions);
+            for (org.apache.kafka.clients.admin.TopicDescription td : topicDescriptions) {
+                String topicName = td.name();
+                Uuid topicId = td.topicId();
+                Config config = allConfigs.get(new ConfigResource(ConfigResource.Type.TOPIC, topicName)).get();
+                ArrayList<TopicPartitionInfo> sortedPartitions = new ArrayList<>(td.partitions());
+                sortedPartitions.sort(Comparator.comparingInt(TopicPartitionInfo::partition));
+                printDescribeConfig(opts, describeOptions, reassignments, td, topicName, topicId, config, sortedPartitions);
+                printPartitionDescription(describeOptions, reassignments, td, topicName, config, sortedPartitions);
+            }
+        }
+
+        private void printPartitionDescription(DescribeOptions describeOptions, Map<TopicPartition, PartitionReassignment> reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Config config, ArrayList<TopicPartitionInfo> sortedPartitions) {
+            if (describeOptions.describePartitions) {
+                for (TopicPartitionInfo partition : sortedPartitions) {
+                    PartitionReassignment reassignment =
+                        reassignments.get(new TopicPartition(td.name(), partition.partition()));
+                    PartitionDescription partitionDesc = new PartitionDescription(topicName,
+                        partition, config, false, reassignment);
+                    describeOptions.maybePrintPartitionDescription(partitionDesc);
+                }
+            }
+        }
+
+        private void printDescribeConfig(TopicCommandOptions opts, DescribeOptions describeOptions, Map<TopicPartition, PartitionReassignment> reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Uuid topicId, Config config, ArrayList<TopicPartitionInfo> sortedPartitions) {
+            if (describeOptions.describeConfigs) {
+                List<ConfigEntry> entries = new ArrayList<>(config.entries());
+                boolean hasNonDefault = entries.stream().anyMatch(e -> !e.isDefault());
+                if (!opts.reportOverriddenConfigs() || hasNonDefault) {
+                    int numPartitions = td.partitions().size();
+                    TopicPartitionInfo firstPartition = sortedPartitions.get(0);
+                    PartitionReassignment reassignment =
+                        reassignments.get(new TopicPartition(td.name(), firstPartition.partition()));
+                    TopicDescription topicDesc = new TopicDescription(topicName, topicId,
+                        numPartitions, getReplicationFactor(firstPartition, reassignment),
+                        config, false);
+                    topicDesc.printDescription();
+                }
+            }
+        }
+
+        public void deleteTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            adminClient.deleteTopics(Collections.unmodifiableList(topics),
+                new DeleteTopicsOptions().retryOnQuotaViolation(false)
+            ).all().get();
+        }
+
+        public List<String> getTopics(Optional<String> topicIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
+            ListTopicsOptions listTopicsOptions = new ListTopicsOptions();
+            if (!excludeInternalTopics) {
+                listTopicsOptions.listInternal(true);
+            }
+
+            Set<String> allTopics = adminClient.listTopics(listTopicsOptions).names().get();
+            return doGetTopics(allTopics.stream().sorted().collect(Collectors.toList()), topicIncludeList, excludeInternalTopics);
+        }
+
+        public List<Uuid> getTopicIds(Uuid topicIdIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
+            ListTopicsResult allTopics = excludeInternalTopics ? adminClient.listTopics() :
+                adminClient.listTopics(new ListTopicsOptions().listInternal(true));
+            List<Uuid> allTopicIds = null;
+            allTopicIds = allTopics.listings().get().stream()
+                .map(TopicListing::topicId)
+                .sorted()
+                .collect(Collectors.toList());
+            return allTopicIds.contains(topicIdIncludeList) ?
+                Collections.singletonList(topicIdIncludeList) :
+                Collections.emptyList();
+        }
+
+        @Override
+        public void close() throws Exception {
+            adminClient.close();
+        }
+    }
+
+    public static class TopicCommandOptions extends CommandDefaultOptions {
+        private final ArgumentAcceptingOptionSpec<String> bootstrapServerOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> commandConfigOpt;
+
+        private final OptionSpecBuilder listOpt;
+
+        private final OptionSpecBuilder createOpt;
+
+        private final OptionSpecBuilder deleteOpt;
+
+        private final OptionSpecBuilder alterOpt;
+
+        private final OptionSpecBuilder describeOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> topicOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> topicIdOpt;
+
+        private final String nl;
+
+        private static final String KAFKA_CONFIGS_CLI_SUPPORTS_ALTERING_TOPIC_CONFIGS_WITH_A_BOOTSTRAP_SERVER =
+                " (the kafka-configs CLI supports altering topic configs with a --bootstrap-server option)";
+
+        private final ArgumentAcceptingOptionSpec<String> configOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> deleteConfigOpt;
+
+        private final ArgumentAcceptingOptionSpec<Integer> partitionsOpt;
+
+        private final ArgumentAcceptingOptionSpec<Integer> replicationFactorOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> replicaAssignmentOpt;
+
+        private final OptionSpecBuilder reportUnderReplicatedPartitionsOpt;
+
+        private final OptionSpecBuilder reportUnavailablePartitionsOpt;
+
+        private final OptionSpecBuilder reportUnderMinIsrPartitionsOpt;
+
+        private final OptionSpecBuilder reportAtMinIsrPartitionsOpt;
+
+        private final OptionSpecBuilder topicsWithOverridesOpt;
+
+        private final OptionSpecBuilder ifExistsOpt;
+
+        private final OptionSpecBuilder ifNotExistsOpt;
+
+        private final OptionSpecBuilder excludeInternalTopicOpt;
+
+        private final Set<OptionSpec<?>> allTopicLevelOpts;
+
+        private final Set<OptionSpecBuilder> allReplicationReportOpts;
+
+        public TopicCommandOptions(String[] args) {
+            super(args);
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The Kafka server to connect to.")
+                .withRequiredArg()
+                .describedAs("server to connect to")
+                .ofType(String.class);
+            commandConfigOpt = parser.accepts("command-config", "Property file containing configs to be passed to Admin Client. " +
+                    "This is used only with --bootstrap-server option for describing and altering broker configs.")
+                .withRequiredArg()
+                .describedAs("command config property file")
+                .ofType(String.class);
+
+
+            listOpt = parser.accepts("list", "List all available topics.");
+            createOpt = parser.accepts("create", "Create a new topic.");
+            deleteOpt = parser.accepts("delete", "Delete a topic");
+            alterOpt = parser.accepts("alter", "Alter the number of partitions, replica assignment, and/or configuration for the topic.");
+            describeOpt = parser.accepts("describe", "List details for the given topics.");
+            topicOpt = parser.accepts("topic", "Alter the number of partitions and replica assignment. " +
+                            "Update the configuration of an existing topic via --alter is no longer supported here" +

Review Comment:
   Why did you removed the message about the alter option not being supported for an existing topic configuration? That way, the user has to run the command and fail to discover that.



##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String[] args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {

Review Comment:
   We never throw TerseException, so this block can be removed.



##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String[] args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {
+            System.err.println(e.getMessage());
+            return 1;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        opts.checkArgs();
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        log.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")
+    static Map<Integer, List<Integer>> parseReplicaAssignment(String replicaAssignmentList) {
+        String[] partitionList = replicaAssignmentList.split(",");
+        Map<Integer, List<Integer>> ret = new LinkedHashMap<Integer, List<Integer>>();
+        for (int i = 0; i < partitionList.length; i++) {
+            List<Integer> brokerList = Arrays.stream(partitionList[i].split(":"))
+                .map(String::trim)
+                .mapToInt(Integer::parseInt)
+                .boxed()
+                .collect(Collectors.toList());
+            Collection<Integer> duplicateBrokers = ToolsUtils.duplicates(brokerList);
+            if (!duplicateBrokers.isEmpty()) {
+                throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: " +
+                    duplicateBrokers.stream()
+                        .map(Object::toString)
+                        .collect(Collectors.joining(","))
+                );
+            }
+            ret.put(i, brokerList);
+            if (ret.get(i).size() != ret.get(0).size()) {
+                throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList);
+            }
+        }
+        return ret;
+    }
+
+    @SuppressWarnings("deprecation")
+    private static Properties parseTopicConfigsToBeAdded(TopicCommandOptions opts) {
+        List<List<String>> configsToBeAdded = opts.topicConfig().orElse(Collections.emptyList())
+            .stream()
+            .map(s -> Arrays.asList(s.split("\\s*=\\s*")))
+            .collect(Collectors.toList());
+
+        if (!configsToBeAdded.stream().allMatch(config -> config.size() == 2)) {
+            throw new IllegalArgumentException("requirement failed: Invalid topic config: all configs to be added must be in the format \"key=val\".");
+        }
+
+        Properties props = new Properties();
+        configsToBeAdded.stream()
+            .forEach(pair -> props.setProperty(pair.get(0).trim(), pair.get(1).trim()));
+        LogConfig.validate(props);
+        if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) {
+            System.out.println("WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " +
+                "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " +
+                "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.");
+        }
+        return props;
+    }
+
+    // It is possible for a reassignment to complete between the time we have fetched its state and the time
+    // we fetch partition metadata. In this case, we ignore the reassignment when determining replication factor.
+    public static boolean isReassignmentInProgress(TopicPartitionInfo tpi, PartitionReassignment ra) {
+        // Reassignment is still in progress as long as the removing and adding replicas are still present
+        Set<Integer> allReplicaIds = tpi.replicas().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> changingReplicaIds = new HashSet<>();
+        if (ra != null) {
+            changingReplicaIds.addAll(ra.removingReplicas());
+            changingReplicaIds.addAll(ra.addingReplicas());
+        }
+        return allReplicaIds.stream().anyMatch(changingReplicaIds::contains);
+
+    }
+
+    private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionReassignment reassignment) {
+        return isReassignmentInProgress(tpi, reassignment) ?
+            reassignment.replicas().size() - reassignment.addingReplicas().size() :
+            tpi.replicas().size();
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopics        Topics that were found to match the requested topic name.
+     * @param requestedTopic     Name of the topic that was requested.
+     * @param requireTopicExists Indicates if the topic needs to exist for the operation to be successful.
+     *                           If set to true, the command will throw an exception if the topic with the
+     *                           requested name does not exist.
+     */
+    private static void ensureTopicExists(List<String> foundTopics, String requestedTopic, Boolean requireTopicExists) {
+        // If no topic name was mentioned, do not need to throw exception.
+        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
+            // If given topic doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
+        }
+    }
+
+    private static List<String> doGetTopics(List<String> allTopics, Optional<String> topicIncludeList, Boolean excludeInternalTopics) {
+        if (topicIncludeList.isPresent()) {
+            IncludeList topicsFilter = new IncludeList(topicIncludeList.get());
+            return allTopics.stream()
+                .filter(topic -> topicsFilter.isTopicAllowed(topic, excludeInternalTopics))
+                .collect(Collectors.toList());
+        } else {
+            return allTopics.stream()
+                .filter(topic -> !(Topic.isInternal(topic) && excludeInternalTopics))
+                .collect(Collectors.toList());
+        }
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopicIds        Topics that were found to match the requested topic id.
+     * @param requestedTopicId     Id of the topic that was requested.
+     * @param requireTopicIdExists Indicates if the topic needs to exist for the operation to be successful.
+     *                             If set to true, the command will throw an exception if the topic with the
+     *                             requested id does not exist.
+     */
+    private static void ensureTopicIdExists(List<Uuid> foundTopicIds, Uuid requestedTopicId, Boolean requireTopicIdExists) {
+        // If no topic id was mentioned, do not need to throw exception.
+        if (requestedTopicId != null && requireTopicIdExists && foundTopicIds.isEmpty()) {
+            // If given topicId doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("TopicId '%s' does not exist as expected", requestedTopicId));
+        }
+    }
+
+    static class CommandTopicPartition {
+        private final TopicCommandOptions opts;
+        private final Optional<String> name;
+        private final Optional<Integer> replicationFactor;
+        private final Optional<Integer> partitions;
+        private final Map<Integer, List<Integer>> replicaAssignment;
+        private final Properties configsToAdd;
+
+        public CommandTopicPartition(TopicCommandOptions options) {
+            opts = options;
+            name = options.topic();
+            partitions = options.partitions();
+            replicationFactor = options.replicationFactor();
+            replicaAssignment = options.replicaAssignment().orElse(Collections.emptyMap());
+            configsToAdd = parseTopicConfigsToBeAdded(options);
+        }
+
+        public Boolean hasReplicaAssignment() {
+            return !replicaAssignment.isEmpty();
+        }
+
+        public Boolean hasPartitions() {
+            return partitions.isPresent();
+        }
+
+        public Boolean ifTopicDoesntExist() {
+            return opts.ifNotExists();
+        }
+    }
+
+    static class TopicDescription {
+        private final String topic;
+        private final Uuid topicId;
+        private final Integer numPartitions;
+        private final Integer replicationFactor;
+        private final Config config;
+        private final Boolean markedForDeletion;
+
+        public TopicDescription(String topic, Uuid topicId, Integer numPartitions, Integer replicationFactor, Config config, Boolean markedForDeletion) {
+            this.topic = topic;
+            this.topicId = topicId;
+            this.numPartitions = numPartitions;
+            this.replicationFactor = replicationFactor;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+        }
+
+        public void printDescription() {
+            String configsAsString = config.entries().stream()
+                .filter(config -> !config.isDefault())
+                .map(ce -> String.format("%s=%s", ce.name(), ce.value()))
+                .collect(Collectors.joining(","));
+            System.out.print(String.format("Topic: %s", topic));
+            if (topicId != Uuid.ZERO_UUID)
+                System.out.print(String.format("\tTopicId: %s", topicId));
+            System.out.print(String.format("\tPartitionCount: %s", numPartitions));
+            System.out.print(String.format("\tReplicationFactor: %s", replicationFactor));
+            System.out.print(String.format("\tConfigs: %s", configsAsString));
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class PartitionDescription {
+        private final String topic;
+        private final TopicPartitionInfo info;
+        private final Config config;
+        private final Boolean markedForDeletion;
+        private final PartitionReassignment reassignment;
+
+        PartitionDescription(String topic,
+                             TopicPartitionInfo info,
+                             Config config,
+                             Boolean markedForDeletion,
+                             PartitionReassignment reassignment) {
+            this.topic = topic;
+            this.info = info;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+            this.reassignment = reassignment;
+        }
+
+        public Integer minIsrCount() {
+            return Integer.parseInt(config.get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value());
+        }
+
+        public Boolean isUnderReplicated() {
+            return getReplicationFactor(info, reassignment) - info.isr().size() > 0;
+        }
+
+        public boolean hasLeader() {
+            return info.leader() != null;
+        }
+
+        public Boolean isUnderMinIsr() {
+            return !hasLeader() ||  info.isr().size() < minIsrCount();
+        }
+
+        public Boolean isAtMinIsrPartitions() {
+            return minIsrCount() == info.isr().size();
+
+        }
+
+        public Boolean hasUnavailablePartitions(Set<Integer> liveBrokers) {
+            return !hasLeader() || !liveBrokers.contains(info.leader().id());
+        }
+
+        public void printDescription() {
+            System.out.print("\tTopic: " + topic);
+            System.out.print("\tPartition: " + info.partition());
+            System.out.print("\tLeader: " + (hasLeader() ? info.leader().id() : "none"));
+            System.out.print("\tReplicas: " + info.replicas().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            System.out.print("\tIsr: " + info.isr().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            if (reassignment != null) {
+                System.out.print("\tAdding Replicas: " + reassignment.addingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+                System.out.print("\tRemoving Replicas: " + reassignment.removingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+            }
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+
+    }
+
+    static class DescribeOptions {
+        private final TopicCommandOptions opts;
+        private final Set<Integer> liveBrokers;
+        private final boolean describeConfigs;
+        private final boolean describePartitions;
+
+        public DescribeOptions(TopicCommandOptions opts, Set<Integer> liveBrokers) {
+            this.opts = opts;
+            this.liveBrokers = liveBrokers;
+            this.describeConfigs = !opts.reportUnavailablePartitions() &&
+                !opts.reportUnderReplicatedPartitions() &&
+                !opts.reportUnderMinIsrPartitions() &&
+                !opts.reportAtMinIsrPartitions();
+            this.describePartitions = !opts.reportOverriddenConfigs();
+        }
+
+        private boolean shouldPrintUnderReplicatedPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderReplicatedPartitions() && partitionDescription.isUnderReplicated();
+        }
+
+        private boolean shouldPrintUnavailablePartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnavailablePartitions() && partitionDescription.hasUnavailablePartitions(liveBrokers);
+        }
+
+        private boolean shouldPrintUnderMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderMinIsrPartitions() && partitionDescription.isUnderMinIsr();
+        }
+
+        private boolean shouldPrintAtMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportAtMinIsrPartitions() && partitionDescription.isAtMinIsrPartitions();
+        }
+
+        private boolean shouldPrintTopicPartition(PartitionDescription partitionDesc) {
+            return describeConfigs ||
+                shouldPrintUnderReplicatedPartitions(partitionDesc) ||
+                shouldPrintUnavailablePartitions(partitionDesc) ||
+                shouldPrintUnderMinIsrPartitions(partitionDesc) ||
+                shouldPrintAtMinIsrPartitions(partitionDesc);
+        }
+
+        public void maybePrintPartitionDescription(PartitionDescription desc) {
+            if (shouldPrintTopicPartition(desc)) {
+                desc.printDescription();
+            }
+        }
+    }
+
+    public static class TopicService implements AutoCloseable {
+        private Admin adminClient;
+
+        public TopicService(Properties commandConfig, Optional<String> bootstrapServer) {
+            this.adminClient = createAdminClient(commandConfig, bootstrapServer);
+        }
+
+        public TopicService(Admin admin) {
+            this.adminClient = admin;
+        }
+
+        private static Admin createAdminClient(Properties commandConfig, Optional<String> bootstrapServer) {
+            if (bootstrapServer.isPresent()) {
+                commandConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+            }
+            return Admin.create(commandConfig);
+        }
+
+        public void createTopic(TopicCommandOptions opts) throws Exception {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            if (Topic.hasCollisionChars(topic.name.get())) {
+                System.out.println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could " +
+                    "collide. To avoid issues it is best to use either, but not both.");
+            }
+            createTopic(topic);
+        }
+
+        public void createTopic(CommandTopicPartition topic) throws Exception {
+            if (topic.replicationFactor.filter(rf -> rf > Short.MAX_VALUE || rf < 1).isPresent()) {
+                throw new IllegalArgumentException("The replication factor must be between 1 and " + Short.MAX_VALUE + " inclusive");
+            }
+            if (topic.partitions.filter(p -> p < 1).isPresent()) {
+                throw new IllegalArgumentException("The partitions must be greater than 0");
+            }
+
+            try {
+                NewTopic newTopic;
+                if (topic.hasReplicaAssignment()) {
+                    newTopic = new NewTopic(topic.name.get(), topic.replicaAssignment);
+                } else {
+                    newTopic = new NewTopic(topic.name.get(), topic.partitions, topic.replicationFactor.map(Integer::shortValue));
+                }
+
+                Map<String, String> configsMap = topic.configsToAdd.stringPropertyNames().stream()
+                    .collect(Collectors.toMap(name -> name, name -> topic.configsToAdd.getProperty(name)));
+
+                newTopic.configs(configsMap);
+                CreateTopicsResult createResult = adminClient.createTopics(Collections.singleton(newTopic),
+                    new CreateTopicsOptions().retryOnQuotaViolation(false));
+                createResult.all().get();
+                System.out.println("Created topic " + topic.name + ".");
+            } catch (ExecutionException e) {
+                if (e.getCause() == null) {
+                    throw e;
+                }
+                if (!(e.getCause() instanceof TopicExistsException && topic.ifTopicDoesntExist())) {
+                    throw (Exception) e.getCause();
+                }
+            }
+        }
+
+        public void listTopics(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            String results = getTopics(opts.topic(), opts.excludeInternalTopics())
+                .stream()
+                .collect(Collectors.joining("\n"));
+            System.out.println(results);
+        }
+
+        public void alterTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+
+            if (!topics.isEmpty()) {
+                Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo = adminClient.describeTopics(topics).topicNameValues();
+                Map<String, NewPartitions> newPartitions = topics.stream()
+                    .map(topicName -> topicNewPartitions(topic, topicsInfo, topicName))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                adminClient.createPartitions(newPartitions, new CreatePartitionsOptions().retryOnQuotaViolation(false)).all().get();
+            }
+        }
+
+        private AbstractMap.SimpleEntry<String, NewPartitions> topicNewPartitions(
+            CommandTopicPartition topic,
+            Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo,
+            String topicName) {
+            if (topic.hasReplicaAssignment()) {
+                try {
+                    Integer startPartitionId = topicsInfo.get(topicName).get().partitions().size();
+                    Map<Integer, List<Integer>> replicaMap = topic.replicaAssignment.entrySet().stream()
+                        .skip(startPartitionId)
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                    List<List<Integer>> newAssignment = replicaMap.entrySet().stream()
+                        .map(Map.Entry::getValue)
+                        .collect(Collectors.toList());
+                    return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get(), newAssignment));
+                } catch (InterruptedException | ExecutionException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+            return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get()));
+        }
+
+        public Map<TopicPartition, PartitionReassignment> listAllReassignments(Set<TopicPartition> topicPartitions) {
+            try {
+                return adminClient.listPartitionReassignments(topicPartitions).reassignments().get();
+            } catch (ExecutionException e) {
+                Throwable cause = e.getCause();
+                if (cause instanceof UnsupportedVersionException || cause instanceof ClusterAuthorizationException) {
+                    log.debug("Couldn't query reassignments through the AdminClient API: " + cause.getMessage(), cause);
+                    return Collections.emptyMap();
+                } else {
+                    throw new RuntimeException(e);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        public void describeTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            // If topicId is provided and not zero, will use topicId regardless of topic name
+            Optional<Uuid> inputTopicId = opts.topicId()
+                .map(Uuid::fromString).filter(uuid -> uuid != Uuid.ZERO_UUID);
+            Boolean useTopicId = inputTopicId.isPresent();
+
+            List<Uuid> topicIds;
+            List<String> topics;
+            if (useTopicId) {
+                topicIds = getTopicIds(inputTopicId.get(), opts.excludeInternalTopics());
+                topics = Collections.emptyList();
+            } else {
+                topicIds = Collections.emptyList();
+                topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+

Review Comment:
   Nit: extra row.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1358473310


##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java:
##########
@@ -0,0 +1,1050 @@
+/*
+ * 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;
+
+import kafka.admin.RackAwareTest;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Tag("integration")
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
+public class TopicCommandIntegrationTest extends kafka.integration.KafkaServerTestHarness implements Logging, RackAwareTest {
+    private Short defaultReplicationFactor = 1;
+    private Integer numPartitions = 1;
+    private TopicCommand.TopicService topicService;
+    private Admin adminClient;
+    private String bootstrapServer;
+    private String testTopicName;
+    private Integer defaultTimeout = 10000;
+
+    /**
+     * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every
+     * test and should not reuse previous configurations unless they select their ports randomly when servers are started.
+     *
+     * Note the replica fetch max bytes is set to `1` in order to throttle the rate of replication for test
+     * `testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress`.
+     */
+    @Override
+    public scala.collection.Seq<KafkaConfig> generateConfigs() {
+        Map<Integer, String> rackInfo = new HashMap<>();
+        rackInfo.put(0, "rack1");
+        rackInfo.put(1, "rack2");
+        rackInfo.put(2, "rack2");
+        rackInfo.put(3, "rack1");
+        rackInfo.put(4, "rack3");
+        rackInfo.put(5, "rack3");
+
+        List<Properties> brokerConfigs = ToolsTestUtils
+            .createBrokerProperties(6, zkConnectOrNull(), rackInfo, numPartitions, defaultReplicationFactor);
+
+        List<KafkaConfig> configs = new ArrayList<>();
+        for (Properties props : brokerConfigs) {
+            props.put(KafkaConfig.ReplicaFetchMaxBytesProp(), "1");
+            configs.add(KafkaConfig.fromProps(props));
+        }
+        return JavaConverters.asScalaBuffer(configs).toSeq();
+    }
+
+    private TopicCommand.TopicCommandOptions buildTopicCommandOptionsWithBootstrap(String... opts) {
+        String[] finalOptions = Stream.concat(Arrays.asList(opts).stream(),
+                Arrays.asList("--bootstrap-server", bootstrapServer).stream()
+        ).toArray(String[]::new);
+        return new TopicCommand.TopicCommandOptions(finalOptions);
+    }
+
+    private void createAndWaitTopic(TopicCommand.TopicCommandOptions opts) throws Exception {
+        topicService.createTopic(opts);
+        waitForTopicCreated(opts.topic().get());
+    }
+
+    private void waitForTopicCreated(String topicName) {
+        waitForTopicCreated(topicName, defaultTimeout);
+    }
+
+    private void waitForTopicCreated(String topicName, Integer timeout) {
+        TestUtils.waitForPartitionMetadata(brokers(), topicName, 0, timeout);
+    }
+
+    @BeforeEach
+    public void setUp(TestInfo info) {
+        super.setUp(info);
+        // create adminClient
+        Properties props = new Properties();
+        bootstrapServer = bootstrapServers(listenerName());
+        props.put(org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+        adminClient = Admin.create(props);
+        topicService = new TopicCommand.TopicService(props, Optional.of(bootstrapServer));
+        testTopicName = String.format("%s-%s", info.getTestMethod().get().getName(),
+            new scala.util.Random().alphanumeric().take(10).mkString());

Review Comment:
   found out that we have `TestUtils.randomString` so I switched to use this one instead. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on a diff in pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1326193756


##########
build.gradle:
##########
@@ -1880,9 +1880,10 @@ project(':tools:tools-api') {
 
 project(':tools') {
   archivesBaseName = "kafka-tools"
-
   dependencies {
+    api libs.scalaLibrary

Review Comment:
   I believe it's needed for the test cases as the Kafka Harness is written in Scala and `createBrokerProperties` convert from Java to scala to setup`TestUtils.createBrokerConfig` which is needed to setup `generateConfigs`



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on a diff in pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1326193756


##########
build.gradle:
##########
@@ -1880,9 +1880,10 @@ project(':tools:tools-api') {
 
 project(':tools') {
   archivesBaseName = "kafka-tools"
-
   dependencies {
+    api libs.scalaLibrary

Review Comment:
   I believe it's needed for the test cases as the Kafka Harness is written in Scala and `createBrokerProperties` convert from java to scala to setup`TestUtils.createBrokerConfig`



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on a diff in pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1326137174


##########
build.gradle:
##########
@@ -1880,9 +1880,10 @@ project(':tools:tools-api') {
 
 project(':tools') {
   archivesBaseName = "kafka-tools"
-
   dependencies {
+    api libs.scalaLibrary

Review Comment:
   I think you don't need this one.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1719434392

   @divijvaidya can you review this please? Also I have a note regard the gradle setup here https://github.com/apache/kafka/pull/13201#issuecomment-1717601042. I tried the with gradle 8.3 and the issue seem to presist so don't think upgrading gradle will make a difference for the time being. And I noticed that `tools-api` for example broke the naming convention of apis so am thinking that we may need to rename all common sub-projects to be `<root-project-name>-api` to avoid such a problem in the future. WDYT? 
   cc: @fvaleri as well.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "mimaison (via GitHub)" <gi...@apache.org>.
mimaison commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1753360551

   @OmniaGM Thanks for the update. I've not had time to take a look yet but noticed there's a compilation failure:
   ```
   > Task :tools:compileTestJava
   
   /home/jenkins/jenkins-agent/workspace/Kafka_kafka-pr_PR-13201/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java:78: error: variable TEST_WITH_PARAMETERIZED_QUORUM_NAME is already defined in class ToolsTestUtils
   
       public static final String TEST_WITH_PARAMETERIZED_QUORUM_NAME = "{displayName}.quorum={0}";
   ```


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1358473808


##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java:
##########
@@ -0,0 +1,1050 @@
+/*
+ * 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;
+
+import kafka.admin.RackAwareTest;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Tag("integration")
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
+public class TopicCommandIntegrationTest extends kafka.integration.KafkaServerTestHarness implements Logging, RackAwareTest {
+    private Short defaultReplicationFactor = 1;
+    private Integer numPartitions = 1;
+    private TopicCommand.TopicService topicService;
+    private Admin adminClient;
+    private String bootstrapServer;
+    private String testTopicName;
+    private Integer defaultTimeout = 10000;
+
+    /**
+     * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every
+     * test and should not reuse previous configurations unless they select their ports randomly when servers are started.
+     *
+     * Note the replica fetch max bytes is set to `1` in order to throttle the rate of replication for test
+     * `testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress`.
+     */
+    @Override
+    public scala.collection.Seq<KafkaConfig> generateConfigs() {
+        Map<Integer, String> rackInfo = new HashMap<>();
+        rackInfo.put(0, "rack1");
+        rackInfo.put(1, "rack2");
+        rackInfo.put(2, "rack2");
+        rackInfo.put(3, "rack1");
+        rackInfo.put(4, "rack3");
+        rackInfo.put(5, "rack3");
+
+        List<Properties> brokerConfigs = ToolsTestUtils
+            .createBrokerProperties(6, zkConnectOrNull(), rackInfo, numPartitions, defaultReplicationFactor);
+
+        List<KafkaConfig> configs = new ArrayList<>();
+        for (Properties props : brokerConfigs) {
+            props.put(KafkaConfig.ReplicaFetchMaxBytesProp(), "1");
+            configs.add(KafkaConfig.fromProps(props));
+        }
+        return JavaConverters.asScalaBuffer(configs).toSeq();
+    }
+
+    private TopicCommand.TopicCommandOptions buildTopicCommandOptionsWithBootstrap(String... opts) {
+        String[] finalOptions = Stream.concat(Arrays.asList(opts).stream(),
+                Arrays.asList("--bootstrap-server", bootstrapServer).stream()
+        ).toArray(String[]::new);
+        return new TopicCommand.TopicCommandOptions(finalOptions);
+    }
+
+    private void createAndWaitTopic(TopicCommand.TopicCommandOptions opts) throws Exception {
+        topicService.createTopic(opts);
+        waitForTopicCreated(opts.topic().get());
+    }
+
+    private void waitForTopicCreated(String topicName) {
+        waitForTopicCreated(topicName, defaultTimeout);
+    }
+
+    private void waitForTopicCreated(String topicName, Integer timeout) {
+        TestUtils.waitForPartitionMetadata(brokers(), topicName, 0, timeout);
+    }
+
+    @BeforeEach
+    public void setUp(TestInfo info) {
+        super.setUp(info);
+        // create adminClient
+        Properties props = new Properties();
+        bootstrapServer = bootstrapServers(listenerName());
+        props.put(org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+        adminClient = Admin.create(props);
+        topicService = new TopicCommand.TopicService(props, Optional.of(bootstrapServer));
+        testTopicName = String.format("%s-%s", info.getTestMethod().get().getName(),
+            new scala.util.Random().alphanumeric().take(10).mkString());
+    }
+
+    @AfterEach
+    public void close() throws Exception {
+        if (topicService != null)
+            topicService.close();
+        if (adminClient != null)
+            adminClient.close();
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreate(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", "2", "--replication-factor", "1", "--topic", testTopicName));
+
+        adminClient.listTopics().names().get().contains(testTopicName);

Review Comment:
   Good catch. fixed this



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java:
##########
@@ -0,0 +1,1050 @@
+/*
+ * 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;
+
+import kafka.admin.RackAwareTest;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Tag("integration")
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
+public class TopicCommandIntegrationTest extends kafka.integration.KafkaServerTestHarness implements Logging, RackAwareTest {
+    private Short defaultReplicationFactor = 1;
+    private Integer numPartitions = 1;
+    private TopicCommand.TopicService topicService;
+    private Admin adminClient;
+    private String bootstrapServer;
+    private String testTopicName;
+    private Integer defaultTimeout = 10000;
+
+    /**
+     * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every
+     * test and should not reuse previous configurations unless they select their ports randomly when servers are started.
+     *
+     * Note the replica fetch max bytes is set to `1` in order to throttle the rate of replication for test
+     * `testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress`.
+     */
+    @Override
+    public scala.collection.Seq<KafkaConfig> generateConfigs() {
+        Map<Integer, String> rackInfo = new HashMap<>();
+        rackInfo.put(0, "rack1");
+        rackInfo.put(1, "rack2");
+        rackInfo.put(2, "rack2");
+        rackInfo.put(3, "rack1");
+        rackInfo.put(4, "rack3");
+        rackInfo.put(5, "rack3");
+
+        List<Properties> brokerConfigs = ToolsTestUtils
+            .createBrokerProperties(6, zkConnectOrNull(), rackInfo, numPartitions, defaultReplicationFactor);
+
+        List<KafkaConfig> configs = new ArrayList<>();
+        for (Properties props : brokerConfigs) {
+            props.put(KafkaConfig.ReplicaFetchMaxBytesProp(), "1");
+            configs.add(KafkaConfig.fromProps(props));
+        }
+        return JavaConverters.asScalaBuffer(configs).toSeq();
+    }
+
+    private TopicCommand.TopicCommandOptions buildTopicCommandOptionsWithBootstrap(String... opts) {
+        String[] finalOptions = Stream.concat(Arrays.asList(opts).stream(),
+                Arrays.asList("--bootstrap-server", bootstrapServer).stream()
+        ).toArray(String[]::new);
+        return new TopicCommand.TopicCommandOptions(finalOptions);
+    }
+
+    private void createAndWaitTopic(TopicCommand.TopicCommandOptions opts) throws Exception {
+        topicService.createTopic(opts);
+        waitForTopicCreated(opts.topic().get());
+    }
+
+    private void waitForTopicCreated(String topicName) {
+        waitForTopicCreated(topicName, defaultTimeout);
+    }
+
+    private void waitForTopicCreated(String topicName, Integer timeout) {
+        TestUtils.waitForPartitionMetadata(brokers(), topicName, 0, timeout);
+    }
+
+    @BeforeEach
+    public void setUp(TestInfo info) {
+        super.setUp(info);
+        // create adminClient
+        Properties props = new Properties();
+        bootstrapServer = bootstrapServers(listenerName());
+        props.put(org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+        adminClient = Admin.create(props);
+        topicService = new TopicCommand.TopicService(props, Optional.of(bootstrapServer));
+        testTopicName = String.format("%s-%s", info.getTestMethod().get().getName(),
+            new scala.util.Random().alphanumeric().take(10).mkString());
+    }
+
+    @AfterEach
+    public void close() throws Exception {
+        if (topicService != null)
+            topicService.close();
+        if (adminClient != null)
+            adminClient.close();
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreate(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", "2", "--replication-factor", "1", "--topic", testTopicName));
+
+        adminClient.listTopics().names().get().contains(testTopicName);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithDefaults(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName));
+
+        List<TopicPartitionInfo> partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+        assertEquals(partitions.size(), numPartitions);

Review Comment:
   done, switched the rest as well



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on a diff in pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1297476207


##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String[] args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {
+            System.err.println(e.getMessage());
+            return 1;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        opts.checkArgs();
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        log.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")
+    static Map<Integer, List<Integer>> parseReplicaAssignment(String replicaAssignmentList) {
+        String[] partitionList = replicaAssignmentList.split(",");
+        Map<Integer, List<Integer>> ret = new LinkedHashMap<Integer, List<Integer>>();
+        for (int i = 0; i < partitionList.length; i++) {
+            List<Integer> brokerList = Arrays.stream(partitionList[i].split(":"))
+                .map(String::trim)
+                .mapToInt(Integer::parseInt)
+                .boxed()
+                .collect(Collectors.toList());
+            Collection<Integer> duplicateBrokers = ToolsUtils.duplicates(brokerList);
+            if (!duplicateBrokers.isEmpty()) {
+                throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: " +
+                    duplicateBrokers.stream()
+                        .map(Object::toString)
+                        .collect(Collectors.joining(","))
+                );
+            }
+            ret.put(i, brokerList);
+            if (ret.get(i).size() != ret.get(0).size()) {
+                throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList);
+            }
+        }
+        return ret;
+    }
+
+    @SuppressWarnings("deprecation")
+    private static Properties parseTopicConfigsToBeAdded(TopicCommandOptions opts) {
+        List<List<String>> configsToBeAdded = opts.topicConfig().orElse(Collections.emptyList())
+            .stream()
+            .map(s -> Arrays.asList(s.split("\\s*=\\s*")))
+            .collect(Collectors.toList());
+
+        if (!configsToBeAdded.stream().allMatch(config -> config.size() == 2)) {
+            throw new IllegalArgumentException("requirement failed: Invalid topic config: all configs to be added must be in the format \"key=val\".");
+        }
+
+        Properties props = new Properties();
+        configsToBeAdded.stream()
+            .forEach(pair -> props.setProperty(pair.get(0).trim(), pair.get(1).trim()));
+        LogConfig.validate(props);
+        if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) {
+            System.out.println("WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " +
+                "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " +
+                "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.");
+        }
+        return props;
+    }
+
+    // It is possible for a reassignment to complete between the time we have fetched its state and the time
+    // we fetch partition metadata. In this case, we ignore the reassignment when determining replication factor.
+    public static boolean isReassignmentInProgress(TopicPartitionInfo tpi, PartitionReassignment ra) {
+        // Reassignment is still in progress as long as the removing and adding replicas are still present
+        Set<Integer> allReplicaIds = tpi.replicas().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> changingReplicaIds = new HashSet<>();
+        if (ra != null) {
+            changingReplicaIds.addAll(ra.removingReplicas());
+            changingReplicaIds.addAll(ra.addingReplicas());
+        }
+        return allReplicaIds.stream().anyMatch(changingReplicaIds::contains);
+
+    }
+
+    private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionReassignment reassignment) {
+        return isReassignmentInProgress(tpi, reassignment) ?
+            reassignment.replicas().size() - reassignment.addingReplicas().size() :
+            tpi.replicas().size();
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopics        Topics that were found to match the requested topic name.
+     * @param requestedTopic     Name of the topic that was requested.
+     * @param requireTopicExists Indicates if the topic needs to exist for the operation to be successful.
+     *                           If set to true, the command will throw an exception if the topic with the
+     *                           requested name does not exist.
+     */
+    private static void ensureTopicExists(List<String> foundTopics, String requestedTopic, Boolean requireTopicExists) {
+        // If no topic name was mentioned, do not need to throw exception.
+        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
+            // If given topic doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
+        }
+    }
+
+    private static List<String> doGetTopics(List<String> allTopics, Optional<String> topicIncludeList, Boolean excludeInternalTopics) {
+        if (topicIncludeList.isPresent()) {
+            IncludeList topicsFilter = new IncludeList(topicIncludeList.get());
+            return allTopics.stream()
+                .filter(topic -> topicsFilter.isTopicAllowed(topic, excludeInternalTopics))
+                .collect(Collectors.toList());
+        } else {
+            return allTopics.stream()
+                .filter(topic -> !(Topic.isInternal(topic) && excludeInternalTopics))
+                .collect(Collectors.toList());
+        }
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopicIds        Topics that were found to match the requested topic id.
+     * @param requestedTopicId     Id of the topic that was requested.
+     * @param requireTopicIdExists Indicates if the topic needs to exist for the operation to be successful.
+     *                             If set to true, the command will throw an exception if the topic with the
+     *                             requested id does not exist.
+     */
+    private static void ensureTopicIdExists(List<Uuid> foundTopicIds, Uuid requestedTopicId, Boolean requireTopicIdExists) {
+        // If no topic id was mentioned, do not need to throw exception.
+        if (requestedTopicId != null && requireTopicIdExists && foundTopicIds.isEmpty()) {
+            // If given topicId doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("TopicId '%s' does not exist as expected", requestedTopicId));
+        }
+    }
+
+    static class CommandTopicPartition {
+        private final TopicCommandOptions opts;
+        private final Optional<String> name;
+        private final Optional<Integer> replicationFactor;
+        private final Optional<Integer> partitions;
+        private final Map<Integer, List<Integer>> replicaAssignment;
+        private final Properties configsToAdd;
+
+        public CommandTopicPartition(TopicCommandOptions options) {
+            opts = options;
+            name = options.topic();
+            partitions = options.partitions();
+            replicationFactor = options.replicationFactor();
+            replicaAssignment = options.replicaAssignment().orElse(Collections.emptyMap());
+            configsToAdd = parseTopicConfigsToBeAdded(options);
+        }
+
+        public Boolean hasReplicaAssignment() {
+            return !replicaAssignment.isEmpty();
+        }
+
+        public Boolean hasPartitions() {
+            return partitions.isPresent();
+        }
+
+        public Boolean ifTopicDoesntExist() {
+            return opts.ifNotExists();
+        }
+    }
+
+    static class TopicDescription {
+        private final String topic;
+        private final Uuid topicId;
+        private final Integer numPartitions;
+        private final Integer replicationFactor;
+        private final Config config;
+        private final Boolean markedForDeletion;
+
+        public TopicDescription(String topic, Uuid topicId, Integer numPartitions, Integer replicationFactor, Config config, Boolean markedForDeletion) {
+            this.topic = topic;
+            this.topicId = topicId;
+            this.numPartitions = numPartitions;
+            this.replicationFactor = replicationFactor;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+        }
+
+        public void printDescription() {
+            String configsAsString = config.entries().stream()
+                .filter(config -> !config.isDefault())
+                .map(ce -> String.format("%s=%s", ce.name(), ce.value()))
+                .collect(Collectors.joining(","));
+            System.out.print(String.format("Topic: %s", topic));
+            if (topicId != Uuid.ZERO_UUID)
+                System.out.print(String.format("\tTopicId: %s", topicId));
+            System.out.print(String.format("\tPartitionCount: %s", numPartitions));
+            System.out.print(String.format("\tReplicationFactor: %s", replicationFactor));
+            System.out.print(String.format("\tConfigs: %s", configsAsString));
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class PartitionDescription {
+        private final String topic;
+        private final TopicPartitionInfo info;
+        private final Config config;
+        private final Boolean markedForDeletion;
+        private final PartitionReassignment reassignment;
+
+        PartitionDescription(String topic,
+                             TopicPartitionInfo info,
+                             Config config,
+                             Boolean markedForDeletion,
+                             PartitionReassignment reassignment) {
+            this.topic = topic;
+            this.info = info;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+            this.reassignment = reassignment;
+        }
+
+        public Integer minIsrCount() {
+            return Integer.parseInt(config.get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value());
+        }
+
+        public Boolean isUnderReplicated() {
+            return getReplicationFactor(info, reassignment) - info.isr().size() > 0;
+        }
+
+        public boolean hasLeader() {
+            return info.leader() != null;
+        }
+
+        public Boolean isUnderMinIsr() {
+            return !hasLeader() ||  info.isr().size() < minIsrCount();
+        }
+
+        public Boolean isAtMinIsrPartitions() {
+            return minIsrCount() == info.isr().size();
+
+        }
+
+        public Boolean hasUnavailablePartitions(Set<Integer> liveBrokers) {
+            return !hasLeader() || !liveBrokers.contains(info.leader().id());
+        }
+
+        public void printDescription() {
+            System.out.print("\tTopic: " + topic);
+            System.out.print("\tPartition: " + info.partition());
+            System.out.print("\tLeader: " + (hasLeader() ? info.leader().id() : "none"));
+            System.out.print("\tReplicas: " + info.replicas().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            System.out.print("\tIsr: " + info.isr().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            if (reassignment != null) {
+                System.out.print("\tAdding Replicas: " + reassignment.addingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+                System.out.print("\tRemoving Replicas: " + reassignment.removingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+            }
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+
+    }
+
+    static class DescribeOptions {
+        private final TopicCommandOptions opts;
+        private final Set<Integer> liveBrokers;
+        private final boolean describeConfigs;
+        private final boolean describePartitions;
+
+        public DescribeOptions(TopicCommandOptions opts, Set<Integer> liveBrokers) {
+            this.opts = opts;
+            this.liveBrokers = liveBrokers;
+            this.describeConfigs = !opts.reportUnavailablePartitions() &&
+                !opts.reportUnderReplicatedPartitions() &&
+                !opts.reportUnderMinIsrPartitions() &&
+                !opts.reportAtMinIsrPartitions();
+            this.describePartitions = !opts.reportOverriddenConfigs();
+        }
+
+        private boolean shouldPrintUnderReplicatedPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderReplicatedPartitions() && partitionDescription.isUnderReplicated();
+        }
+
+        private boolean shouldPrintUnavailablePartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnavailablePartitions() && partitionDescription.hasUnavailablePartitions(liveBrokers);
+        }
+
+        private boolean shouldPrintUnderMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderMinIsrPartitions() && partitionDescription.isUnderMinIsr();
+        }
+
+        private boolean shouldPrintAtMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportAtMinIsrPartitions() && partitionDescription.isAtMinIsrPartitions();
+        }
+
+        private boolean shouldPrintTopicPartition(PartitionDescription partitionDesc) {
+            return describeConfigs ||
+                shouldPrintUnderReplicatedPartitions(partitionDesc) ||
+                shouldPrintUnavailablePartitions(partitionDesc) ||
+                shouldPrintUnderMinIsrPartitions(partitionDesc) ||
+                shouldPrintAtMinIsrPartitions(partitionDesc);
+        }
+
+        public void maybePrintPartitionDescription(PartitionDescription desc) {
+            if (shouldPrintTopicPartition(desc)) {
+                desc.printDescription();
+            }
+        }
+    }
+
+    public static class TopicService implements AutoCloseable {
+        private Admin adminClient;
+
+        public TopicService(Properties commandConfig, Optional<String> bootstrapServer) {
+            this.adminClient = createAdminClient(commandConfig, bootstrapServer);
+        }
+
+        public TopicService(Admin admin) {
+            this.adminClient = admin;
+        }
+
+        private static Admin createAdminClient(Properties commandConfig, Optional<String> bootstrapServer) {
+            if (bootstrapServer.isPresent()) {
+                commandConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+            }
+            return Admin.create(commandConfig);
+        }
+
+        public void createTopic(TopicCommandOptions opts) throws Exception {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            if (Topic.hasCollisionChars(topic.name.get())) {
+                System.out.println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could " +
+                    "collide. To avoid issues it is best to use either, but not both.");
+            }
+            createTopic(topic);
+        }
+
+        public void createTopic(CommandTopicPartition topic) throws Exception {
+            if (topic.replicationFactor.filter(rf -> rf > Short.MAX_VALUE || rf < 1).isPresent()) {
+                throw new IllegalArgumentException("The replication factor must be between 1 and " + Short.MAX_VALUE + " inclusive");
+            }
+            if (topic.partitions.filter(p -> p < 1).isPresent()) {
+                throw new IllegalArgumentException("The partitions must be greater than 0");
+            }
+
+            try {
+                NewTopic newTopic;
+                if (topic.hasReplicaAssignment()) {
+                    newTopic = new NewTopic(topic.name.get(), topic.replicaAssignment);
+                } else {
+                    newTopic = new NewTopic(topic.name.get(), topic.partitions, topic.replicationFactor.map(Integer::shortValue));
+                }
+
+                Map<String, String> configsMap = topic.configsToAdd.stringPropertyNames().stream()
+                    .collect(Collectors.toMap(name -> name, name -> topic.configsToAdd.getProperty(name)));
+
+                newTopic.configs(configsMap);
+                CreateTopicsResult createResult = adminClient.createTopics(Collections.singleton(newTopic),
+                    new CreateTopicsOptions().retryOnQuotaViolation(false));
+                createResult.all().get();
+                System.out.println("Created topic " + topic.name + ".");
+            } catch (ExecutionException e) {
+                if (e.getCause() == null) {
+                    throw e;
+                }
+                if (!(e.getCause() instanceof TopicExistsException && topic.ifTopicDoesntExist())) {
+                    throw (Exception) e.getCause();
+                }
+            }
+        }
+
+        public void listTopics(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            String results = getTopics(opts.topic(), opts.excludeInternalTopics())
+                .stream()
+                .collect(Collectors.joining("\n"));
+            System.out.println(results);
+        }
+
+        public void alterTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+
+            if (!topics.isEmpty()) {
+                Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo = adminClient.describeTopics(topics).topicNameValues();
+                Map<String, NewPartitions> newPartitions = topics.stream()
+                    .map(topicName -> topicNewPartitions(topic, topicsInfo, topicName))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                adminClient.createPartitions(newPartitions, new CreatePartitionsOptions().retryOnQuotaViolation(false)).all().get();
+            }
+        }
+
+        private AbstractMap.SimpleEntry<String, NewPartitions> topicNewPartitions(
+            CommandTopicPartition topic,
+            Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo,
+            String topicName) {
+            if (topic.hasReplicaAssignment()) {
+                try {
+                    Integer startPartitionId = topicsInfo.get(topicName).get().partitions().size();
+                    Map<Integer, List<Integer>> replicaMap = topic.replicaAssignment.entrySet().stream()
+                        .skip(startPartitionId)
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                    List<List<Integer>> newAssignment = replicaMap.entrySet().stream()
+                        .map(Map.Entry::getValue)
+                        .collect(Collectors.toList());
+                    return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get(), newAssignment));
+                } catch (InterruptedException | ExecutionException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+            return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get()));
+        }
+
+        public Map<TopicPartition, PartitionReassignment> listAllReassignments(Set<TopicPartition> topicPartitions) {
+            try {
+                return adminClient.listPartitionReassignments(topicPartitions).reassignments().get();
+            } catch (ExecutionException e) {
+                Throwable cause = e.getCause();
+                if (cause instanceof UnsupportedVersionException || cause instanceof ClusterAuthorizationException) {
+                    log.debug("Couldn't query reassignments through the AdminClient API: " + cause.getMessage(), cause);
+                    return Collections.emptyMap();
+                } else {
+                    throw new RuntimeException(e);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        public void describeTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            // If topicId is provided and not zero, will use topicId regardless of topic name
+            Optional<Uuid> inputTopicId = opts.topicId()
+                .map(Uuid::fromString).filter(uuid -> uuid != Uuid.ZERO_UUID);
+            Boolean useTopicId = inputTopicId.isPresent();
+
+            List<Uuid> topicIds;
+            List<String> topics;
+            if (useTopicId) {
+                topicIds = getTopicIds(inputTopicId.get(), opts.excludeInternalTopics());
+                topics = Collections.emptyList();
+            } else {
+                topicIds = Collections.emptyList();
+                topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+
+            }
+
+            // Only check topic name when topicId is not provided
+            if (useTopicId) {
+                ensureTopicIdExists(topicIds, inputTopicId.get(), !opts.ifExists());
+            } else {
+                ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            }
+            List<org.apache.kafka.clients.admin.TopicDescription> topicDescriptions = new ArrayList<>();
+
+            if (!topicIds.isEmpty()) {
+                Map<Uuid, org.apache.kafka.clients.admin.TopicDescription> descTopics =
+                    adminClient.describeTopics(TopicCollection.ofTopicIds(topicIds)).allTopicIds().get();
+                topicDescriptions = new ArrayList<>(descTopics.values());
+            }
+
+            if (!topics.isEmpty()) {
+                Map<String, org.apache.kafka.clients.admin.TopicDescription> descTopics =
+                    adminClient.describeTopics(TopicCollection.ofTopicNames(topics)).allTopicNames().get();
+                topicDescriptions = new ArrayList<>(descTopics.values());
+            }
+
+            List<String> topicNames = topicDescriptions.stream()
+                .map(org.apache.kafka.clients.admin.TopicDescription::name)
+                .collect(Collectors.toList());
+            Map<ConfigResource, KafkaFuture<Config>> allConfigs = adminClient.describeConfigs(
+                topicNames.stream()
+                    .map(name -> new ConfigResource(ConfigResource.Type.TOPIC, name))
+                    .collect(Collectors.toList())
+            ).values();
+            List<Integer> liveBrokers = adminClient.describeCluster().nodes().get().stream()
+                .map(Node::id)
+                .collect(Collectors.toList());
+            DescribeOptions describeOptions = new DescribeOptions(opts, new HashSet<>(liveBrokers));
+            Set<TopicPartition> topicPartitions = topicDescriptions
+                .stream()
+                .flatMap(td -> td.partitions().stream()
+                    .map(p -> new TopicPartition(td.name(), p.partition())))
+                .collect(Collectors.toSet());
+            Map<TopicPartition, PartitionReassignment> reassignments = listAllReassignments(topicPartitions);
+            for (org.apache.kafka.clients.admin.TopicDescription td : topicDescriptions) {
+                String topicName = td.name();
+                Uuid topicId = td.topicId();
+                Config config = allConfigs.get(new ConfigResource(ConfigResource.Type.TOPIC, topicName)).get();
+                ArrayList<TopicPartitionInfo> sortedPartitions = new ArrayList<>(td.partitions());
+                sortedPartitions.sort(Comparator.comparingInt(TopicPartitionInfo::partition));
+                printDescribeConfig(opts, describeOptions, reassignments, td, topicName, topicId, config, sortedPartitions);
+                printPartitionDescription(describeOptions, reassignments, td, topicName, config, sortedPartitions);
+            }
+        }
+
+        private void printPartitionDescription(DescribeOptions describeOptions, Map<TopicPartition, PartitionReassignment> reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Config config, ArrayList<TopicPartitionInfo> sortedPartitions) {
+            if (describeOptions.describePartitions) {
+                for (TopicPartitionInfo partition : sortedPartitions) {
+                    PartitionReassignment reassignment =
+                        reassignments.get(new TopicPartition(td.name(), partition.partition()));
+                    PartitionDescription partitionDesc = new PartitionDescription(topicName,
+                        partition, config, false, reassignment);
+                    describeOptions.maybePrintPartitionDescription(partitionDesc);
+                }
+            }
+        }
+
+        private void printDescribeConfig(TopicCommandOptions opts, DescribeOptions describeOptions, Map<TopicPartition, PartitionReassignment> reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Uuid topicId, Config config, ArrayList<TopicPartitionInfo> sortedPartitions) {
+            if (describeOptions.describeConfigs) {
+                List<ConfigEntry> entries = new ArrayList<>(config.entries());
+                boolean hasNonDefault = entries.stream().anyMatch(e -> !e.isDefault());
+                if (!opts.reportOverriddenConfigs() || hasNonDefault) {
+                    int numPartitions = td.partitions().size();
+                    TopicPartitionInfo firstPartition = sortedPartitions.get(0);
+                    PartitionReassignment reassignment =
+                        reassignments.get(new TopicPartition(td.name(), firstPartition.partition()));
+                    TopicDescription topicDesc = new TopicDescription(topicName, topicId,
+                        numPartitions, getReplicationFactor(firstPartition, reassignment),
+                        config, false);
+                    topicDesc.printDescription();
+                }
+            }
+        }
+
+        public void deleteTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            adminClient.deleteTopics(Collections.unmodifiableList(topics),
+                new DeleteTopicsOptions().retryOnQuotaViolation(false)
+            ).all().get();
+        }
+
+        public List<String> getTopics(Optional<String> topicIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
+            ListTopicsOptions listTopicsOptions = new ListTopicsOptions();
+            if (!excludeInternalTopics) {
+                listTopicsOptions.listInternal(true);
+            }
+
+            Set<String> allTopics = adminClient.listTopics(listTopicsOptions).names().get();
+            return doGetTopics(allTopics.stream().sorted().collect(Collectors.toList()), topicIncludeList, excludeInternalTopics);
+        }
+
+        public List<Uuid> getTopicIds(Uuid topicIdIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
+            ListTopicsResult allTopics = excludeInternalTopics ? adminClient.listTopics() :
+                adminClient.listTopics(new ListTopicsOptions().listInternal(true));
+            List<Uuid> allTopicIds = null;
+            allTopicIds = allTopics.listings().get().stream()
+                .map(TopicListing::topicId)
+                .sorted()
+                .collect(Collectors.toList());
+            return allTopicIds.contains(topicIdIncludeList) ?
+                Collections.singletonList(topicIdIncludeList) :
+                Collections.emptyList();
+        }
+
+        @Override
+        public void close() throws Exception {
+            adminClient.close();
+        }
+    }
+
+    public static class TopicCommandOptions extends CommandDefaultOptions {
+        private final ArgumentAcceptingOptionSpec<String> bootstrapServerOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> commandConfigOpt;
+
+        private final OptionSpecBuilder listOpt;
+
+        private final OptionSpecBuilder createOpt;
+
+        private final OptionSpecBuilder deleteOpt;
+
+        private final OptionSpecBuilder alterOpt;
+
+        private final OptionSpecBuilder describeOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> topicOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> topicIdOpt;
+
+        private final String nl;
+
+        private static final String KAFKA_CONFIGS_CLI_SUPPORTS_ALTERING_TOPIC_CONFIGS_WITH_A_BOOTSTRAP_SERVER =
+                " (the kafka-configs CLI supports altering topic configs with a --bootstrap-server option)";
+
+        private final ArgumentAcceptingOptionSpec<String> configOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> deleteConfigOpt;
+
+        private final ArgumentAcceptingOptionSpec<Integer> partitionsOpt;
+
+        private final ArgumentAcceptingOptionSpec<Integer> replicationFactorOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> replicaAssignmentOpt;
+
+        private final OptionSpecBuilder reportUnderReplicatedPartitionsOpt;
+
+        private final OptionSpecBuilder reportUnavailablePartitionsOpt;
+
+        private final OptionSpecBuilder reportUnderMinIsrPartitionsOpt;
+
+        private final OptionSpecBuilder reportAtMinIsrPartitionsOpt;
+
+        private final OptionSpecBuilder topicsWithOverridesOpt;
+
+        private final OptionSpecBuilder ifExistsOpt;
+
+        private final OptionSpecBuilder ifNotExistsOpt;
+
+        private final OptionSpecBuilder excludeInternalTopicOpt;
+
+        private final Set<OptionSpec<?>> allTopicLevelOpts;
+
+        private final Set<OptionSpecBuilder> allReplicationReportOpts;
+
+        public TopicCommandOptions(String[] args) {
+            super(args);
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The Kafka server to connect to.")
+                .withRequiredArg()
+                .describedAs("server to connect to")
+                .ofType(String.class);
+            commandConfigOpt = parser.accepts("command-config", "Property file containing configs to be passed to Admin Client. " +
+                    "This is used only with --bootstrap-server option for describing and altering broker configs.")
+                .withRequiredArg()
+                .describedAs("command config property file")
+                .ofType(String.class);
+
+
+            listOpt = parser.accepts("list", "List all available topics.");
+            createOpt = parser.accepts("create", "Create a new topic.");
+            deleteOpt = parser.accepts("delete", "Delete a topic");
+            alterOpt = parser.accepts("alter", "Alter the number of partitions, replica assignment, and/or configuration for the topic.");
+            describeOpt = parser.accepts("describe", "List details for the given topics.");
+            topicOpt = parser.accepts("topic", "Alter the number of partitions and replica assignment. " +
+                            "Update the configuration of an existing topic via --alter is no longer supported here" +

Review Comment:
   This was a mistake during fixing a conflict with trunk



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on a diff in pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1297477323


##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String[] args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {

Review Comment:
   remove it 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1661932121

   I have updated the PR to use #13158 and #13278. The test checks are failing because of an unrelated class. However the `:tools:compileTestJava` is failing because it doesn't like `JavaConverters.asScala` and I can not reproduce this problem locally. Logs are [here](https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-13201/11/pipeline/9#step-59-log-700) any suggestion? 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1596238230

   Hi @divijvaidya I just did rebase this with trunk


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1719406781

   Pipeline is failing for unrealted test cases https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-13201/25/#showFailuresLink 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1716058400

   > > java.lang.NoClassDefFoundError: org/apache/kafka/server/log/remote/storage/RemoteStorageException
   > 
   > Hi @fvaleri, I am getting this error with all branches once I move any tool outside the core. I also noticed that `PlaintextAdminIntegrationTest` gets this error as well and I am not sure why. Details are [here](https://github.com/apache/kafka/pull/13204#issuecomment-1689824516).
   > 
   > This happened when the cluster was trying to initialize `ReplicaFetcherTierStateMachine` while initializing `ReplicaFetcherThread`. I'm not sure what has changed here besides moving the command out of `core`.
   
   I think there's a transitive dependency. I tried to add `implementation project(':storage:api')` which solved the problem with `RemoteStorageException` but it created another issue with `org.apache.kafka.connect.data` and ` org.apache.kafka.connect.storage` in `TestPlugins` in `connect:runtime`. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1755578715

   @mimaison fixed the compilation issue. I spotted another one with Java21 and fixed it as well. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "mimaison (via GitHub)" <gi...@apache.org>.
mimaison commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1357047038


##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1007 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+public abstract class TopicCommand {
+    private static final Logger LOG = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String... args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        LOG.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")
+    static Map<Integer, List<Integer>> parseReplicaAssignment(String replicaAssignmentList) {
+        String[] partitionList = replicaAssignmentList.split(",");
+        Map<Integer, List<Integer>> ret = new LinkedHashMap<Integer, List<Integer>>();

Review Comment:
   We can remove the types on the right



##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1007 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+public abstract class TopicCommand {
+    private static final Logger LOG = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String... args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        LOG.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")

Review Comment:
   Do we need this `@SuppressWarnings`?



##########
checkstyle/import-control.xml:
##########
@@ -286,6 +289,8 @@
     <allow pkg="net.sourceforge.argparse4j" />
     <allow pkg="org.apache.log4j" />
     <allow pkg="kafka.test" />
+    <allow pkg="scala.collection" />
+    <allow pkg="scala.compat" />

Review Comment:
   Do we really need these? Checkstyle seems to pass without these 2 `allow`



##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1007 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+public abstract class TopicCommand {
+    private static final Logger LOG = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String... args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        LOG.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")
+    static Map<Integer, List<Integer>> parseReplicaAssignment(String replicaAssignmentList) {
+        String[] partitionList = replicaAssignmentList.split(",");
+        Map<Integer, List<Integer>> ret = new LinkedHashMap<Integer, List<Integer>>();
+        for (int i = 0; i < partitionList.length; i++) {
+            List<Integer> brokerList = Arrays.stream(partitionList[i].split(":"))
+                .map(String::trim)
+                .mapToInt(Integer::parseInt)
+                .boxed()
+                .collect(Collectors.toList());
+            Collection<Integer> duplicateBrokers = ToolsUtils.duplicates(brokerList);
+            if (!duplicateBrokers.isEmpty()) {
+                throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: " +
+                    duplicateBrokers.stream()
+                        .map(Object::toString)
+                        .collect(Collectors.joining(","))
+                );
+            }
+            ret.put(i, brokerList);
+            if (ret.get(i).size() != ret.get(0).size()) {
+                throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList);
+            }
+        }
+        return ret;
+    }
+
+    @SuppressWarnings("deprecation")
+    private static Properties parseTopicConfigsToBeAdded(TopicCommandOptions opts) {
+        List<List<String>> configsToBeAdded = opts.topicConfig().orElse(Collections.emptyList())
+            .stream()
+            .map(s -> Arrays.asList(s.split("\\s*=\\s*")))
+            .collect(Collectors.toList());
+
+        if (!configsToBeAdded.stream().allMatch(config -> config.size() == 2)) {
+            throw new IllegalArgumentException("requirement failed: Invalid topic config: all configs to be added must be in the format \"key=val\".");
+        }
+
+        Properties props = new Properties();
+        configsToBeAdded.stream()
+            .forEach(pair -> props.setProperty(pair.get(0).trim(), pair.get(1).trim()));
+        LogConfig.validate(props);
+        if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) {
+            System.out.println("WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " +
+                "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " +
+                "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.");
+        }
+        return props;
+    }
+
+    // It is possible for a reassignment to complete between the time we have fetched its state and the time
+    // we fetch partition metadata. In this case, we ignore the reassignment when determining replication factor.
+    public static boolean isReassignmentInProgress(TopicPartitionInfo tpi, PartitionReassignment ra) {
+        // Reassignment is still in progress as long as the removing and adding replicas are still present
+        Set<Integer> allReplicaIds = tpi.replicas().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> changingReplicaIds = new HashSet<>();
+        if (ra != null) {
+            changingReplicaIds.addAll(ra.removingReplicas());
+            changingReplicaIds.addAll(ra.addingReplicas());
+        }
+        return allReplicaIds.stream().anyMatch(changingReplicaIds::contains);
+
+    }
+
+    private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionReassignment reassignment) {
+        return isReassignmentInProgress(tpi, reassignment) ?
+            reassignment.replicas().size() - reassignment.addingReplicas().size() :
+            tpi.replicas().size();
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopics        Topics that were found to match the requested topic name.
+     * @param requestedTopic     Name of the topic that was requested.
+     * @param requireTopicExists Indicates if the topic needs to exist for the operation to be successful.
+     *                           If set to true, the command will throw an exception if the topic with the
+     *                           requested name does not exist.
+     */
+    private static void ensureTopicExists(List<String> foundTopics, String requestedTopic, Boolean requireTopicExists) {
+        // If no topic name was mentioned, do not need to throw exception.
+        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
+            // If given topic doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
+        }
+    }
+
+    private static List<String> doGetTopics(List<String> allTopics, Optional<String> topicIncludeList, Boolean excludeInternalTopics) {
+        if (topicIncludeList.isPresent()) {
+            IncludeList topicsFilter = new IncludeList(topicIncludeList.get());
+            return allTopics.stream()
+                .filter(topic -> topicsFilter.isTopicAllowed(topic, excludeInternalTopics))
+                .collect(Collectors.toList());
+        } else {
+            return allTopics.stream()
+                .filter(topic -> !(Topic.isInternal(topic) && excludeInternalTopics))
+                .collect(Collectors.toList());
+        }
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopicIds        Topics that were found to match the requested topic id.
+     * @param requestedTopicId     Id of the topic that was requested.
+     * @param requireTopicIdExists Indicates if the topic needs to exist for the operation to be successful.
+     *                             If set to true, the command will throw an exception if the topic with the
+     *                             requested id does not exist.
+     */
+    private static void ensureTopicIdExists(List<Uuid> foundTopicIds, Uuid requestedTopicId, Boolean requireTopicIdExists) {
+        // If no topic id was mentioned, do not need to throw exception.
+        if (requestedTopicId != null && requireTopicIdExists && foundTopicIds.isEmpty()) {
+            // If given topicId doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("TopicId '%s' does not exist as expected", requestedTopicId));
+        }
+    }
+
+    static class CommandTopicPartition {
+        private final Optional<String> name;
+        private final Optional<Integer> partitions;
+        private final Optional<Integer> replicationFactor;
+        private final Map<Integer, List<Integer>> replicaAssignment;
+        private final Properties configsToAdd;
+
+        private final TopicCommandOptions opts;
+
+        public CommandTopicPartition(TopicCommandOptions options) {
+            opts = options;
+            name = options.topic();
+            partitions = options.partitions();
+            replicationFactor = options.replicationFactor();
+            replicaAssignment = options.replicaAssignment().orElse(Collections.emptyMap());
+            configsToAdd = parseTopicConfigsToBeAdded(options);
+        }
+
+        public Boolean hasReplicaAssignment() {
+            return !replicaAssignment.isEmpty();
+        }
+
+        public Boolean ifTopicDoesntExist() {
+            return opts.ifNotExists();
+        }
+    }
+
+    static class TopicDescription {
+        private final String topic;
+        private final Uuid topicId;
+        private final Integer numPartitions;
+        private final Integer replicationFactor;
+        private final Config config;
+        private final Boolean markedForDeletion;
+
+        public TopicDescription(String topic, Uuid topicId, Integer numPartitions, Integer replicationFactor, Config config, Boolean markedForDeletion) {
+            this.topic = topic;
+            this.topicId = topicId;
+            this.numPartitions = numPartitions;
+            this.replicationFactor = replicationFactor;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+        }
+
+        public void printDescription() {
+            String configsAsString = config.entries().stream()
+                .filter(config -> !config.isDefault())
+                .map(ce -> ce.name() + "=" + ce.value())
+                .collect(Collectors.joining(","));
+            System.out.print("Topic: " +  topic);
+            if (topicId != Uuid.ZERO_UUID)
+                System.out.print("\tTopicId: " + topicId);
+            System.out.print("\tPartitionCount: " + numPartitions);
+            System.out.print("\tReplicationFactor: " + replicationFactor);
+            System.out.print("\tConfigs: " + configsAsString);
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class PartitionDescription {
+        private final String topic;
+        private final TopicPartitionInfo info;
+        private final Config config;
+        private final Boolean markedForDeletion;
+        private final PartitionReassignment reassignment;
+
+        PartitionDescription(String topic,
+                             TopicPartitionInfo info,
+                             Config config,
+                             Boolean markedForDeletion,
+                             PartitionReassignment reassignment) {
+            this.topic = topic;
+            this.info = info;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+            this.reassignment = reassignment;
+        }
+
+        public Integer minIsrCount() {
+            return Integer.parseInt(config.get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value());
+        }
+
+        public Boolean isUnderReplicated() {
+            return getReplicationFactor(info, reassignment) - info.isr().size() > 0;
+        }
+
+        public boolean hasLeader() {
+            return info.leader() != null;
+        }
+
+        public Boolean isUnderMinIsr() {
+            return !hasLeader() ||  info.isr().size() < minIsrCount();
+        }
+
+        public Boolean isAtMinIsrPartitions() {
+            return minIsrCount() == info.isr().size();
+        }
+
+        public Boolean hasUnavailablePartitions(Set<Integer> liveBrokers) {
+            return !hasLeader() || !liveBrokers.contains(info.leader().id());
+        }
+
+        public void printDescription() {
+            System.out.print("\tTopic: " + topic);
+            System.out.print("\tPartition: " + info.partition());
+            System.out.print("\tLeader: " + (hasLeader() ? info.leader().id() : "none"));
+            System.out.print("\tReplicas: " + info.replicas().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            System.out.print("\tIsr: " + info.isr().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            if (reassignment != null) {
+                System.out.print("\tAdding Replicas: " + reassignment.addingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+                System.out.print("\tRemoving Replicas: " + reassignment.removingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+            }
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class DescribeOptions {
+        private final TopicCommandOptions opts;
+        private final Set<Integer> liveBrokers;
+        private final boolean describeConfigs;
+        private final boolean describePartitions;
+
+        public DescribeOptions(TopicCommandOptions opts, Set<Integer> liveBrokers) {
+            this.opts = opts;
+            this.liveBrokers = liveBrokers;
+            this.describeConfigs = !opts.reportUnavailablePartitions() &&
+                !opts.reportUnderReplicatedPartitions() &&
+                !opts.reportUnderMinIsrPartitions() &&
+                !opts.reportAtMinIsrPartitions();
+            this.describePartitions = !opts.reportOverriddenConfigs();
+        }
+
+        private boolean shouldPrintUnderReplicatedPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderReplicatedPartitions() && partitionDescription.isUnderReplicated();
+        }
+
+        private boolean shouldPrintUnavailablePartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnavailablePartitions() && partitionDescription.hasUnavailablePartitions(liveBrokers);
+        }
+
+        private boolean shouldPrintUnderMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderMinIsrPartitions() && partitionDescription.isUnderMinIsr();
+        }
+
+        private boolean shouldPrintAtMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportAtMinIsrPartitions() && partitionDescription.isAtMinIsrPartitions();
+        }
+
+        private boolean shouldPrintTopicPartition(PartitionDescription partitionDesc) {
+            return describeConfigs ||
+                shouldPrintUnderReplicatedPartitions(partitionDesc) ||
+                shouldPrintUnavailablePartitions(partitionDesc) ||
+                shouldPrintUnderMinIsrPartitions(partitionDesc) ||
+                shouldPrintAtMinIsrPartitions(partitionDesc);
+        }
+
+        public void maybePrintPartitionDescription(PartitionDescription desc) {
+            if (shouldPrintTopicPartition(desc)) {
+                desc.printDescription();
+            }
+        }
+    }
+
+    public static class TopicService implements AutoCloseable {
+        private Admin adminClient;
+
+        public TopicService(Properties commandConfig, Optional<String> bootstrapServer) {
+            this.adminClient = createAdminClient(commandConfig, bootstrapServer);
+        }
+
+        public TopicService(Admin admin) {
+            this.adminClient = admin;
+        }
+
+        private static Admin createAdminClient(Properties commandConfig, Optional<String> bootstrapServer) {
+            if (bootstrapServer.isPresent()) {
+                commandConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer.get());
+            }
+            return Admin.create(commandConfig);
+        }
+
+        public void createTopic(TopicCommandOptions opts) throws Exception {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            if (Topic.hasCollisionChars(topic.name.get())) {
+                System.out.println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could " +
+                    "collide. To avoid issues it is best to use either, but not both.");
+            }
+            createTopic(topic);
+        }
+
+        public void createTopic(CommandTopicPartition topic) throws Exception {
+            if (topic.replicationFactor.filter(rf -> rf > Short.MAX_VALUE || rf < 1).isPresent()) {
+                throw new IllegalArgumentException("The replication factor must be between 1 and " + Short.MAX_VALUE + " inclusive");
+            }
+            if (topic.partitions.filter(p -> p < 1).isPresent()) {
+                throw new IllegalArgumentException("The partitions must be greater than 0");
+            }
+
+            try {
+                NewTopic newTopic;
+                if (topic.hasReplicaAssignment()) {
+                    newTopic = new NewTopic(topic.name.get(), topic.replicaAssignment);
+                } else {
+                    newTopic = new NewTopic(topic.name.get(), topic.partitions, topic.replicationFactor.map(Integer::shortValue));
+                }
+
+                Map<String, String> configsMap = topic.configsToAdd.stringPropertyNames().stream()
+                    .collect(Collectors.toMap(name -> name, name -> topic.configsToAdd.getProperty(name)));
+
+                newTopic.configs(configsMap);
+                CreateTopicsResult createResult = adminClient.createTopics(Collections.singleton(newTopic),
+                    new CreateTopicsOptions().retryOnQuotaViolation(false));
+                createResult.all().get();
+                System.out.println("Created topic " + topic.name + ".");
+            } catch (ExecutionException e) {
+                if (e.getCause() == null) {
+                    throw e;
+                }
+                if (!(e.getCause() instanceof TopicExistsException && topic.ifTopicDoesntExist())) {
+                    throw (Exception) e.getCause();
+                }
+            }
+        }
+
+        public void listTopics(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            String results = getTopics(opts.topic(), opts.excludeInternalTopics())
+                .stream()
+                .collect(Collectors.joining("\n"));
+            System.out.println(results);
+        }
+
+        public void alterTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+
+            if (!topics.isEmpty()) {
+                Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo = adminClient.describeTopics(topics).topicNameValues();
+                Map<String, NewPartitions> newPartitions = topics.stream()
+                    .map(topicName -> topicNewPartitions(topic, topicsInfo, topicName))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                adminClient.createPartitions(newPartitions, new CreatePartitionsOptions().retryOnQuotaViolation(false)).all().get();
+            }
+        }
+
+        private AbstractMap.SimpleEntry<String, NewPartitions> topicNewPartitions(
+            CommandTopicPartition topic,
+            Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo,
+            String topicName) {
+            if (topic.hasReplicaAssignment()) {
+                try {
+                    Integer startPartitionId = topicsInfo.get(topicName).get().partitions().size();
+                    Map<Integer, List<Integer>> replicaMap = topic.replicaAssignment.entrySet().stream()
+                        .skip(startPartitionId)
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                    List<List<Integer>> newAssignment = replicaMap.entrySet().stream()
+                        .map(Map.Entry::getValue)
+                        .collect(Collectors.toList());
+                    return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get(), newAssignment));
+                } catch (InterruptedException | ExecutionException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+            return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get()));
+        }
+
+        public Map<TopicPartition, PartitionReassignment> listAllReassignments(Set<TopicPartition> topicPartitions) {
+            try {
+                return adminClient.listPartitionReassignments(topicPartitions).reassignments().get();
+            } catch (ExecutionException e) {
+                Throwable cause = e.getCause();
+                if (cause instanceof UnsupportedVersionException || cause instanceof ClusterAuthorizationException) {
+                    LOG.debug("Couldn't query reassignments through the AdminClient API: " + cause.getMessage(), cause);
+                    return Collections.emptyMap();
+                } else {
+                    throw new RuntimeException(e);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        public void describeTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            // If topicId is provided and not zero, will use topicId regardless of topic name
+            Optional<Uuid> inputTopicId = opts.topicId()
+                .map(Uuid::fromString).filter(uuid -> uuid != Uuid.ZERO_UUID);
+            Boolean useTopicId = inputTopicId.isPresent();
+
+            List<Uuid> topicIds;
+            List<String> topics;
+            if (useTopicId) {
+                topicIds = getTopicIds(inputTopicId.get(), opts.excludeInternalTopics());
+                topics = Collections.emptyList();
+            } else {
+                topicIds = Collections.emptyList();
+                topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            }
+
+            // Only check topic name when topicId is not provided
+            if (useTopicId) {
+                ensureTopicIdExists(topicIds, inputTopicId.get(), !opts.ifExists());
+            } else {
+                ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            }
+            List<org.apache.kafka.clients.admin.TopicDescription> topicDescriptions = new ArrayList<>();
+
+            if (!topicIds.isEmpty()) {
+                Map<Uuid, org.apache.kafka.clients.admin.TopicDescription> descTopics =
+                    adminClient.describeTopics(TopicCollection.ofTopicIds(topicIds)).allTopicIds().get();
+                topicDescriptions = new ArrayList<>(descTopics.values());
+            }
+
+            if (!topics.isEmpty()) {
+                Map<String, org.apache.kafka.clients.admin.TopicDescription> descTopics =
+                    adminClient.describeTopics(TopicCollection.ofTopicNames(topics)).allTopicNames().get();
+                topicDescriptions = new ArrayList<>(descTopics.values());
+            }
+
+            List<String> topicNames = topicDescriptions.stream()
+                .map(org.apache.kafka.clients.admin.TopicDescription::name)
+                .collect(Collectors.toList());
+            Map<ConfigResource, KafkaFuture<Config>> allConfigs = adminClient.describeConfigs(
+                topicNames.stream()
+                    .map(name -> new ConfigResource(ConfigResource.Type.TOPIC, name))
+                    .collect(Collectors.toList())
+            ).values();
+            List<Integer> liveBrokers = adminClient.describeCluster().nodes().get().stream()
+                .map(Node::id)
+                .collect(Collectors.toList());
+            DescribeOptions describeOptions = new DescribeOptions(opts, new HashSet<>(liveBrokers));
+            Set<TopicPartition> topicPartitions = topicDescriptions
+                .stream()
+                .flatMap(td -> td.partitions().stream()
+                    .map(p -> new TopicPartition(td.name(), p.partition())))
+                .collect(Collectors.toSet());
+            Map<TopicPartition, PartitionReassignment> reassignments = listAllReassignments(topicPartitions);
+            for (org.apache.kafka.clients.admin.TopicDescription td : topicDescriptions) {
+                String topicName = td.name();
+                Uuid topicId = td.topicId();
+                Config config = allConfigs.get(new ConfigResource(ConfigResource.Type.TOPIC, topicName)).get();
+                ArrayList<TopicPartitionInfo> sortedPartitions = new ArrayList<>(td.partitions());
+                sortedPartitions.sort(Comparator.comparingInt(TopicPartitionInfo::partition));
+                printDescribeConfig(opts, describeOptions, reassignments, td, topicName, topicId, config, sortedPartitions);
+                printPartitionDescription(describeOptions, reassignments, td, topicName, config, sortedPartitions);
+            }
+        }
+
+        private void printPartitionDescription(DescribeOptions describeOptions, Map<TopicPartition, PartitionReassignment> reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Config config, ArrayList<TopicPartitionInfo> sortedPartitions) {
+            if (describeOptions.describePartitions) {
+                for (TopicPartitionInfo partition : sortedPartitions) {
+                    PartitionReassignment reassignment =
+                        reassignments.get(new TopicPartition(td.name(), partition.partition()));
+                    PartitionDescription partitionDesc = new PartitionDescription(topicName,
+                        partition, config, false, reassignment);
+                    describeOptions.maybePrintPartitionDescription(partitionDesc);
+                }
+            }
+        }
+
+        private void printDescribeConfig(TopicCommandOptions opts, DescribeOptions describeOptions, Map<TopicPartition, PartitionReassignment> reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Uuid topicId, Config config, ArrayList<TopicPartitionInfo> sortedPartitions) {
+            if (describeOptions.describeConfigs) {
+                List<ConfigEntry> entries = new ArrayList<>(config.entries());
+                boolean hasNonDefault = entries.stream().anyMatch(e -> !e.isDefault());
+                if (!opts.reportOverriddenConfigs() || hasNonDefault) {
+                    int numPartitions = td.partitions().size();
+                    TopicPartitionInfo firstPartition = sortedPartitions.get(0);
+                    PartitionReassignment reassignment =
+                        reassignments.get(new TopicPartition(td.name(), firstPartition.partition()));
+                    TopicDescription topicDesc = new TopicDescription(topicName, topicId,
+                        numPartitions, getReplicationFactor(firstPartition, reassignment),
+                        config, false);
+                    topicDesc.printDescription();
+                }
+            }
+        }
+
+        public void deleteTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            adminClient.deleteTopics(Collections.unmodifiableList(topics),
+                new DeleteTopicsOptions().retryOnQuotaViolation(false)
+            ).all().get();
+        }
+
+        public List<String> getTopics(Optional<String> topicIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
+            ListTopicsOptions listTopicsOptions = new ListTopicsOptions();
+            if (!excludeInternalTopics) {
+                listTopicsOptions.listInternal(true);
+            }
+
+            Set<String> allTopics = adminClient.listTopics(listTopicsOptions).names().get();
+            return doGetTopics(allTopics.stream().sorted().collect(Collectors.toList()), topicIncludeList, excludeInternalTopics);
+        }
+
+        public List<Uuid> getTopicIds(Uuid topicIdIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
+            ListTopicsResult allTopics = excludeInternalTopics ? adminClient.listTopics() :
+                adminClient.listTopics(new ListTopicsOptions().listInternal(true));
+            List<Uuid> allTopicIds = null;

Review Comment:
   We can remove `= null;` as it's the default value



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java:
##########
@@ -0,0 +1,1050 @@
+/*
+ * 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;
+
+import kafka.admin.RackAwareTest;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Tag("integration")
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
+public class TopicCommandIntegrationTest extends kafka.integration.KafkaServerTestHarness implements Logging, RackAwareTest {
+    private Short defaultReplicationFactor = 1;
+    private Integer numPartitions = 1;
+    private TopicCommand.TopicService topicService;
+    private Admin adminClient;
+    private String bootstrapServer;
+    private String testTopicName;
+    private Integer defaultTimeout = 10000;
+
+    /**
+     * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every
+     * test and should not reuse previous configurations unless they select their ports randomly when servers are started.
+     *
+     * Note the replica fetch max bytes is set to `1` in order to throttle the rate of replication for test
+     * `testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress`.
+     */
+    @Override
+    public scala.collection.Seq<KafkaConfig> generateConfigs() {
+        Map<Integer, String> rackInfo = new HashMap<>();
+        rackInfo.put(0, "rack1");
+        rackInfo.put(1, "rack2");
+        rackInfo.put(2, "rack2");
+        rackInfo.put(3, "rack1");
+        rackInfo.put(4, "rack3");
+        rackInfo.put(5, "rack3");
+
+        List<Properties> brokerConfigs = ToolsTestUtils
+            .createBrokerProperties(6, zkConnectOrNull(), rackInfo, numPartitions, defaultReplicationFactor);
+
+        List<KafkaConfig> configs = new ArrayList<>();
+        for (Properties props : brokerConfigs) {
+            props.put(KafkaConfig.ReplicaFetchMaxBytesProp(), "1");
+            configs.add(KafkaConfig.fromProps(props));
+        }
+        return JavaConverters.asScalaBuffer(configs).toSeq();
+    }
+
+    private TopicCommand.TopicCommandOptions buildTopicCommandOptionsWithBootstrap(String... opts) {
+        String[] finalOptions = Stream.concat(Arrays.asList(opts).stream(),
+                Arrays.asList("--bootstrap-server", bootstrapServer).stream()
+        ).toArray(String[]::new);
+        return new TopicCommand.TopicCommandOptions(finalOptions);
+    }
+
+    private void createAndWaitTopic(TopicCommand.TopicCommandOptions opts) throws Exception {
+        topicService.createTopic(opts);
+        waitForTopicCreated(opts.topic().get());
+    }
+
+    private void waitForTopicCreated(String topicName) {
+        waitForTopicCreated(topicName, defaultTimeout);
+    }
+
+    private void waitForTopicCreated(String topicName, Integer timeout) {
+        TestUtils.waitForPartitionMetadata(brokers(), topicName, 0, timeout);
+    }
+
+    @BeforeEach
+    public void setUp(TestInfo info) {
+        super.setUp(info);
+        // create adminClient
+        Properties props = new Properties();
+        bootstrapServer = bootstrapServers(listenerName());
+        props.put(org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+        adminClient = Admin.create(props);
+        topicService = new TopicCommand.TopicService(props, Optional.of(bootstrapServer));
+        testTopicName = String.format("%s-%s", info.getTestMethod().get().getName(),
+            new scala.util.Random().alphanumeric().take(10).mkString());
+    }
+
+    @AfterEach
+    public void close() throws Exception {
+        if (topicService != null)
+            topicService.close();
+        if (adminClient != null)
+            adminClient.close();
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreate(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", "2", "--replication-factor", "1", "--topic", testTopicName));
+
+        adminClient.listTopics().names().get().contains(testTopicName);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithDefaults(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName));
+
+        List<TopicPartitionInfo> partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+        assertEquals(partitions.size(), numPartitions);

Review Comment:
   Could we swap the values? The first field is the expected value. Same in a few other tests below



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java:
##########
@@ -0,0 +1,1050 @@
+/*
+ * 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;
+
+import kafka.admin.RackAwareTest;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Tag("integration")
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
+public class TopicCommandIntegrationTest extends kafka.integration.KafkaServerTestHarness implements Logging, RackAwareTest {
+    private Short defaultReplicationFactor = 1;
+    private Integer numPartitions = 1;
+    private TopicCommand.TopicService topicService;
+    private Admin adminClient;
+    private String bootstrapServer;
+    private String testTopicName;
+    private Integer defaultTimeout = 10000;
+
+    /**
+     * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every
+     * test and should not reuse previous configurations unless they select their ports randomly when servers are started.
+     *
+     * Note the replica fetch max bytes is set to `1` in order to throttle the rate of replication for test
+     * `testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress`.
+     */
+    @Override
+    public scala.collection.Seq<KafkaConfig> generateConfigs() {
+        Map<Integer, String> rackInfo = new HashMap<>();
+        rackInfo.put(0, "rack1");
+        rackInfo.put(1, "rack2");
+        rackInfo.put(2, "rack2");
+        rackInfo.put(3, "rack1");
+        rackInfo.put(4, "rack3");
+        rackInfo.put(5, "rack3");
+
+        List<Properties> brokerConfigs = ToolsTestUtils
+            .createBrokerProperties(6, zkConnectOrNull(), rackInfo, numPartitions, defaultReplicationFactor);
+
+        List<KafkaConfig> configs = new ArrayList<>();
+        for (Properties props : brokerConfigs) {
+            props.put(KafkaConfig.ReplicaFetchMaxBytesProp(), "1");
+            configs.add(KafkaConfig.fromProps(props));
+        }
+        return JavaConverters.asScalaBuffer(configs).toSeq();
+    }
+
+    private TopicCommand.TopicCommandOptions buildTopicCommandOptionsWithBootstrap(String... opts) {
+        String[] finalOptions = Stream.concat(Arrays.asList(opts).stream(),
+                Arrays.asList("--bootstrap-server", bootstrapServer).stream()
+        ).toArray(String[]::new);
+        return new TopicCommand.TopicCommandOptions(finalOptions);
+    }
+
+    private void createAndWaitTopic(TopicCommand.TopicCommandOptions opts) throws Exception {
+        topicService.createTopic(opts);
+        waitForTopicCreated(opts.topic().get());
+    }
+
+    private void waitForTopicCreated(String topicName) {
+        waitForTopicCreated(topicName, defaultTimeout);
+    }
+
+    private void waitForTopicCreated(String topicName, Integer timeout) {
+        TestUtils.waitForPartitionMetadata(brokers(), topicName, 0, timeout);
+    }
+
+    @BeforeEach
+    public void setUp(TestInfo info) {
+        super.setUp(info);
+        // create adminClient
+        Properties props = new Properties();
+        bootstrapServer = bootstrapServers(listenerName());
+        props.put(org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);

Review Comment:
   I guess we could import `org.apache.kafka.clients.CommonClientConfigs`



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import kafka.utils.Exit;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientTestUtils;
+import org.apache.kafka.clients.admin.CreatePartitionsResult;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.DeleteTopicsResult;
+import org.apache.kafka.clients.admin.DescribeTopicsResult;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.errors.ThrottlingQuotaExceededException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyCollection;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.verify;
+
+@Timeout(value = 60)
+public class TopicCommandTest {
+    private String bootstrapServer = "localhost:9092";
+    private String topicName = "topicName";
+
+    @Test
+    public void testIsNotUnderReplicatedWhenAdding() {
+        List<Integer> replicaIds = Arrays.asList(1, 2);
+        List<Node> replicas = new ArrayList<>();
+        for (int id : replicaIds) {
+            replicas.add(new Node(id, "localhost", 9090 + id));
+        }
+
+        TopicCommand.PartitionDescription partitionDescription = new TopicCommand.PartitionDescription("test-topic",
+            new TopicPartitionInfo(0, new Node(1, "localhost", 9091), replicas,
+                Collections.singletonList(new Node(1, "localhost", 9091))),
+            null, false,
+                new PartitionReassignment(replicaIds, Arrays.asList(2), Collections.emptyList())
+        );
+
+        assertFalse(partitionDescription.isUnderReplicated());
+    }
+
+    @Test
+    public void testAlterWithUnspecifiedPartitionCount() {
+        String[] options = new String[] {" --bootstrap-server", bootstrapServer, "--alter", "--topic", topicName};
+        assertInitializeInvalidOptionsExitCode(1, options);
+    }
+
+    @Test
+    public void testConfigOptWithBootstrapServers() {
+        assertInitializeInvalidOptionsExitCode(1,
+            new String[] {"--bootstrap-server", bootstrapServer, "--alter", "--topic", topicName,
+                "--partitions", "3", "--config", "cleanup.policy=compact"});
+        assertInitializeInvalidOptionsExitCode(1,
+            new String[] {"--bootstrap-server", bootstrapServer, "--alter", "--topic", topicName,
+                "--partitions", "3", "--delete-config", "cleanup.policy"});
+        TopicCommand.TopicCommandOptions opts =
+            new TopicCommand.TopicCommandOptions(
+                new String[] {"--bootstrap-server", bootstrapServer, "--create", "--topic", topicName, "--partitions", "3",
+                    "--replication-factor", "3", "--config", "cleanup.policy=compact"});
+        assertTrue(opts.hasCreateOption());
+        assertEquals(bootstrapServer, opts.bootstrapServer().get());
+        assertEquals("cleanup.policy=compact", opts.topicConfig().get().get(0));
+    }
+
+    @Test
+    public void testCreateWithPartitionCountWithoutReplicationFactorShouldSucceed() {
+        TopicCommand.TopicCommandOptions opts = new TopicCommand.TopicCommandOptions(
+            new String[] {"--bootstrap-server", bootstrapServer,
+                "--create",
+                "--partitions", "2",
+                "--topic", topicName});
+        assertTrue(opts.hasCreateOption());
+        assertEquals(topicName, opts.topic().get());
+        assertEquals(2, opts.partitions().get());
+    }
+
+    @Test
+    public void testCreateWithReplicationFactorWithoutPartitionCountShouldSucceed() {
+        TopicCommand.TopicCommandOptions opts = new TopicCommand.TopicCommandOptions(
+            new String[] {"--bootstrap-server", bootstrapServer,
+                "--create",
+                "--replication-factor", "3",
+                "--topic", topicName});
+        assertTrue(opts.hasCreateOption());
+        assertEquals(topicName, opts.topic().get());
+        assertEquals(3, opts.replicationFactor().get());

Review Comment:
   The original test called opts.checkArgs(), should we do that here to?



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java:
##########
@@ -0,0 +1,1050 @@
+/*
+ * 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;
+
+import kafka.admin.RackAwareTest;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Tag("integration")
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
+public class TopicCommandIntegrationTest extends kafka.integration.KafkaServerTestHarness implements Logging, RackAwareTest {
+    private Short defaultReplicationFactor = 1;
+    private Integer numPartitions = 1;
+    private TopicCommand.TopicService topicService;
+    private Admin adminClient;
+    private String bootstrapServer;
+    private String testTopicName;
+    private Integer defaultTimeout = 10000;
+
+    /**
+     * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every
+     * test and should not reuse previous configurations unless they select their ports randomly when servers are started.
+     *
+     * Note the replica fetch max bytes is set to `1` in order to throttle the rate of replication for test
+     * `testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress`.
+     */
+    @Override
+    public scala.collection.Seq<KafkaConfig> generateConfigs() {
+        Map<Integer, String> rackInfo = new HashMap<>();
+        rackInfo.put(0, "rack1");
+        rackInfo.put(1, "rack2");
+        rackInfo.put(2, "rack2");
+        rackInfo.put(3, "rack1");
+        rackInfo.put(4, "rack3");
+        rackInfo.put(5, "rack3");
+
+        List<Properties> brokerConfigs = ToolsTestUtils
+            .createBrokerProperties(6, zkConnectOrNull(), rackInfo, numPartitions, defaultReplicationFactor);
+
+        List<KafkaConfig> configs = new ArrayList<>();
+        for (Properties props : brokerConfigs) {
+            props.put(KafkaConfig.ReplicaFetchMaxBytesProp(), "1");
+            configs.add(KafkaConfig.fromProps(props));
+        }
+        return JavaConverters.asScalaBuffer(configs).toSeq();
+    }
+
+    private TopicCommand.TopicCommandOptions buildTopicCommandOptionsWithBootstrap(String... opts) {
+        String[] finalOptions = Stream.concat(Arrays.asList(opts).stream(),
+                Arrays.asList("--bootstrap-server", bootstrapServer).stream()
+        ).toArray(String[]::new);
+        return new TopicCommand.TopicCommandOptions(finalOptions);
+    }
+
+    private void createAndWaitTopic(TopicCommand.TopicCommandOptions opts) throws Exception {
+        topicService.createTopic(opts);
+        waitForTopicCreated(opts.topic().get());
+    }
+
+    private void waitForTopicCreated(String topicName) {
+        waitForTopicCreated(topicName, defaultTimeout);
+    }
+
+    private void waitForTopicCreated(String topicName, Integer timeout) {
+        TestUtils.waitForPartitionMetadata(brokers(), topicName, 0, timeout);
+    }
+
+    @BeforeEach
+    public void setUp(TestInfo info) {
+        super.setUp(info);
+        // create adminClient
+        Properties props = new Properties();
+        bootstrapServer = bootstrapServers(listenerName());
+        props.put(org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+        adminClient = Admin.create(props);
+        topicService = new TopicCommand.TopicService(props, Optional.of(bootstrapServer));
+        testTopicName = String.format("%s-%s", info.getTestMethod().get().getName(),
+            new scala.util.Random().alphanumeric().take(10).mkString());
+    }
+
+    @AfterEach
+    public void close() throws Exception {
+        if (topicService != null)
+            topicService.close();
+        if (adminClient != null)
+            adminClient.close();
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreate(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", "2", "--replication-factor", "1", "--topic", testTopicName));
+
+        adminClient.listTopics().names().get().contains(testTopicName);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithDefaults(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName));
+
+        List<TopicPartitionInfo> partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+        assertEquals(partitions.size(), numPartitions);
+        assertEquals((short) partitions.get(0).replicas().size(), defaultReplicationFactor);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithDefaultReplication(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName, "--partitions", "2"));
+
+        List<TopicPartitionInfo>  partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+        assertEquals(partitions.size(), 2);
+        assertEquals((short) partitions.get(0).replicas().size(), defaultReplicationFactor);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithDefaultPartitions(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName, "--replication-factor", "2"));
+
+        List<TopicPartitionInfo> partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+
+        assertEquals(partitions.size(), numPartitions);
+        assertEquals((short) partitions.get(0).replicas().size(), 2);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithConfigs(String quorum) throws Exception {
+        ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName);
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "2", "--replication-factor", "2", "--topic", testTopicName, "--config",
+                "delete.retention.ms=1000"));
+
+        Config configs = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource);
+        assertEquals(1000, Integer.valueOf(configs.get("delete.retention.ms").value()));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWhenAlreadyExists(String quorum) throws Exception {
+        int numPartitions = 1;
+
+        // create the topic
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", Integer.toString(numPartitions), "--replication-factor", "1",
+                "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+
+        // try to re-create the topic
+        assertThrows(TopicExistsException.class, () -> topicService.createTopic(createOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWhenAlreadyExistsWithIfNotExists(String quorum) throws Exception {
+        TopicCommand.TopicCommandOptions createOpts =
+                buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName, "--if-not-exists");
+        createAndWaitTopic(createOpts);
+        topicService.createTopic(createOpts);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithReplicaAssignment(String quorum) throws Exception {
+        // create the topic
+        TopicCommand.TopicCommandOptions createOpts =
+                buildTopicCommandOptionsWithBootstrap("--create", "--replica-assignment", "5:4,3:2,1:0", "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+
+        List<TopicPartitionInfo> partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+        
+        assertEquals(3, partitions.size());
+        assertEquals(Arrays.asList(5, 4), getPartitionReplicas(partitions, 0));
+        assertEquals(Arrays.asList(3, 2), getPartitionReplicas(partitions, 1));
+        assertEquals(Arrays.asList(1, 0), getPartitionReplicas(partitions, 2));
+    }
+
+    private List<Integer> getPartitionReplicas(List<TopicPartitionInfo> partitions, int partitionNumber) {
+        return partitions.get(partitionNumber).replicas().stream().map(Node::id).collect(Collectors.toList());
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithInvalidReplicationFactor(String quorum) {
+        TopicCommand.TopicCommandOptions opts = buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "2", "--replication-factor", Integer.toString(Short.MAX_VALUE + 1),
+            "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.createTopic(opts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithNegativeReplicationFactor(String quorum) {
+        TopicCommand.TopicCommandOptions opts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "2", "--replication-factor", "-1", "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.createTopic(opts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithNegativePartitionCount(String quorum) {
+        TopicCommand.TopicCommandOptions opts = buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "-1", "--replication-factor", "1", "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.createTopic(opts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testInvalidTopicLevelConfig(String quorum) {
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "1", "--replication-factor", "1", "--topic", testTopicName,
+            "--config", "message.timestamp.type=boom");
+        assertThrows(ConfigException.class, () -> topicService.createTopic(createOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListTopics(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", "1", "--replication-factor", "1", "--topic", testTopicName));
+
+        String output = captureListTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--list"));
+        assertTrue(output.contains(testTopicName));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListTopicsWithIncludeList(String quorum) throws ExecutionException, InterruptedException {
+        String topic1 = "kafka.testTopic1";
+        String topic2 = "kafka.testTopic2";
+        String topic3 = "oooof.testTopic1";
+        adminClient.createTopics(
+                Arrays.asList(new NewTopic(topic1, 2, (short) 2),
+                    new NewTopic(topic2, 2, (short) 2),
+                    new NewTopic(topic3, 2, (short) 2)))
+            .all().get();
+        waitForTopicCreated(topic1);
+        waitForTopicCreated(topic2);
+        waitForTopicCreated(topic3);
+
+        String output = captureListTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--list", "--topic", "kafka.*"));
+
+        assertTrue(output.contains(topic1));
+        assertTrue(output.contains(topic2));
+        assertFalse(output.contains(topic3));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListTopicsWithExcludeInternal(String quorum) throws ExecutionException, InterruptedException {
+        String topic1 = "kafka.testTopic1";
+        adminClient.createTopics(
+                Arrays.asList(new NewTopic(topic1, 2, (short) 2),
+                    new NewTopic(Topic.GROUP_METADATA_TOPIC_NAME, 2, (short) 2)))
+            .all().get();
+        waitForTopicCreated(topic1);
+
+        String output = captureListTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--list", "--exclude-internal"));
+
+        assertTrue(output.contains(topic1));
+        assertFalse(output.contains(Topic.GROUP_METADATA_TOPIC_NAME));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterPartitionCount(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Arrays.asList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter", "--topic", testTopicName, "--partitions", "3"));
+
+        kafka.utils.TestUtils.waitUntilTrue(
+            () -> brokers().forall(b -> b.metadataCache().getTopicPartitions(testTopicName).size() == 3),
+            () -> "Timeout waiting for new assignment propagating to broker", org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+        TopicDescription topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get();
+        assertTrue(topicDescription.partitions().size() == 3);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterAssignment(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter",
+            "--topic", testTopicName, "--replica-assignment", "5:3,3:1,4:2", "--partitions", "3"));
+        kafka.utils.TestUtils.waitUntilTrue(
+            () -> brokers().forall(b -> b.metadataCache().getTopicPartitions(testTopicName).size() == 3),
+            () -> "Timeout waiting for new assignment propagating to broker",
+            org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+
+        TopicDescription topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get();
+        assertTrue(topicDescription.partitions().size() == 3);
+        List<Integer> partitionReplicas = getPartitionReplicas(topicDescription.partitions(), 2);
+        assertEquals(Arrays.asList(4, 2), partitionReplicas);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterAssignmentWithMoreAssignmentThanPartitions(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Arrays.asList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        assertThrows(ExecutionException.class,
+            () -> topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter",
+                "--topic", testTopicName, "--replica-assignment", "5:3,3:1,4:2,3:2", "--partitions", "3")));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterAssignmentWithMorePartitionsThanAssignment(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Arrays.asList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        assertThrows(ExecutionException.class,
+            () -> topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter", "--topic", testTopicName,
+                "--replica-assignment", "5:3,3:1,4:2", "--partitions", "6")));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterWithInvalidPartitionCount(String quorum) throws Exception {
+        createAndWaitTopic(
+                buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "1", "--replication-factor", "1", "--topic", testTopicName)
+        );
+
+        assertThrows(ExecutionException.class,
+            () -> topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter", "--partitions", "-1", "--topic", testTopicName)));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterWhenTopicDoesntExist(String quorum) {
+        // alter a topic that does not exist without --if-exists
+        TopicCommand.TopicCommandOptions alterOpts = buildTopicCommandOptionsWithBootstrap("--alter", "--topic", testTopicName, "--partitions", "1");
+        TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient);
+        assertThrows(IllegalArgumentException.class, () -> topicService.alterTopic(alterOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterWhenTopicDoesntExistWithIfExists(String quorum) throws ExecutionException, InterruptedException {
+        topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter", "--topic", testTopicName, "--partitions", "1", "--if-exists"));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateAlterTopicWithRackAware(String quorum) throws Exception {
+        Map<Integer, String> rackInfo = new HashMap<Integer, String>();
+        rackInfo.put(0, "rack1");
+        rackInfo.put(1, "rack2");
+        rackInfo.put(2, "rack2");
+        rackInfo.put(3, "rack1");
+        rackInfo.put(4, "rack3");
+        rackInfo.put(5, "rack3");
+
+        int numPartitions = 18;
+        int replicationFactor = 3;
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", Integer.toString(numPartitions),
+            "--replication-factor", Integer.toString(replicationFactor),
+            "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+
+        Map<Integer, List<Integer>> assignment = adminClient.describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames().get().get(testTopicName).partitions()
+            .stream()
+            .collect(Collectors.toMap(
+                info -> info.partition(),
+                info -> info.replicas().stream().map(Node::id).collect(Collectors.toList())));
+        checkReplicaDistribution(assignment, rackInfo, rackInfo.size(), numPartitions,
+            replicationFactor, true, true, true);
+
+        int alteredNumPartitions = 36;
+        // verify that adding partitions will also be rack aware
+        TopicCommand.TopicCommandOptions alterOpts = buildTopicCommandOptionsWithBootstrap("--alter",
+            "--partitions", Integer.toString(alteredNumPartitions),
+            "--topic", testTopicName);
+        topicService.alterTopic(alterOpts);
+
+        kafka.utils.TestUtils.waitUntilTrue(
+            () -> brokers().forall(p -> p.metadataCache().getTopicPartitions(testTopicName).size() == alteredNumPartitions),
+            () -> "Timeout waiting for new assignment propagating to broker",
+            org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+
+        assignment = adminClient.describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames().get().get(testTopicName).partitions().stream()
+            .collect(Collectors.toMap(info -> info.partition(), info -> info.replicas().stream().map(Node::id).collect(Collectors.toList())));
+        checkReplicaDistribution(assignment, rackInfo, rackInfo.size(), alteredNumPartitions, replicationFactor,
+            true, true, true);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testConfigPreservationAcrossPartitionAlteration(String quorum) throws Exception {
+        int numPartitionsOriginal = 1;
+        String cleanupKey = "cleanup.policy";
+        String cleanupVal = "compact";
+
+        // create the topic
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", Integer.toString(numPartitionsOriginal),
+            "--replication-factor", "1",
+            "--config", cleanupKey + "=" + cleanupVal,
+            "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+        ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName);
+        Config props = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource);
+        // val props = adminZkClient.fetchEntityConfig(ConfigType.Topic, testTopicName)
+        assertNotNull(props.get(cleanupKey), "Properties after creation don't contain " + cleanupKey);
+        assertEquals(cleanupVal, props.get(cleanupKey).value(), "Properties after creation have incorrect value");
+
+        // pre-create the topic config changes path to avoid a NoNodeException
+        if (!isKRaftTest()) {
+            zkClient().makeSurePersistentPathExists(kafka.zk.ConfigEntityChangeNotificationZNode.path());
+        }
+
+        // modify the topic to add new partitions
+        int numPartitionsModified = 3;
+        TopicCommand.TopicCommandOptions alterOpts = buildTopicCommandOptionsWithBootstrap("--alter",
+            "--partitions", Integer.toString(numPartitionsModified), "--topic", testTopicName);
+        topicService.alterTopic(alterOpts);
+        Config newProps = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource);
+        assertNotNull(newProps.get(cleanupKey), "Updated properties do not contain " + cleanupKey);
+        assertEquals(cleanupVal, newProps.get(cleanupKey).value(), "Updated properties have incorrect value");
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testTopicDeletion(String quorum) throws Exception {
+        // create the NormalTopic
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "1",
+            "--replication-factor", "1",
+            "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+
+        // delete the NormalTopic
+        TopicCommand.TopicCommandOptions deleteOpts = buildTopicCommandOptionsWithBootstrap("--delete", "--topic", testTopicName);
+
+        if (!isKRaftTest()) {
+            String deletePath = kafka.zk.DeleteTopicsTopicZNode.path(testTopicName);
+            assertFalse(zkClient().pathExists(deletePath), "Delete path for topic shouldn't exist before deletion.");
+        }
+        topicService.deleteTopic(deleteOpts);
+        TestUtils.verifyTopicDeletion(zkClientOrNull(), testTopicName, 1, brokers());
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testTopicWithCollidingCharDeletionAndCreateAgain(String quorum) throws Exception {
+        // create the topic with colliding chars
+        String topicWithCollidingChar = "test.a";
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "1",
+            "--replication-factor", "1",
+            "--topic", topicWithCollidingChar);
+        createAndWaitTopic(createOpts);
+
+        // delete the topic
+        TopicCommand.TopicCommandOptions deleteOpts = buildTopicCommandOptionsWithBootstrap("--delete", "--topic", topicWithCollidingChar);
+
+        if (!isKRaftTest()) {
+            String deletePath = kafka.zk.DeleteTopicsTopicZNode.path(topicWithCollidingChar);
+            assertFalse(zkClient().pathExists(deletePath), "Delete path for topic shouldn't exist before deletion.");
+        }
+        topicService.deleteTopic(deleteOpts);
+        TestUtils.verifyTopicDeletion(zkClientOrNull(), topicWithCollidingChar, 1, brokers());
+        assertDoesNotThrow(() -> createAndWaitTopic(createOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteInternalTopic(String quorum) throws Exception {
+        // create the offset topic
+        TopicCommand.TopicCommandOptions createOffsetTopicOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "1",
+            "--replication-factor", "1",
+            "--topic", Topic.GROUP_METADATA_TOPIC_NAME);
+        createAndWaitTopic(createOffsetTopicOpts);
+
+        // Try to delete the Topic.GROUP_METADATA_TOPIC_NAME which is allowed by default.
+        // This is a difference between the new and the old command as the old one didn't allow internal topic deletion.
+        // If deleting internal topics is not desired, ACLS should be used to control it.
+        TopicCommand.TopicCommandOptions deleteOffsetTopicOpts =
+                buildTopicCommandOptionsWithBootstrap("--delete", "--topic", Topic.GROUP_METADATA_TOPIC_NAME);
+        String deleteOffsetTopicPath = kafka.zk.DeleteTopicsTopicZNode.path(Topic.GROUP_METADATA_TOPIC_NAME);
+        if (!isKRaftTest()) {
+            assertFalse(zkClient().pathExists(deleteOffsetTopicPath), "Delete path for topic shouldn't exist before deletion.");
+        }
+        topicService.deleteTopic(deleteOffsetTopicOpts);
+        TestUtils.verifyTopicDeletion(zkClientOrNull(), Topic.GROUP_METADATA_TOPIC_NAME, 1, brokers());
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteWhenTopicDoesntExist(String quorum) {
+        // delete a topic that does not exist
+        TopicCommand.TopicCommandOptions deleteOpts = buildTopicCommandOptionsWithBootstrap("--delete", "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.deleteTopic(deleteOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteWhenTopicDoesntExistWithIfExists(String quorum) throws ExecutionException, InterruptedException {
+        topicService.deleteTopic(buildTopicCommandOptionsWithBootstrap("--delete", "--topic", testTopicName, "--if-exists"));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribe(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName));
+        String[] rows = output.split("\n");
+        assertEquals(3, rows.length);
+        assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeWhenTopicDoesntExist(String quorum) {
+        assertThrows(IllegalArgumentException.class,
+            () -> topicService.describeTopic(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName)));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeWhenTopicDoesntExistWithIfExists(String quorum) throws ExecutionException, InterruptedException {
+        topicService.describeTopic(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName, "--if-exists"));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeUnavailablePartitions(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 6, (short) 1))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        try {
+            // check which partition is on broker 0 which we'll kill
+            TopicDescription testTopicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName))
+                .allTopicNames().get().get(testTopicName);
+            int partitionOnBroker0 = testTopicDescription.partitions().stream()
+                .filter(partition -> partition.leader().id() == 0)
+                .findFirst().get().partition();
+
+            killBroker(0);
+
+            // wait until the topic metadata for the test topic is propagated to each alive broker
+            kafka.utils.TestUtils.waitUntilTrue(
+                () -> {
+                    boolean result = true;
+                    for (KafkaBroker server : JavaConverters.asJavaCollection(brokers())) {
+                        if (server.config().brokerId() != 0) {
+                            Set<String> topicNames = Collections.singleton(testTopicName);
+                            Collection<MetadataResponseData.MetadataResponseTopic> topicMetadatas =
+                                JavaConverters.asJavaCollection(server.dataPlaneRequestProcessor().metadataCache()
+                                .getTopicMetadata(JavaConverters.asScalaSetConverter(topicNames).asScala().toSet(),
+                                    ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
+                                    false, false)
+                                );
+                            Optional<MetadataResponseData.MetadataResponseTopic> testTopicMetadata = topicMetadatas.stream()
+                                .filter(metadata -> metadata.name().equals(testTopicName))
+                                .findFirst();
+                            if (!testTopicMetadata.isPresent()) {
+                                throw new AssertionError("Topic metadata is not found in metadata cache");
+                            }
+                            Optional<MetadataResponseData.MetadataResponsePartition> testPartitionMetadata = testTopicMetadata.get().partitions().stream()
+                                .filter(metadata -> metadata.partitionIndex() == partitionOnBroker0)
+                                .findFirst();
+                            if (!testPartitionMetadata.isPresent()) {
+                                throw new AssertionError("Partition metadata is not found in metadata cache");
+                            }
+                            result = result && testPartitionMetadata.get().errorCode() == Errors.LEADER_NOT_AVAILABLE.code();
+                        }
+                    }
+                    return result;
+                },
+                () -> String.format("Partition metadata for %s is not propagated", testTopicName),
+                org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+
+            // grab the console output and assert
+            String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName, "--unavailable-partitions"));
+            String[] rows = output.split("\n");
+            assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)));
+            assertTrue(rows[0].contains("Leader: none\tReplicas: 0\tIsr:"));
+        } finally {
+            restartDeadBrokers(false);
+        }
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeUnderReplicatedPartitions(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 1, (short) 6))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        try {
+            killBroker(0);
+            if (isKRaftTest()) {
+                ensureConsistentKRaftMetadata();
+            } else {
+                TestUtils.waitForPartitionMetadata(aliveBrokers(), testTopicName, 0, org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS);
+            }
+            String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--under-replicated-partitions"));
+            String[] rows = output.split("\n");
+            assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)), String.format("Unexpected output: %s", rows[0]));
+        } finally {
+            restartDeadBrokers(false);
+        }
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeUnderMinIsrPartitions(String quorum) throws ExecutionException, InterruptedException {
+        Map<String, String> configMap = new HashMap<>();
+        configMap.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "6");
+
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 1, (short) 6).configs(configMap))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        try {
+            killBroker(0);
+            if (isKRaftTest()) {
+                ensureConsistentKRaftMetadata();
+            } else {
+                kafka.utils.TestUtils.waitUntilTrue(
+                    () -> aliveBrokers().forall(b -> b.metadataCache().getPartitionInfo(testTopicName, 0).get().isr().size() == 5),
+                    () -> String.format("Timeout waiting for partition metadata propagating to brokers for %s topic", testTopicName),
+                    org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L
+                );
+            }
+            String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--under-min-isr-partitions"));
+            String[] rows = output.split("\n");
+            assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)));
+        } finally {
+            restartDeadBrokers(false);
+        }
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress(String quorum) throws ExecutionException, InterruptedException {
+        Map<String, String> configMap = new HashMap<>();
+        Short replicationFactor = 1;
+        int partitions = 1;
+        TopicPartition tp = new TopicPartition(testTopicName, 0);
+
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, partitions, replicationFactor).configs(configMap))
+        ).all().get();
+        waitForTopicCreated(testTopicName);
+
+        // Produce multiple batches.
+        TestUtils.generateAndProduceMessages(brokers(), testTopicName, 10, -1);
+        TestUtils.generateAndProduceMessages(brokers(), testTopicName, 10, -1);
+
+        // Enable throttling. Note the broker config sets the replica max fetch bytes to `1` upon to minimize replication
+        // throughput so the reassignment doesn't complete quickly.
+        List<Integer> brokerIds = JavaConverters.seqAsJavaList(brokers()).stream()
+            .map(broker -> broker.config().brokerId()).collect(Collectors.toList());
+
+        ToolsTestUtils.setReplicationThrottleForPartitions(adminClient, brokerIds, Collections.singleton(tp), 1);
+
+        TopicDescription testTopicDesc = adminClient.describeTopics(Collections.singleton(testTopicName)).allTopicNames().get().get(testTopicName);
+        TopicPartitionInfo firstPartition = testTopicDesc.partitions().get(0);
+
+        List<Integer> replicasOfFirstPartition = firstPartition.replicas().stream().map(Node::id).collect(Collectors.toList());
+        List<Integer> replicasDiff = new ArrayList<>(brokerIds);
+        replicasDiff.removeAll(replicasOfFirstPartition);
+        Integer targetReplica = replicasDiff.get(0);
+
+        adminClient.alterPartitionReassignments(Collections.singletonMap(tp,
+            Optional.of(new NewPartitionReassignment(Collections.singletonList(targetReplica))))).all().get();
+
+        // let's wait until the LAIR is propagated
+        kafka.utils.TestUtils.waitUntilTrue(
+            () -> {
+                try {
+                    return !adminClient.listPartitionReassignments(Collections.singleton(tp)).reassignments().get()
+                        .get(tp).addingReplicas().isEmpty();
+                } catch (InterruptedException | ExecutionException e) {
+                    throw new RuntimeException(e);
+                }
+            },
+            () -> "Reassignment didn't add the second node",
+            org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+
+        // describe the topic and test if it's under-replicated
+        String simpleDescribeOutput = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName));
+        String[] simpleDescribeOutputRows = simpleDescribeOutput.split("\n");
+        assertTrue(simpleDescribeOutputRows[0].startsWith(String.format("Topic: %s", testTopicName)));
+        assertEquals(2, simpleDescribeOutputRows.length);
+
+        String underReplicatedOutput = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--under-replicated-partitions"));
+        assertEquals("", underReplicatedOutput,
+            String.format("--under-replicated-partitions shouldn't return anything: '%s'", underReplicatedOutput));
+
+        // Verify reassignment is still ongoing.
+        PartitionReassignment reassignments = adminClient.listPartitionReassignments(Collections.singleton(tp)).reassignments().get().get(tp);
+        assertFalse(reassignments.addingReplicas().isEmpty());
+
+        ToolsTestUtils.removeReplicationThrottleForPartitions(adminClient, brokerIds, Collections.singleton(tp));
+        TestUtils.waitForAllReassignmentsToComplete(adminClient, 100L);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeAtMinIsrPartitions(String quorum) throws ExecutionException, InterruptedException {
+        Map<String, String> configMap = new HashMap<>();
+        configMap.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "4");
+
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 1, (short) 6).configs(configMap))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        try {
+            killBroker(0);
+            killBroker(1);
+
+            if (isKRaftTest()) {
+                ensureConsistentKRaftMetadata();
+            } else {
+                kafka.utils.TestUtils.waitUntilTrue(
+                    () -> aliveBrokers().forall(broker -> broker.metadataCache().getPartitionInfo(testTopicName, 0).get().isr().size() == 4),
+                    () -> String.format("Timeout waiting for partition metadata propagating to brokers for %s topic", testTopicName),
+                    org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L
+                );
+            }
+
+            String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--at-min-isr-partitions"));
+            String[] rows = output.split("\n");
+            assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)));
+            assertEquals(1, rows.length);
+        } finally {
+            restartDeadBrokers(false);
+        }
+    }
+
+    /**
+     * Test describe --under-min-isr-partitions option with four topics:
+     *   (1) topic with partition under the configured min ISR count
+     *   (2) topic with under-replicated partition (but not under min ISR count)
+     *   (3) topic with offline partition
+     *   (4) topic with fully replicated partition
+     *
+     * Output should only display the (1) topic with partition under min ISR count and (3) topic with offline partition
+     */
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeUnderMinIsrPartitionsMixed(String quorum) throws ExecutionException, InterruptedException {
+        String underMinIsrTopic = "under-min-isr-topic";
+        String notUnderMinIsrTopic = "not-under-min-isr-topic";
+        String offlineTopic = "offline-topic";
+        String fullyReplicatedTopic = "fully-replicated-topic";
+
+        Map<String, String> configMap = new HashMap<>();
+        configMap.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "6");
+
+        adminClient.createTopics(
+            java.util.Arrays.asList(
+                new NewTopic(underMinIsrTopic, 1, (short) 6).configs(configMap),
+                new NewTopic(notUnderMinIsrTopic, 1, (short) 6),
+                new NewTopic(offlineTopic, Collections.singletonMap(0, Collections.singletonList(0))),
+                new NewTopic(fullyReplicatedTopic, Collections.singletonMap(0, java.util.Arrays.asList(1, 2, 3))))
+        ).all().get();
+
+        waitForTopicCreated(underMinIsrTopic);
+        waitForTopicCreated(notUnderMinIsrTopic);
+        waitForTopicCreated(offlineTopic);
+        waitForTopicCreated(fullyReplicatedTopic);
+
+        try {
+            killBroker(0);
+            if (isKRaftTest()) {
+                ensureConsistentKRaftMetadata();
+            } else {
+                kafka.utils.TestUtils.waitUntilTrue(
+                    () -> aliveBrokers().forall(broker ->
+                        broker.metadataCache().getPartitionInfo(underMinIsrTopic, 0).get().isr().size() < 6 &&
+                            broker.metadataCache().getPartitionInfo(offlineTopic, 0).get().leader() == MetadataResponse.NO_LEADER_ID),
+                    () -> "Timeout waiting for partition metadata propagating to brokers for underMinIsrTopic topic",
+                    org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+            }
+            String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--under-min-isr-partitions"));
+            String[] rows = output.split("\n");
+            assertTrue(rows[0].startsWith(String.format("Topic: %s", underMinIsrTopic)));
+            assertTrue(rows[1].startsWith(String.format("\tTopic: %s", offlineTopic)));
+            assertEquals(2, rows.length);
+        } finally {
+            restartDeadBrokers(false);
+        }
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeReportOverriddenConfigs(String quorum) throws Exception {
+        String config = "file.delete.delay.ms=1000";
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "2",
+            "--replication-factor", "2", "--topic", testTopicName, "--config", config));
+        String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe"));
+        assertTrue(output.contains(config), String.format("Describe output should have contained %s", config));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeAndListTopicsWithoutInternalTopics(String quorum) throws Exception {
+        createAndWaitTopic(
+            buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "1", "--replication-factor", "1", "--topic", testTopicName));
+        // create a internal topic
+        createAndWaitTopic(
+            buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "1", "--replication-factor", "1", "--topic", Topic.GROUP_METADATA_TOPIC_NAME));
+
+        // test describe
+        String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--describe", "--exclude-internal"));
+        assertTrue(output.contains(testTopicName),
+            String.format("Output should have contained %s", testTopicName));
+        assertFalse(output.contains(Topic.GROUP_METADATA_TOPIC_NAME));
+
+        // test list
+        output = captureListTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--list", "--exclude-internal"));
+        assertTrue(output.contains(testTopicName));
+        assertFalse(output.contains(Topic.GROUP_METADATA_TOPIC_NAME));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized(String quorum) throws ExecutionException, InterruptedException {

Review Comment:
   We don't seem to have logic to have the `listPartitionReassignments()` call fail with `ClusterAuthorizationException` 



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import kafka.utils.Exit;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientTestUtils;
+import org.apache.kafka.clients.admin.CreatePartitionsResult;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.DeleteTopicsResult;
+import org.apache.kafka.clients.admin.DescribeTopicsResult;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.errors.ThrottlingQuotaExceededException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyCollection;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.verify;
+
+@Timeout(value = 60)
+public class TopicCommandTest {
+    private String bootstrapServer = "localhost:9092";
+    private String topicName = "topicName";
+
+    @Test
+    public void testIsNotUnderReplicatedWhenAdding() {
+        List<Integer> replicaIds = Arrays.asList(1, 2);
+        List<Node> replicas = new ArrayList<>();
+        for (int id : replicaIds) {
+            replicas.add(new Node(id, "localhost", 9090 + id));
+        }
+
+        TopicCommand.PartitionDescription partitionDescription = new TopicCommand.PartitionDescription("test-topic",
+            new TopicPartitionInfo(0, new Node(1, "localhost", 9091), replicas,
+                Collections.singletonList(new Node(1, "localhost", 9091))),
+            null, false,
+                new PartitionReassignment(replicaIds, Arrays.asList(2), Collections.emptyList())
+        );
+
+        assertFalse(partitionDescription.isUnderReplicated());
+    }
+
+    @Test
+    public void testAlterWithUnspecifiedPartitionCount() {
+        String[] options = new String[] {" --bootstrap-server", bootstrapServer, "--alter", "--topic", topicName};
+        assertInitializeInvalidOptionsExitCode(1, options);
+    }
+
+    @Test
+    public void testConfigOptWithBootstrapServers() {
+        assertInitializeInvalidOptionsExitCode(1,
+            new String[] {"--bootstrap-server", bootstrapServer, "--alter", "--topic", topicName,
+                "--partitions", "3", "--config", "cleanup.policy=compact"});
+        assertInitializeInvalidOptionsExitCode(1,
+            new String[] {"--bootstrap-server", bootstrapServer, "--alter", "--topic", topicName,
+                "--partitions", "3", "--delete-config", "cleanup.policy"});
+        TopicCommand.TopicCommandOptions opts =
+            new TopicCommand.TopicCommandOptions(
+                new String[] {"--bootstrap-server", bootstrapServer, "--create", "--topic", topicName, "--partitions", "3",
+                    "--replication-factor", "3", "--config", "cleanup.policy=compact"});
+        assertTrue(opts.hasCreateOption());
+        assertEquals(bootstrapServer, opts.bootstrapServer().get());
+        assertEquals("cleanup.policy=compact", opts.topicConfig().get().get(0));
+    }
+
+    @Test
+    public void testCreateWithPartitionCountWithoutReplicationFactorShouldSucceed() {
+        TopicCommand.TopicCommandOptions opts = new TopicCommand.TopicCommandOptions(
+            new String[] {"--bootstrap-server", bootstrapServer,
+                "--create",
+                "--partitions", "2",
+                "--topic", topicName});
+        assertTrue(opts.hasCreateOption());
+        assertEquals(topicName, opts.topic().get());
+        assertEquals(2, opts.partitions().get());

Review Comment:
   The original test called `opts.checkArgs()`, should we do that here to?



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java:
##########
@@ -0,0 +1,1050 @@
+/*
+ * 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;
+
+import kafka.admin.RackAwareTest;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Tag("integration")
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
+public class TopicCommandIntegrationTest extends kafka.integration.KafkaServerTestHarness implements Logging, RackAwareTest {
+    private Short defaultReplicationFactor = 1;
+    private Integer numPartitions = 1;
+    private TopicCommand.TopicService topicService;
+    private Admin adminClient;
+    private String bootstrapServer;
+    private String testTopicName;
+    private Integer defaultTimeout = 10000;

Review Comment:
   This can be a `long`



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java:
##########
@@ -0,0 +1,1050 @@
+/*
+ * 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;
+
+import kafka.admin.RackAwareTest;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Tag("integration")
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
+public class TopicCommandIntegrationTest extends kafka.integration.KafkaServerTestHarness implements Logging, RackAwareTest {
+    private Short defaultReplicationFactor = 1;

Review Comment:
   This can be a `short`



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java:
##########
@@ -0,0 +1,1050 @@
+/*
+ * 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;
+
+import kafka.admin.RackAwareTest;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Tag("integration")
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
+public class TopicCommandIntegrationTest extends kafka.integration.KafkaServerTestHarness implements Logging, RackAwareTest {
+    private Short defaultReplicationFactor = 1;
+    private Integer numPartitions = 1;

Review Comment:
   This can be an `int`



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java:
##########
@@ -0,0 +1,1050 @@
+/*
+ * 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;
+
+import kafka.admin.RackAwareTest;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Tag("integration")
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
+public class TopicCommandIntegrationTest extends kafka.integration.KafkaServerTestHarness implements Logging, RackAwareTest {
+    private Short defaultReplicationFactor = 1;
+    private Integer numPartitions = 1;
+    private TopicCommand.TopicService topicService;
+    private Admin adminClient;
+    private String bootstrapServer;
+    private String testTopicName;
+    private Integer defaultTimeout = 10000;
+
+    /**
+     * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every
+     * test and should not reuse previous configurations unless they select their ports randomly when servers are started.
+     *
+     * Note the replica fetch max bytes is set to `1` in order to throttle the rate of replication for test
+     * `testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress`.
+     */
+    @Override
+    public scala.collection.Seq<KafkaConfig> generateConfigs() {
+        Map<Integer, String> rackInfo = new HashMap<>();
+        rackInfo.put(0, "rack1");
+        rackInfo.put(1, "rack2");
+        rackInfo.put(2, "rack2");
+        rackInfo.put(3, "rack1");
+        rackInfo.put(4, "rack3");
+        rackInfo.put(5, "rack3");
+
+        List<Properties> brokerConfigs = ToolsTestUtils
+            .createBrokerProperties(6, zkConnectOrNull(), rackInfo, numPartitions, defaultReplicationFactor);
+
+        List<KafkaConfig> configs = new ArrayList<>();
+        for (Properties props : brokerConfigs) {
+            props.put(KafkaConfig.ReplicaFetchMaxBytesProp(), "1");
+            configs.add(KafkaConfig.fromProps(props));
+        }
+        return JavaConverters.asScalaBuffer(configs).toSeq();
+    }
+
+    private TopicCommand.TopicCommandOptions buildTopicCommandOptionsWithBootstrap(String... opts) {
+        String[] finalOptions = Stream.concat(Arrays.asList(opts).stream(),
+                Arrays.asList("--bootstrap-server", bootstrapServer).stream()
+        ).toArray(String[]::new);
+        return new TopicCommand.TopicCommandOptions(finalOptions);
+    }
+
+    private void createAndWaitTopic(TopicCommand.TopicCommandOptions opts) throws Exception {
+        topicService.createTopic(opts);
+        waitForTopicCreated(opts.topic().get());
+    }
+
+    private void waitForTopicCreated(String topicName) {
+        waitForTopicCreated(topicName, defaultTimeout);
+    }
+
+    private void waitForTopicCreated(String topicName, Integer timeout) {
+        TestUtils.waitForPartitionMetadata(brokers(), topicName, 0, timeout);
+    }
+
+    @BeforeEach
+    public void setUp(TestInfo info) {
+        super.setUp(info);
+        // create adminClient
+        Properties props = new Properties();
+        bootstrapServer = bootstrapServers(listenerName());
+        props.put(org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+        adminClient = Admin.create(props);
+        topicService = new TopicCommand.TopicService(props, Optional.of(bootstrapServer));
+        testTopicName = String.format("%s-%s", info.getTestMethod().get().getName(),
+            new scala.util.Random().alphanumeric().take(10).mkString());
+    }
+
+    @AfterEach
+    public void close() throws Exception {
+        if (topicService != null)
+            topicService.close();
+        if (adminClient != null)
+            adminClient.close();
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreate(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", "2", "--replication-factor", "1", "--topic", testTopicName));
+
+        adminClient.listTopics().names().get().contains(testTopicName);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithDefaults(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName));
+
+        List<TopicPartitionInfo> partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+        assertEquals(partitions.size(), numPartitions);
+        assertEquals((short) partitions.get(0).replicas().size(), defaultReplicationFactor);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithDefaultReplication(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName, "--partitions", "2"));
+
+        List<TopicPartitionInfo>  partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+        assertEquals(partitions.size(), 2);
+        assertEquals((short) partitions.get(0).replicas().size(), defaultReplicationFactor);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithDefaultPartitions(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName, "--replication-factor", "2"));
+
+        List<TopicPartitionInfo> partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+
+        assertEquals(partitions.size(), numPartitions);
+        assertEquals((short) partitions.get(0).replicas().size(), 2);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithConfigs(String quorum) throws Exception {
+        ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName);
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "2", "--replication-factor", "2", "--topic", testTopicName, "--config",
+                "delete.retention.ms=1000"));
+
+        Config configs = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource);
+        assertEquals(1000, Integer.valueOf(configs.get("delete.retention.ms").value()));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWhenAlreadyExists(String quorum) throws Exception {
+        int numPartitions = 1;
+
+        // create the topic
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", Integer.toString(numPartitions), "--replication-factor", "1",
+                "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+
+        // try to re-create the topic
+        assertThrows(TopicExistsException.class, () -> topicService.createTopic(createOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWhenAlreadyExistsWithIfNotExists(String quorum) throws Exception {
+        TopicCommand.TopicCommandOptions createOpts =
+                buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName, "--if-not-exists");
+        createAndWaitTopic(createOpts);
+        topicService.createTopic(createOpts);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithReplicaAssignment(String quorum) throws Exception {
+        // create the topic
+        TopicCommand.TopicCommandOptions createOpts =
+                buildTopicCommandOptionsWithBootstrap("--create", "--replica-assignment", "5:4,3:2,1:0", "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+
+        List<TopicPartitionInfo> partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+        
+        assertEquals(3, partitions.size());
+        assertEquals(Arrays.asList(5, 4), getPartitionReplicas(partitions, 0));
+        assertEquals(Arrays.asList(3, 2), getPartitionReplicas(partitions, 1));
+        assertEquals(Arrays.asList(1, 0), getPartitionReplicas(partitions, 2));
+    }
+
+    private List<Integer> getPartitionReplicas(List<TopicPartitionInfo> partitions, int partitionNumber) {
+        return partitions.get(partitionNumber).replicas().stream().map(Node::id).collect(Collectors.toList());
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithInvalidReplicationFactor(String quorum) {
+        TopicCommand.TopicCommandOptions opts = buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "2", "--replication-factor", Integer.toString(Short.MAX_VALUE + 1),
+            "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.createTopic(opts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithNegativeReplicationFactor(String quorum) {
+        TopicCommand.TopicCommandOptions opts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "2", "--replication-factor", "-1", "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.createTopic(opts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithNegativePartitionCount(String quorum) {
+        TopicCommand.TopicCommandOptions opts = buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "-1", "--replication-factor", "1", "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.createTopic(opts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testInvalidTopicLevelConfig(String quorum) {
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "1", "--replication-factor", "1", "--topic", testTopicName,
+            "--config", "message.timestamp.type=boom");
+        assertThrows(ConfigException.class, () -> topicService.createTopic(createOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListTopics(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", "1", "--replication-factor", "1", "--topic", testTopicName));
+
+        String output = captureListTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--list"));
+        assertTrue(output.contains(testTopicName));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListTopicsWithIncludeList(String quorum) throws ExecutionException, InterruptedException {
+        String topic1 = "kafka.testTopic1";
+        String topic2 = "kafka.testTopic2";
+        String topic3 = "oooof.testTopic1";
+        adminClient.createTopics(
+                Arrays.asList(new NewTopic(topic1, 2, (short) 2),
+                    new NewTopic(topic2, 2, (short) 2),
+                    new NewTopic(topic3, 2, (short) 2)))
+            .all().get();
+        waitForTopicCreated(topic1);
+        waitForTopicCreated(topic2);
+        waitForTopicCreated(topic3);
+
+        String output = captureListTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--list", "--topic", "kafka.*"));
+
+        assertTrue(output.contains(topic1));
+        assertTrue(output.contains(topic2));
+        assertFalse(output.contains(topic3));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListTopicsWithExcludeInternal(String quorum) throws ExecutionException, InterruptedException {
+        String topic1 = "kafka.testTopic1";
+        adminClient.createTopics(
+                Arrays.asList(new NewTopic(topic1, 2, (short) 2),
+                    new NewTopic(Topic.GROUP_METADATA_TOPIC_NAME, 2, (short) 2)))
+            .all().get();
+        waitForTopicCreated(topic1);
+
+        String output = captureListTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--list", "--exclude-internal"));
+
+        assertTrue(output.contains(topic1));
+        assertFalse(output.contains(Topic.GROUP_METADATA_TOPIC_NAME));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterPartitionCount(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Arrays.asList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter", "--topic", testTopicName, "--partitions", "3"));
+
+        kafka.utils.TestUtils.waitUntilTrue(
+            () -> brokers().forall(b -> b.metadataCache().getTopicPartitions(testTopicName).size() == 3),
+            () -> "Timeout waiting for new assignment propagating to broker", org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+        TopicDescription topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get();
+        assertTrue(topicDescription.partitions().size() == 3);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterAssignment(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter",
+            "--topic", testTopicName, "--replica-assignment", "5:3,3:1,4:2", "--partitions", "3"));
+        kafka.utils.TestUtils.waitUntilTrue(
+            () -> brokers().forall(b -> b.metadataCache().getTopicPartitions(testTopicName).size() == 3),
+            () -> "Timeout waiting for new assignment propagating to broker",
+            org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+
+        TopicDescription topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get();
+        assertTrue(topicDescription.partitions().size() == 3);

Review Comment:
   nit: `assertEqual(3` is slightly simpler



##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1007 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+public abstract class TopicCommand {
+    private static final Logger LOG = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String... args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        LOG.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")
+    static Map<Integer, List<Integer>> parseReplicaAssignment(String replicaAssignmentList) {
+        String[] partitionList = replicaAssignmentList.split(",");
+        Map<Integer, List<Integer>> ret = new LinkedHashMap<Integer, List<Integer>>();
+        for (int i = 0; i < partitionList.length; i++) {
+            List<Integer> brokerList = Arrays.stream(partitionList[i].split(":"))
+                .map(String::trim)
+                .mapToInt(Integer::parseInt)
+                .boxed()
+                .collect(Collectors.toList());
+            Collection<Integer> duplicateBrokers = ToolsUtils.duplicates(brokerList);
+            if (!duplicateBrokers.isEmpty()) {
+                throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: " +
+                    duplicateBrokers.stream()
+                        .map(Object::toString)
+                        .collect(Collectors.joining(","))
+                );
+            }
+            ret.put(i, brokerList);
+            if (ret.get(i).size() != ret.get(0).size()) {
+                throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList);
+            }
+        }
+        return ret;
+    }
+
+    @SuppressWarnings("deprecation")
+    private static Properties parseTopicConfigsToBeAdded(TopicCommandOptions opts) {
+        List<List<String>> configsToBeAdded = opts.topicConfig().orElse(Collections.emptyList())
+            .stream()
+            .map(s -> Arrays.asList(s.split("\\s*=\\s*")))
+            .collect(Collectors.toList());
+
+        if (!configsToBeAdded.stream().allMatch(config -> config.size() == 2)) {
+            throw new IllegalArgumentException("requirement failed: Invalid topic config: all configs to be added must be in the format \"key=val\".");
+        }
+
+        Properties props = new Properties();
+        configsToBeAdded.stream()
+            .forEach(pair -> props.setProperty(pair.get(0).trim(), pair.get(1).trim()));
+        LogConfig.validate(props);
+        if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) {
+            System.out.println("WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " +
+                "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " +
+                "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.");
+        }
+        return props;
+    }
+
+    // It is possible for a reassignment to complete between the time we have fetched its state and the time
+    // we fetch partition metadata. In this case, we ignore the reassignment when determining replication factor.
+    public static boolean isReassignmentInProgress(TopicPartitionInfo tpi, PartitionReassignment ra) {
+        // Reassignment is still in progress as long as the removing and adding replicas are still present
+        Set<Integer> allReplicaIds = tpi.replicas().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> changingReplicaIds = new HashSet<>();
+        if (ra != null) {
+            changingReplicaIds.addAll(ra.removingReplicas());
+            changingReplicaIds.addAll(ra.addingReplicas());
+        }
+        return allReplicaIds.stream().anyMatch(changingReplicaIds::contains);
+
+    }
+
+    private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionReassignment reassignment) {
+        return isReassignmentInProgress(tpi, reassignment) ?
+            reassignment.replicas().size() - reassignment.addingReplicas().size() :
+            tpi.replicas().size();
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopics        Topics that were found to match the requested topic name.
+     * @param requestedTopic     Name of the topic that was requested.
+     * @param requireTopicExists Indicates if the topic needs to exist for the operation to be successful.
+     *                           If set to true, the command will throw an exception if the topic with the
+     *                           requested name does not exist.
+     */
+    private static void ensureTopicExists(List<String> foundTopics, String requestedTopic, Boolean requireTopicExists) {
+        // If no topic name was mentioned, do not need to throw exception.
+        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
+            // If given topic doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
+        }
+    }
+
+    private static List<String> doGetTopics(List<String> allTopics, Optional<String> topicIncludeList, Boolean excludeInternalTopics) {
+        if (topicIncludeList.isPresent()) {
+            IncludeList topicsFilter = new IncludeList(topicIncludeList.get());
+            return allTopics.stream()
+                .filter(topic -> topicsFilter.isTopicAllowed(topic, excludeInternalTopics))
+                .collect(Collectors.toList());
+        } else {
+            return allTopics.stream()
+                .filter(topic -> !(Topic.isInternal(topic) && excludeInternalTopics))
+                .collect(Collectors.toList());
+        }
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopicIds        Topics that were found to match the requested topic id.
+     * @param requestedTopicId     Id of the topic that was requested.
+     * @param requireTopicIdExists Indicates if the topic needs to exist for the operation to be successful.
+     *                             If set to true, the command will throw an exception if the topic with the
+     *                             requested id does not exist.
+     */
+    private static void ensureTopicIdExists(List<Uuid> foundTopicIds, Uuid requestedTopicId, Boolean requireTopicIdExists) {
+        // If no topic id was mentioned, do not need to throw exception.
+        if (requestedTopicId != null && requireTopicIdExists && foundTopicIds.isEmpty()) {
+            // If given topicId doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("TopicId '%s' does not exist as expected", requestedTopicId));
+        }
+    }
+
+    static class CommandTopicPartition {
+        private final Optional<String> name;
+        private final Optional<Integer> partitions;
+        private final Optional<Integer> replicationFactor;
+        private final Map<Integer, List<Integer>> replicaAssignment;
+        private final Properties configsToAdd;
+
+        private final TopicCommandOptions opts;
+
+        public CommandTopicPartition(TopicCommandOptions options) {
+            opts = options;
+            name = options.topic();
+            partitions = options.partitions();
+            replicationFactor = options.replicationFactor();
+            replicaAssignment = options.replicaAssignment().orElse(Collections.emptyMap());
+            configsToAdd = parseTopicConfigsToBeAdded(options);
+        }
+
+        public Boolean hasReplicaAssignment() {
+            return !replicaAssignment.isEmpty();
+        }
+
+        public Boolean ifTopicDoesntExist() {
+            return opts.ifNotExists();
+        }
+    }
+
+    static class TopicDescription {
+        private final String topic;
+        private final Uuid topicId;
+        private final Integer numPartitions;
+        private final Integer replicationFactor;
+        private final Config config;
+        private final Boolean markedForDeletion;
+
+        public TopicDescription(String topic, Uuid topicId, Integer numPartitions, Integer replicationFactor, Config config, Boolean markedForDeletion) {
+            this.topic = topic;
+            this.topicId = topicId;
+            this.numPartitions = numPartitions;
+            this.replicationFactor = replicationFactor;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+        }
+
+        public void printDescription() {
+            String configsAsString = config.entries().stream()
+                .filter(config -> !config.isDefault())
+                .map(ce -> ce.name() + "=" + ce.value())
+                .collect(Collectors.joining(","));
+            System.out.print("Topic: " +  topic);
+            if (topicId != Uuid.ZERO_UUID)
+                System.out.print("\tTopicId: " + topicId);
+            System.out.print("\tPartitionCount: " + numPartitions);
+            System.out.print("\tReplicationFactor: " + replicationFactor);
+            System.out.print("\tConfigs: " + configsAsString);
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class PartitionDescription {
+        private final String topic;
+        private final TopicPartitionInfo info;
+        private final Config config;
+        private final Boolean markedForDeletion;
+        private final PartitionReassignment reassignment;
+
+        PartitionDescription(String topic,
+                             TopicPartitionInfo info,
+                             Config config,
+                             Boolean markedForDeletion,
+                             PartitionReassignment reassignment) {
+            this.topic = topic;
+            this.info = info;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+            this.reassignment = reassignment;
+        }
+
+        public Integer minIsrCount() {
+            return Integer.parseInt(config.get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value());
+        }
+
+        public Boolean isUnderReplicated() {
+            return getReplicationFactor(info, reassignment) - info.isr().size() > 0;
+        }
+
+        public boolean hasLeader() {
+            return info.leader() != null;
+        }
+
+        public Boolean isUnderMinIsr() {
+            return !hasLeader() ||  info.isr().size() < minIsrCount();
+        }
+
+        public Boolean isAtMinIsrPartitions() {
+            return minIsrCount() == info.isr().size();
+        }
+
+        public Boolean hasUnavailablePartitions(Set<Integer> liveBrokers) {
+            return !hasLeader() || !liveBrokers.contains(info.leader().id());
+        }
+
+        public void printDescription() {
+            System.out.print("\tTopic: " + topic);
+            System.out.print("\tPartition: " + info.partition());
+            System.out.print("\tLeader: " + (hasLeader() ? info.leader().id() : "none"));
+            System.out.print("\tReplicas: " + info.replicas().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            System.out.print("\tIsr: " + info.isr().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            if (reassignment != null) {
+                System.out.print("\tAdding Replicas: " + reassignment.addingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+                System.out.print("\tRemoving Replicas: " + reassignment.removingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+            }
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class DescribeOptions {
+        private final TopicCommandOptions opts;
+        private final Set<Integer> liveBrokers;
+        private final boolean describeConfigs;
+        private final boolean describePartitions;
+
+        public DescribeOptions(TopicCommandOptions opts, Set<Integer> liveBrokers) {
+            this.opts = opts;
+            this.liveBrokers = liveBrokers;
+            this.describeConfigs = !opts.reportUnavailablePartitions() &&
+                !opts.reportUnderReplicatedPartitions() &&
+                !opts.reportUnderMinIsrPartitions() &&
+                !opts.reportAtMinIsrPartitions();
+            this.describePartitions = !opts.reportOverriddenConfigs();
+        }
+
+        private boolean shouldPrintUnderReplicatedPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderReplicatedPartitions() && partitionDescription.isUnderReplicated();
+        }
+
+        private boolean shouldPrintUnavailablePartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnavailablePartitions() && partitionDescription.hasUnavailablePartitions(liveBrokers);
+        }
+
+        private boolean shouldPrintUnderMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderMinIsrPartitions() && partitionDescription.isUnderMinIsr();
+        }
+
+        private boolean shouldPrintAtMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportAtMinIsrPartitions() && partitionDescription.isAtMinIsrPartitions();
+        }
+
+        private boolean shouldPrintTopicPartition(PartitionDescription partitionDesc) {
+            return describeConfigs ||
+                shouldPrintUnderReplicatedPartitions(partitionDesc) ||
+                shouldPrintUnavailablePartitions(partitionDesc) ||
+                shouldPrintUnderMinIsrPartitions(partitionDesc) ||
+                shouldPrintAtMinIsrPartitions(partitionDesc);
+        }
+
+        public void maybePrintPartitionDescription(PartitionDescription desc) {
+            if (shouldPrintTopicPartition(desc)) {
+                desc.printDescription();
+            }
+        }
+    }
+
+    public static class TopicService implements AutoCloseable {
+        private Admin adminClient;
+
+        public TopicService(Properties commandConfig, Optional<String> bootstrapServer) {
+            this.adminClient = createAdminClient(commandConfig, bootstrapServer);
+        }
+
+        public TopicService(Admin admin) {
+            this.adminClient = admin;
+        }
+
+        private static Admin createAdminClient(Properties commandConfig, Optional<String> bootstrapServer) {
+            if (bootstrapServer.isPresent()) {
+                commandConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer.get());
+            }
+            return Admin.create(commandConfig);
+        }
+
+        public void createTopic(TopicCommandOptions opts) throws Exception {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            if (Topic.hasCollisionChars(topic.name.get())) {
+                System.out.println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could " +
+                    "collide. To avoid issues it is best to use either, but not both.");
+            }
+            createTopic(topic);
+        }
+
+        public void createTopic(CommandTopicPartition topic) throws Exception {
+            if (topic.replicationFactor.filter(rf -> rf > Short.MAX_VALUE || rf < 1).isPresent()) {
+                throw new IllegalArgumentException("The replication factor must be between 1 and " + Short.MAX_VALUE + " inclusive");
+            }
+            if (topic.partitions.filter(p -> p < 1).isPresent()) {
+                throw new IllegalArgumentException("The partitions must be greater than 0");
+            }
+
+            try {
+                NewTopic newTopic;
+                if (topic.hasReplicaAssignment()) {
+                    newTopic = new NewTopic(topic.name.get(), topic.replicaAssignment);
+                } else {
+                    newTopic = new NewTopic(topic.name.get(), topic.partitions, topic.replicationFactor.map(Integer::shortValue));
+                }
+
+                Map<String, String> configsMap = topic.configsToAdd.stringPropertyNames().stream()
+                    .collect(Collectors.toMap(name -> name, name -> topic.configsToAdd.getProperty(name)));
+
+                newTopic.configs(configsMap);
+                CreateTopicsResult createResult = adminClient.createTopics(Collections.singleton(newTopic),
+                    new CreateTopicsOptions().retryOnQuotaViolation(false));
+                createResult.all().get();
+                System.out.println("Created topic " + topic.name + ".");
+            } catch (ExecutionException e) {
+                if (e.getCause() == null) {
+                    throw e;
+                }
+                if (!(e.getCause() instanceof TopicExistsException && topic.ifTopicDoesntExist())) {
+                    throw (Exception) e.getCause();
+                }
+            }
+        }
+
+        public void listTopics(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            String results = getTopics(opts.topic(), opts.excludeInternalTopics())
+                .stream()
+                .collect(Collectors.joining("\n"));
+            System.out.println(results);
+        }
+
+        public void alterTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+
+            if (!topics.isEmpty()) {
+                Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo = adminClient.describeTopics(topics).topicNameValues();
+                Map<String, NewPartitions> newPartitions = topics.stream()
+                    .map(topicName -> topicNewPartitions(topic, topicsInfo, topicName))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                adminClient.createPartitions(newPartitions, new CreatePartitionsOptions().retryOnQuotaViolation(false)).all().get();
+            }
+        }
+
+        private AbstractMap.SimpleEntry<String, NewPartitions> topicNewPartitions(
+            CommandTopicPartition topic,
+            Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo,
+            String topicName) {
+            if (topic.hasReplicaAssignment()) {
+                try {
+                    Integer startPartitionId = topicsInfo.get(topicName).get().partitions().size();
+                    Map<Integer, List<Integer>> replicaMap = topic.replicaAssignment.entrySet().stream()
+                        .skip(startPartitionId)
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                    List<List<Integer>> newAssignment = replicaMap.entrySet().stream()

Review Comment:
   I think this can be simplified into:
   ```
   List<List<Integer>> newAssignment = new ArrayList<>(replicaMap.values());
   ```



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java:
##########
@@ -0,0 +1,1050 @@
+/*
+ * 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;
+
+import kafka.admin.RackAwareTest;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Tag("integration")
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
+public class TopicCommandIntegrationTest extends kafka.integration.KafkaServerTestHarness implements Logging, RackAwareTest {
+    private Short defaultReplicationFactor = 1;
+    private Integer numPartitions = 1;
+    private TopicCommand.TopicService topicService;
+    private Admin adminClient;
+    private String bootstrapServer;
+    private String testTopicName;
+    private Integer defaultTimeout = 10000;
+
+    /**
+     * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every
+     * test and should not reuse previous configurations unless they select their ports randomly when servers are started.
+     *
+     * Note the replica fetch max bytes is set to `1` in order to throttle the rate of replication for test
+     * `testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress`.
+     */
+    @Override
+    public scala.collection.Seq<KafkaConfig> generateConfigs() {
+        Map<Integer, String> rackInfo = new HashMap<>();
+        rackInfo.put(0, "rack1");
+        rackInfo.put(1, "rack2");
+        rackInfo.put(2, "rack2");
+        rackInfo.put(3, "rack1");
+        rackInfo.put(4, "rack3");
+        rackInfo.put(5, "rack3");
+
+        List<Properties> brokerConfigs = ToolsTestUtils
+            .createBrokerProperties(6, zkConnectOrNull(), rackInfo, numPartitions, defaultReplicationFactor);
+
+        List<KafkaConfig> configs = new ArrayList<>();
+        for (Properties props : brokerConfigs) {
+            props.put(KafkaConfig.ReplicaFetchMaxBytesProp(), "1");
+            configs.add(KafkaConfig.fromProps(props));
+        }
+        return JavaConverters.asScalaBuffer(configs).toSeq();
+    }
+
+    private TopicCommand.TopicCommandOptions buildTopicCommandOptionsWithBootstrap(String... opts) {
+        String[] finalOptions = Stream.concat(Arrays.asList(opts).stream(),
+                Arrays.asList("--bootstrap-server", bootstrapServer).stream()
+        ).toArray(String[]::new);
+        return new TopicCommand.TopicCommandOptions(finalOptions);
+    }
+
+    private void createAndWaitTopic(TopicCommand.TopicCommandOptions opts) throws Exception {
+        topicService.createTopic(opts);
+        waitForTopicCreated(opts.topic().get());
+    }
+
+    private void waitForTopicCreated(String topicName) {
+        waitForTopicCreated(topicName, defaultTimeout);
+    }
+
+    private void waitForTopicCreated(String topicName, Integer timeout) {
+        TestUtils.waitForPartitionMetadata(brokers(), topicName, 0, timeout);
+    }
+
+    @BeforeEach
+    public void setUp(TestInfo info) {
+        super.setUp(info);
+        // create adminClient
+        Properties props = new Properties();
+        bootstrapServer = bootstrapServers(listenerName());
+        props.put(org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+        adminClient = Admin.create(props);
+        topicService = new TopicCommand.TopicService(props, Optional.of(bootstrapServer));
+        testTopicName = String.format("%s-%s", info.getTestMethod().get().getName(),
+            new scala.util.Random().alphanumeric().take(10).mkString());
+    }
+
+    @AfterEach
+    public void close() throws Exception {
+        if (topicService != null)
+            topicService.close();
+        if (adminClient != null)
+            adminClient.close();
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreate(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", "2", "--replication-factor", "1", "--topic", testTopicName));
+
+        adminClient.listTopics().names().get().contains(testTopicName);

Review Comment:
   Should we check the return value with `assertTrue()`?



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java:
##########
@@ -0,0 +1,1050 @@
+/*
+ * 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;
+
+import kafka.admin.RackAwareTest;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Tag("integration")
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
+public class TopicCommandIntegrationTest extends kafka.integration.KafkaServerTestHarness implements Logging, RackAwareTest {
+    private Short defaultReplicationFactor = 1;
+    private Integer numPartitions = 1;
+    private TopicCommand.TopicService topicService;
+    private Admin adminClient;
+    private String bootstrapServer;
+    private String testTopicName;
+    private Integer defaultTimeout = 10000;
+
+    /**
+     * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every
+     * test and should not reuse previous configurations unless they select their ports randomly when servers are started.
+     *
+     * Note the replica fetch max bytes is set to `1` in order to throttle the rate of replication for test
+     * `testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress`.
+     */
+    @Override
+    public scala.collection.Seq<KafkaConfig> generateConfigs() {
+        Map<Integer, String> rackInfo = new HashMap<>();
+        rackInfo.put(0, "rack1");
+        rackInfo.put(1, "rack2");
+        rackInfo.put(2, "rack2");
+        rackInfo.put(3, "rack1");
+        rackInfo.put(4, "rack3");
+        rackInfo.put(5, "rack3");
+
+        List<Properties> brokerConfigs = ToolsTestUtils
+            .createBrokerProperties(6, zkConnectOrNull(), rackInfo, numPartitions, defaultReplicationFactor);
+
+        List<KafkaConfig> configs = new ArrayList<>();
+        for (Properties props : brokerConfigs) {
+            props.put(KafkaConfig.ReplicaFetchMaxBytesProp(), "1");
+            configs.add(KafkaConfig.fromProps(props));
+        }
+        return JavaConverters.asScalaBuffer(configs).toSeq();
+    }
+
+    private TopicCommand.TopicCommandOptions buildTopicCommandOptionsWithBootstrap(String... opts) {
+        String[] finalOptions = Stream.concat(Arrays.asList(opts).stream(),
+                Arrays.asList("--bootstrap-server", bootstrapServer).stream()
+        ).toArray(String[]::new);
+        return new TopicCommand.TopicCommandOptions(finalOptions);
+    }
+
+    private void createAndWaitTopic(TopicCommand.TopicCommandOptions opts) throws Exception {
+        topicService.createTopic(opts);
+        waitForTopicCreated(opts.topic().get());
+    }
+
+    private void waitForTopicCreated(String topicName) {
+        waitForTopicCreated(topicName, defaultTimeout);
+    }
+
+    private void waitForTopicCreated(String topicName, Integer timeout) {
+        TestUtils.waitForPartitionMetadata(brokers(), topicName, 0, timeout);
+    }
+
+    @BeforeEach
+    public void setUp(TestInfo info) {
+        super.setUp(info);
+        // create adminClient
+        Properties props = new Properties();
+        bootstrapServer = bootstrapServers(listenerName());
+        props.put(org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+        adminClient = Admin.create(props);
+        topicService = new TopicCommand.TopicService(props, Optional.of(bootstrapServer));
+        testTopicName = String.format("%s-%s", info.getTestMethod().get().getName(),
+            new scala.util.Random().alphanumeric().take(10).mkString());

Review Comment:
   It's likely going to be a bit longer but could we replace this Scala snippet by Java?



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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;
+
+import kafka.utils.Exit;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientTestUtils;
+import org.apache.kafka.clients.admin.CreatePartitionsResult;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.DeleteTopicsResult;
+import org.apache.kafka.clients.admin.DescribeTopicsResult;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.errors.ThrottlingQuotaExceededException;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyCollection;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.verify;
+
+@Timeout(value = 60)
+public class TopicCommandTest {
+    private String bootstrapServer = "localhost:9092";
+    private String topicName = "topicName";
+
+    @Test
+    public void testIsNotUnderReplicatedWhenAdding() {
+        List<Integer> replicaIds = Arrays.asList(1, 2);
+        List<Node> replicas = new ArrayList<>();
+        for (int id : replicaIds) {
+            replicas.add(new Node(id, "localhost", 9090 + id));
+        }
+
+        TopicCommand.PartitionDescription partitionDescription = new TopicCommand.PartitionDescription("test-topic",
+            new TopicPartitionInfo(0, new Node(1, "localhost", 9091), replicas,
+                Collections.singletonList(new Node(1, "localhost", 9091))),
+            null, false,
+                new PartitionReassignment(replicaIds, Arrays.asList(2), Collections.emptyList())
+        );
+
+        assertFalse(partitionDescription.isUnderReplicated());
+    }
+
+    @Test
+    public void testAlterWithUnspecifiedPartitionCount() {
+        String[] options = new String[] {" --bootstrap-server", bootstrapServer, "--alter", "--topic", topicName};
+        assertInitializeInvalidOptionsExitCode(1, options);
+    }
+
+    @Test
+    public void testConfigOptWithBootstrapServers() {
+        assertInitializeInvalidOptionsExitCode(1,
+            new String[] {"--bootstrap-server", bootstrapServer, "--alter", "--topic", topicName,
+                "--partitions", "3", "--config", "cleanup.policy=compact"});
+        assertInitializeInvalidOptionsExitCode(1,
+            new String[] {"--bootstrap-server", bootstrapServer, "--alter", "--topic", topicName,
+                "--partitions", "3", "--delete-config", "cleanup.policy"});
+        TopicCommand.TopicCommandOptions opts =
+            new TopicCommand.TopicCommandOptions(
+                new String[] {"--bootstrap-server", bootstrapServer, "--create", "--topic", topicName, "--partitions", "3",
+                    "--replication-factor", "3", "--config", "cleanup.policy=compact"});
+        assertTrue(opts.hasCreateOption());
+        assertEquals(bootstrapServer, opts.bootstrapServer().get());
+        assertEquals("cleanup.policy=compact", opts.topicConfig().get().get(0));
+    }
+
+    @Test
+    public void testCreateWithPartitionCountWithoutReplicationFactorShouldSucceed() {
+        TopicCommand.TopicCommandOptions opts = new TopicCommand.TopicCommandOptions(
+            new String[] {"--bootstrap-server", bootstrapServer,
+                "--create",
+                "--partitions", "2",
+                "--topic", topicName});
+        assertTrue(opts.hasCreateOption());
+        assertEquals(topicName, opts.topic().get());
+        assertEquals(2, opts.partitions().get());
+    }
+
+    @Test
+    public void testCreateWithReplicationFactorWithoutPartitionCountShouldSucceed() {
+        TopicCommand.TopicCommandOptions opts = new TopicCommand.TopicCommandOptions(
+            new String[] {"--bootstrap-server", bootstrapServer,
+                "--create",
+                "--replication-factor", "3",
+                "--topic", topicName});
+        assertTrue(opts.hasCreateOption());
+        assertEquals(topicName, opts.topic().get());
+        assertEquals(3, opts.replicationFactor().get());
+    }
+
+    @Test
+    public void testCreateWithAssignmentAndPartitionCount() {
+        assertInitializeInvalidOptionsExitCode(1,
+            new String[]{"--bootstrap-server", bootstrapServer,
+                "--create",
+                "--replica-assignment", "3:0,5:1",
+                "--partitions", "2",
+                "--topic", topicName});
+    }
+
+    @Test
+    public void testCreateWithAssignmentAndReplicationFactor() {
+        assertInitializeInvalidOptionsExitCode(1,
+            new String[] {"--bootstrap-server", bootstrapServer,
+                "--create",
+                "--replica-assignment", "3:0,5:1",
+                "--replication-factor", "2",
+                "--topic", topicName});
+    }
+
+    @Test
+    public void testCreateWithoutPartitionCountAndReplicationFactorShouldSucceed() {
+        TopicCommand.TopicCommandOptions opts = new TopicCommand.TopicCommandOptions(
+            new String[] {"--bootstrap-server", bootstrapServer,
+                "--create",
+                "--topic", topicName});
+        assertTrue(opts.hasCreateOption());
+        assertEquals(topicName, opts.topic().get());
+        assertFalse(opts.partitions().isPresent());
+    }
+
+    @Test
+    public void testDescribeShouldSucceed() {
+        TopicCommand.TopicCommandOptions opts = new TopicCommand.TopicCommandOptions(
+            new String[] {"--bootstrap-server", bootstrapServer,
+                "--describe",
+                "--topic", topicName});
+        assertTrue(opts.hasDescribeOption());
+        assertEquals(topicName, opts.topic().get());
+    }
+
+
+    @Test
+    public void testParseAssignmentDuplicateEntries() {
+        assertThrows(AdminCommandFailedException.class, () -> TopicCommand.parseReplicaAssignment("5:5"));
+    }
+
+    @Test
+    public void testParseAssignmentPartitionsOfDifferentSize() {
+        assertThrows(AdminOperationException.class, () -> TopicCommand.parseReplicaAssignment("5:4:3,2:1"));
+    }
+
+    @Test
+    public void testParseAssignment() {
+        Map<Integer, List<Integer>> actualAssignment = TopicCommand.parseReplicaAssignment("5:4,3:2,1:0");
+        Map<Integer, List<Integer>>  expectedAssignment = new HashMap<Integer, List<Integer>>();

Review Comment:
   We can remove the types on the right



##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java:
##########
@@ -0,0 +1,1050 @@
+/*
+ * 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;
+
+import kafka.admin.RackAwareTest;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Tag("integration")
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
+public class TopicCommandIntegrationTest extends kafka.integration.KafkaServerTestHarness implements Logging, RackAwareTest {
+    private Short defaultReplicationFactor = 1;
+    private Integer numPartitions = 1;
+    private TopicCommand.TopicService topicService;
+    private Admin adminClient;
+    private String bootstrapServer;
+    private String testTopicName;
+    private Integer defaultTimeout = 10000;
+
+    /**
+     * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every
+     * test and should not reuse previous configurations unless they select their ports randomly when servers are started.
+     *
+     * Note the replica fetch max bytes is set to `1` in order to throttle the rate of replication for test
+     * `testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress`.
+     */
+    @Override
+    public scala.collection.Seq<KafkaConfig> generateConfigs() {
+        Map<Integer, String> rackInfo = new HashMap<>();
+        rackInfo.put(0, "rack1");
+        rackInfo.put(1, "rack2");
+        rackInfo.put(2, "rack2");
+        rackInfo.put(3, "rack1");
+        rackInfo.put(4, "rack3");
+        rackInfo.put(5, "rack3");
+
+        List<Properties> brokerConfigs = ToolsTestUtils
+            .createBrokerProperties(6, zkConnectOrNull(), rackInfo, numPartitions, defaultReplicationFactor);
+
+        List<KafkaConfig> configs = new ArrayList<>();
+        for (Properties props : brokerConfigs) {
+            props.put(KafkaConfig.ReplicaFetchMaxBytesProp(), "1");
+            configs.add(KafkaConfig.fromProps(props));
+        }
+        return JavaConverters.asScalaBuffer(configs).toSeq();
+    }
+
+    private TopicCommand.TopicCommandOptions buildTopicCommandOptionsWithBootstrap(String... opts) {
+        String[] finalOptions = Stream.concat(Arrays.asList(opts).stream(),
+                Arrays.asList("--bootstrap-server", bootstrapServer).stream()
+        ).toArray(String[]::new);
+        return new TopicCommand.TopicCommandOptions(finalOptions);
+    }
+
+    private void createAndWaitTopic(TopicCommand.TopicCommandOptions opts) throws Exception {
+        topicService.createTopic(opts);
+        waitForTopicCreated(opts.topic().get());
+    }
+
+    private void waitForTopicCreated(String topicName) {
+        waitForTopicCreated(topicName, defaultTimeout);
+    }
+
+    private void waitForTopicCreated(String topicName, Integer timeout) {
+        TestUtils.waitForPartitionMetadata(brokers(), topicName, 0, timeout);
+    }
+
+    @BeforeEach
+    public void setUp(TestInfo info) {
+        super.setUp(info);
+        // create adminClient
+        Properties props = new Properties();
+        bootstrapServer = bootstrapServers(listenerName());
+        props.put(org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+        adminClient = Admin.create(props);
+        topicService = new TopicCommand.TopicService(props, Optional.of(bootstrapServer));
+        testTopicName = String.format("%s-%s", info.getTestMethod().get().getName(),
+            new scala.util.Random().alphanumeric().take(10).mkString());
+    }
+
+    @AfterEach
+    public void close() throws Exception {
+        if (topicService != null)
+            topicService.close();
+        if (adminClient != null)
+            adminClient.close();
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreate(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", "2", "--replication-factor", "1", "--topic", testTopicName));
+
+        adminClient.listTopics().names().get().contains(testTopicName);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithDefaults(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName));
+
+        List<TopicPartitionInfo> partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+        assertEquals(partitions.size(), numPartitions);
+        assertEquals((short) partitions.get(0).replicas().size(), defaultReplicationFactor);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithDefaultReplication(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName, "--partitions", "2"));
+
+        List<TopicPartitionInfo>  partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+        assertEquals(partitions.size(), 2);
+        assertEquals((short) partitions.get(0).replicas().size(), defaultReplicationFactor);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithDefaultPartitions(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName, "--replication-factor", "2"));
+
+        List<TopicPartitionInfo> partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+
+        assertEquals(partitions.size(), numPartitions);
+        assertEquals((short) partitions.get(0).replicas().size(), 2);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithConfigs(String quorum) throws Exception {
+        ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName);
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "2", "--replication-factor", "2", "--topic", testTopicName, "--config",
+                "delete.retention.ms=1000"));
+
+        Config configs = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource);
+        assertEquals(1000, Integer.valueOf(configs.get("delete.retention.ms").value()));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWhenAlreadyExists(String quorum) throws Exception {
+        int numPartitions = 1;
+
+        // create the topic
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", Integer.toString(numPartitions), "--replication-factor", "1",
+                "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+
+        // try to re-create the topic
+        assertThrows(TopicExistsException.class, () -> topicService.createTopic(createOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWhenAlreadyExistsWithIfNotExists(String quorum) throws Exception {
+        TopicCommand.TopicCommandOptions createOpts =
+                buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName, "--if-not-exists");
+        createAndWaitTopic(createOpts);
+        topicService.createTopic(createOpts);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithReplicaAssignment(String quorum) throws Exception {
+        // create the topic
+        TopicCommand.TopicCommandOptions createOpts =
+                buildTopicCommandOptionsWithBootstrap("--create", "--replica-assignment", "5:4,3:2,1:0", "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+
+        List<TopicPartitionInfo> partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+        
+        assertEquals(3, partitions.size());
+        assertEquals(Arrays.asList(5, 4), getPartitionReplicas(partitions, 0));
+        assertEquals(Arrays.asList(3, 2), getPartitionReplicas(partitions, 1));
+        assertEquals(Arrays.asList(1, 0), getPartitionReplicas(partitions, 2));
+    }
+
+    private List<Integer> getPartitionReplicas(List<TopicPartitionInfo> partitions, int partitionNumber) {
+        return partitions.get(partitionNumber).replicas().stream().map(Node::id).collect(Collectors.toList());
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithInvalidReplicationFactor(String quorum) {
+        TopicCommand.TopicCommandOptions opts = buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "2", "--replication-factor", Integer.toString(Short.MAX_VALUE + 1),
+            "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.createTopic(opts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithNegativeReplicationFactor(String quorum) {
+        TopicCommand.TopicCommandOptions opts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "2", "--replication-factor", "-1", "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.createTopic(opts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithNegativePartitionCount(String quorum) {
+        TopicCommand.TopicCommandOptions opts = buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "-1", "--replication-factor", "1", "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.createTopic(opts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testInvalidTopicLevelConfig(String quorum) {
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "1", "--replication-factor", "1", "--topic", testTopicName,
+            "--config", "message.timestamp.type=boom");
+        assertThrows(ConfigException.class, () -> topicService.createTopic(createOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListTopics(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", "1", "--replication-factor", "1", "--topic", testTopicName));
+
+        String output = captureListTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--list"));
+        assertTrue(output.contains(testTopicName));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListTopicsWithIncludeList(String quorum) throws ExecutionException, InterruptedException {
+        String topic1 = "kafka.testTopic1";
+        String topic2 = "kafka.testTopic2";
+        String topic3 = "oooof.testTopic1";
+        adminClient.createTopics(
+                Arrays.asList(new NewTopic(topic1, 2, (short) 2),
+                    new NewTopic(topic2, 2, (short) 2),
+                    new NewTopic(topic3, 2, (short) 2)))
+            .all().get();
+        waitForTopicCreated(topic1);
+        waitForTopicCreated(topic2);
+        waitForTopicCreated(topic3);
+
+        String output = captureListTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--list", "--topic", "kafka.*"));
+
+        assertTrue(output.contains(topic1));
+        assertTrue(output.contains(topic2));
+        assertFalse(output.contains(topic3));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListTopicsWithExcludeInternal(String quorum) throws ExecutionException, InterruptedException {
+        String topic1 = "kafka.testTopic1";
+        adminClient.createTopics(
+                Arrays.asList(new NewTopic(topic1, 2, (short) 2),
+                    new NewTopic(Topic.GROUP_METADATA_TOPIC_NAME, 2, (short) 2)))
+            .all().get();
+        waitForTopicCreated(topic1);
+
+        String output = captureListTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--list", "--exclude-internal"));
+
+        assertTrue(output.contains(topic1));
+        assertFalse(output.contains(Topic.GROUP_METADATA_TOPIC_NAME));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterPartitionCount(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Arrays.asList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter", "--topic", testTopicName, "--partitions", "3"));
+
+        kafka.utils.TestUtils.waitUntilTrue(
+            () -> brokers().forall(b -> b.metadataCache().getTopicPartitions(testTopicName).size() == 3),
+            () -> "Timeout waiting for new assignment propagating to broker", org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+        TopicDescription topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get();
+        assertTrue(topicDescription.partitions().size() == 3);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterAssignment(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter",
+            "--topic", testTopicName, "--replica-assignment", "5:3,3:1,4:2", "--partitions", "3"));
+        kafka.utils.TestUtils.waitUntilTrue(
+            () -> brokers().forall(b -> b.metadataCache().getTopicPartitions(testTopicName).size() == 3),
+            () -> "Timeout waiting for new assignment propagating to broker",
+            org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+
+        TopicDescription topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get();
+        assertTrue(topicDescription.partitions().size() == 3);
+        List<Integer> partitionReplicas = getPartitionReplicas(topicDescription.partitions(), 2);
+        assertEquals(Arrays.asList(4, 2), partitionReplicas);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterAssignmentWithMoreAssignmentThanPartitions(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Arrays.asList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        assertThrows(ExecutionException.class,
+            () -> topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter",
+                "--topic", testTopicName, "--replica-assignment", "5:3,3:1,4:2,3:2", "--partitions", "3")));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterAssignmentWithMorePartitionsThanAssignment(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Arrays.asList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        assertThrows(ExecutionException.class,
+            () -> topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter", "--topic", testTopicName,
+                "--replica-assignment", "5:3,3:1,4:2", "--partitions", "6")));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterWithInvalidPartitionCount(String quorum) throws Exception {
+        createAndWaitTopic(
+                buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "1", "--replication-factor", "1", "--topic", testTopicName)
+        );
+
+        assertThrows(ExecutionException.class,
+            () -> topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter", "--partitions", "-1", "--topic", testTopicName)));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterWhenTopicDoesntExist(String quorum) {
+        // alter a topic that does not exist without --if-exists
+        TopicCommand.TopicCommandOptions alterOpts = buildTopicCommandOptionsWithBootstrap("--alter", "--topic", testTopicName, "--partitions", "1");
+        TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient);
+        assertThrows(IllegalArgumentException.class, () -> topicService.alterTopic(alterOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterWhenTopicDoesntExistWithIfExists(String quorum) throws ExecutionException, InterruptedException {
+        topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter", "--topic", testTopicName, "--partitions", "1", "--if-exists"));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateAlterTopicWithRackAware(String quorum) throws Exception {
+        Map<Integer, String> rackInfo = new HashMap<Integer, String>();
+        rackInfo.put(0, "rack1");
+        rackInfo.put(1, "rack2");
+        rackInfo.put(2, "rack2");
+        rackInfo.put(3, "rack1");
+        rackInfo.put(4, "rack3");
+        rackInfo.put(5, "rack3");
+
+        int numPartitions = 18;
+        int replicationFactor = 3;
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", Integer.toString(numPartitions),
+            "--replication-factor", Integer.toString(replicationFactor),
+            "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+
+        Map<Integer, List<Integer>> assignment = adminClient.describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames().get().get(testTopicName).partitions()
+            .stream()
+            .collect(Collectors.toMap(
+                info -> info.partition(),
+                info -> info.replicas().stream().map(Node::id).collect(Collectors.toList())));
+        checkReplicaDistribution(assignment, rackInfo, rackInfo.size(), numPartitions,
+            replicationFactor, true, true, true);
+
+        int alteredNumPartitions = 36;
+        // verify that adding partitions will also be rack aware
+        TopicCommand.TopicCommandOptions alterOpts = buildTopicCommandOptionsWithBootstrap("--alter",
+            "--partitions", Integer.toString(alteredNumPartitions),
+            "--topic", testTopicName);
+        topicService.alterTopic(alterOpts);
+
+        kafka.utils.TestUtils.waitUntilTrue(
+            () -> brokers().forall(p -> p.metadataCache().getTopicPartitions(testTopicName).size() == alteredNumPartitions),
+            () -> "Timeout waiting for new assignment propagating to broker",
+            org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+
+        assignment = adminClient.describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames().get().get(testTopicName).partitions().stream()
+            .collect(Collectors.toMap(info -> info.partition(), info -> info.replicas().stream().map(Node::id).collect(Collectors.toList())));
+        checkReplicaDistribution(assignment, rackInfo, rackInfo.size(), alteredNumPartitions, replicationFactor,
+            true, true, true);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testConfigPreservationAcrossPartitionAlteration(String quorum) throws Exception {
+        int numPartitionsOriginal = 1;
+        String cleanupKey = "cleanup.policy";
+        String cleanupVal = "compact";
+
+        // create the topic
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", Integer.toString(numPartitionsOriginal),
+            "--replication-factor", "1",
+            "--config", cleanupKey + "=" + cleanupVal,
+            "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+        ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName);
+        Config props = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource);
+        // val props = adminZkClient.fetchEntityConfig(ConfigType.Topic, testTopicName)
+        assertNotNull(props.get(cleanupKey), "Properties after creation don't contain " + cleanupKey);
+        assertEquals(cleanupVal, props.get(cleanupKey).value(), "Properties after creation have incorrect value");
+
+        // pre-create the topic config changes path to avoid a NoNodeException
+        if (!isKRaftTest()) {
+            zkClient().makeSurePersistentPathExists(kafka.zk.ConfigEntityChangeNotificationZNode.path());
+        }
+
+        // modify the topic to add new partitions
+        int numPartitionsModified = 3;
+        TopicCommand.TopicCommandOptions alterOpts = buildTopicCommandOptionsWithBootstrap("--alter",
+            "--partitions", Integer.toString(numPartitionsModified), "--topic", testTopicName);
+        topicService.alterTopic(alterOpts);
+        Config newProps = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource);
+        assertNotNull(newProps.get(cleanupKey), "Updated properties do not contain " + cleanupKey);
+        assertEquals(cleanupVal, newProps.get(cleanupKey).value(), "Updated properties have incorrect value");
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testTopicDeletion(String quorum) throws Exception {
+        // create the NormalTopic
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "1",
+            "--replication-factor", "1",
+            "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+
+        // delete the NormalTopic
+        TopicCommand.TopicCommandOptions deleteOpts = buildTopicCommandOptionsWithBootstrap("--delete", "--topic", testTopicName);
+
+        if (!isKRaftTest()) {
+            String deletePath = kafka.zk.DeleteTopicsTopicZNode.path(testTopicName);
+            assertFalse(zkClient().pathExists(deletePath), "Delete path for topic shouldn't exist before deletion.");
+        }
+        topicService.deleteTopic(deleteOpts);
+        TestUtils.verifyTopicDeletion(zkClientOrNull(), testTopicName, 1, brokers());
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testTopicWithCollidingCharDeletionAndCreateAgain(String quorum) throws Exception {
+        // create the topic with colliding chars
+        String topicWithCollidingChar = "test.a";
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "1",
+            "--replication-factor", "1",
+            "--topic", topicWithCollidingChar);
+        createAndWaitTopic(createOpts);
+
+        // delete the topic
+        TopicCommand.TopicCommandOptions deleteOpts = buildTopicCommandOptionsWithBootstrap("--delete", "--topic", topicWithCollidingChar);
+
+        if (!isKRaftTest()) {
+            String deletePath = kafka.zk.DeleteTopicsTopicZNode.path(topicWithCollidingChar);
+            assertFalse(zkClient().pathExists(deletePath), "Delete path for topic shouldn't exist before deletion.");
+        }
+        topicService.deleteTopic(deleteOpts);
+        TestUtils.verifyTopicDeletion(zkClientOrNull(), topicWithCollidingChar, 1, brokers());
+        assertDoesNotThrow(() -> createAndWaitTopic(createOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteInternalTopic(String quorum) throws Exception {
+        // create the offset topic
+        TopicCommand.TopicCommandOptions createOffsetTopicOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "1",
+            "--replication-factor", "1",
+            "--topic", Topic.GROUP_METADATA_TOPIC_NAME);
+        createAndWaitTopic(createOffsetTopicOpts);
+
+        // Try to delete the Topic.GROUP_METADATA_TOPIC_NAME which is allowed by default.
+        // This is a difference between the new and the old command as the old one didn't allow internal topic deletion.
+        // If deleting internal topics is not desired, ACLS should be used to control it.
+        TopicCommand.TopicCommandOptions deleteOffsetTopicOpts =
+                buildTopicCommandOptionsWithBootstrap("--delete", "--topic", Topic.GROUP_METADATA_TOPIC_NAME);
+        String deleteOffsetTopicPath = kafka.zk.DeleteTopicsTopicZNode.path(Topic.GROUP_METADATA_TOPIC_NAME);
+        if (!isKRaftTest()) {
+            assertFalse(zkClient().pathExists(deleteOffsetTopicPath), "Delete path for topic shouldn't exist before deletion.");
+        }
+        topicService.deleteTopic(deleteOffsetTopicOpts);
+        TestUtils.verifyTopicDeletion(zkClientOrNull(), Topic.GROUP_METADATA_TOPIC_NAME, 1, brokers());
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteWhenTopicDoesntExist(String quorum) {
+        // delete a topic that does not exist
+        TopicCommand.TopicCommandOptions deleteOpts = buildTopicCommandOptionsWithBootstrap("--delete", "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.deleteTopic(deleteOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteWhenTopicDoesntExistWithIfExists(String quorum) throws ExecutionException, InterruptedException {
+        topicService.deleteTopic(buildTopicCommandOptionsWithBootstrap("--delete", "--topic", testTopicName, "--if-exists"));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribe(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName));
+        String[] rows = output.split("\n");
+        assertEquals(3, rows.length);
+        assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeWhenTopicDoesntExist(String quorum) {
+        assertThrows(IllegalArgumentException.class,
+            () -> topicService.describeTopic(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName)));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeWhenTopicDoesntExistWithIfExists(String quorum) throws ExecutionException, InterruptedException {
+        topicService.describeTopic(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName, "--if-exists"));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeUnavailablePartitions(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 6, (short) 1))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        try {
+            // check which partition is on broker 0 which we'll kill
+            TopicDescription testTopicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName))
+                .allTopicNames().get().get(testTopicName);
+            int partitionOnBroker0 = testTopicDescription.partitions().stream()
+                .filter(partition -> partition.leader().id() == 0)
+                .findFirst().get().partition();
+
+            killBroker(0);
+
+            // wait until the topic metadata for the test topic is propagated to each alive broker
+            kafka.utils.TestUtils.waitUntilTrue(
+                () -> {
+                    boolean result = true;
+                    for (KafkaBroker server : JavaConverters.asJavaCollection(brokers())) {
+                        if (server.config().brokerId() != 0) {
+                            Set<String> topicNames = Collections.singleton(testTopicName);
+                            Collection<MetadataResponseData.MetadataResponseTopic> topicMetadatas =
+                                JavaConverters.asJavaCollection(server.dataPlaneRequestProcessor().metadataCache()
+                                .getTopicMetadata(JavaConverters.asScalaSetConverter(topicNames).asScala().toSet(),
+                                    ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
+                                    false, false)
+                                );
+                            Optional<MetadataResponseData.MetadataResponseTopic> testTopicMetadata = topicMetadatas.stream()
+                                .filter(metadata -> metadata.name().equals(testTopicName))
+                                .findFirst();
+                            if (!testTopicMetadata.isPresent()) {
+                                throw new AssertionError("Topic metadata is not found in metadata cache");
+                            }
+                            Optional<MetadataResponseData.MetadataResponsePartition> testPartitionMetadata = testTopicMetadata.get().partitions().stream()
+                                .filter(metadata -> metadata.partitionIndex() == partitionOnBroker0)
+                                .findFirst();
+                            if (!testPartitionMetadata.isPresent()) {
+                                throw new AssertionError("Partition metadata is not found in metadata cache");
+                            }
+                            result = result && testPartitionMetadata.get().errorCode() == Errors.LEADER_NOT_AVAILABLE.code();
+                        }
+                    }
+                    return result;
+                },
+                () -> String.format("Partition metadata for %s is not propagated", testTopicName),
+                org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+
+            // grab the console output and assert
+            String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName, "--unavailable-partitions"));
+            String[] rows = output.split("\n");
+            assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)));
+            assertTrue(rows[0].contains("Leader: none\tReplicas: 0\tIsr:"));
+        } finally {
+            restartDeadBrokers(false);
+        }
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeUnderReplicatedPartitions(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 1, (short) 6))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        try {
+            killBroker(0);
+            if (isKRaftTest()) {
+                ensureConsistentKRaftMetadata();
+            } else {
+                TestUtils.waitForPartitionMetadata(aliveBrokers(), testTopicName, 0, org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS);
+            }
+            String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--under-replicated-partitions"));
+            String[] rows = output.split("\n");
+            assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)), String.format("Unexpected output: %s", rows[0]));
+        } finally {
+            restartDeadBrokers(false);
+        }
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeUnderMinIsrPartitions(String quorum) throws ExecutionException, InterruptedException {
+        Map<String, String> configMap = new HashMap<>();
+        configMap.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "6");
+
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 1, (short) 6).configs(configMap))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        try {
+            killBroker(0);
+            if (isKRaftTest()) {
+                ensureConsistentKRaftMetadata();
+            } else {
+                kafka.utils.TestUtils.waitUntilTrue(
+                    () -> aliveBrokers().forall(b -> b.metadataCache().getPartitionInfo(testTopicName, 0).get().isr().size() == 5),
+                    () -> String.format("Timeout waiting for partition metadata propagating to brokers for %s topic", testTopicName),
+                    org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L
+                );
+            }
+            String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--under-min-isr-partitions"));
+            String[] rows = output.split("\n");
+            assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)));
+        } finally {
+            restartDeadBrokers(false);
+        }
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress(String quorum) throws ExecutionException, InterruptedException {
+        Map<String, String> configMap = new HashMap<>();
+        Short replicationFactor = 1;

Review Comment:
   We can use `short` here



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1717601042

   @fvaleri It turns out that we are hitting a similar problem to this gradle [issue#847](https://github.com/gradle/gradle/issues/847) that causing transitive dependency problems where `storage:api` and `connect:api` were causing unintended conflicts. The gradle issue seems to not be 100% solved so I used a workaround to rename the `storage:api` project to `storage:storage-api`. This shouldn't impact the name of the final jar.  


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1713875320

   > java.lang.NoClassDefFoundError: org/apache/kafka/server/log/remote/storage/RemoteStorageException
   
   I am getting this error with all branches once I move the tools outside the core. I also noticed that `PlaintextAdminIntegrationTest` gets this error as well and I am not sure why. Details are [here](https://github.com/apache/kafka/pull/13204#issuecomment-1689824516)
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] github-actions[bot] commented on pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1585813229

   This PR is being marked as stale since it has not had any activity in 90 days. If you would like to keep this PR alive, please ask a committer for review. If the PR has  merge conflicts, please update it with the latest from trunk (or appropriate release branch)
   If this PR is no longer valid or desired, please feel free to close it. If no activity occurrs in the next 30 days, it will be automatically closed.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] fvaleri commented on pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "fvaleri (via GitHub)" <gi...@apache.org>.
fvaleri commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1627395334

   Hi @OmniaGM.
   
   > Move TopicFilter out of core. (This is part of https://issues.apache.org/jira/browse/KAFKA-14647 which seems not moving for a while )
   
   I think #13158 is ready. I'll try to drive some attention to it.
   
   > Move CoreUtils.duplicate to ToolsUtils
   > Duplicate AdminCommandFailedException and AdminOperationException out of core.
   
   I looks like `AdminCommandFailedException` and `AdminOperationException` are being moved in #13278. This also seems to be almost ready.
   
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] vamossagar12 commented on pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "vamossagar12 (via GitHub)" <gi...@apache.org>.
vamossagar12 commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1631984718

   Thanks @OmniaGM , @fvaleri . https://github.com/apache/kafka/pull/13158 has been approved by Federico. Just waiting for a committer to review/merge it.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on a diff in pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1297476446


##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String[] args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {
+            System.err.println(e.getMessage());
+            return 1;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        opts.checkArgs();
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        log.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")
+    static Map<Integer, List<Integer>> parseReplicaAssignment(String replicaAssignmentList) {
+        String[] partitionList = replicaAssignmentList.split(",");
+        Map<Integer, List<Integer>> ret = new LinkedHashMap<Integer, List<Integer>>();
+        for (int i = 0; i < partitionList.length; i++) {
+            List<Integer> brokerList = Arrays.stream(partitionList[i].split(":"))
+                .map(String::trim)
+                .mapToInt(Integer::parseInt)
+                .boxed()
+                .collect(Collectors.toList());
+            Collection<Integer> duplicateBrokers = ToolsUtils.duplicates(brokerList);
+            if (!duplicateBrokers.isEmpty()) {
+                throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: " +
+                    duplicateBrokers.stream()
+                        .map(Object::toString)
+                        .collect(Collectors.joining(","))
+                );
+            }
+            ret.put(i, brokerList);
+            if (ret.get(i).size() != ret.get(0).size()) {
+                throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList);
+            }
+        }
+        return ret;
+    }
+
+    @SuppressWarnings("deprecation")
+    private static Properties parseTopicConfigsToBeAdded(TopicCommandOptions opts) {
+        List<List<String>> configsToBeAdded = opts.topicConfig().orElse(Collections.emptyList())
+            .stream()
+            .map(s -> Arrays.asList(s.split("\\s*=\\s*")))
+            .collect(Collectors.toList());
+
+        if (!configsToBeAdded.stream().allMatch(config -> config.size() == 2)) {
+            throw new IllegalArgumentException("requirement failed: Invalid topic config: all configs to be added must be in the format \"key=val\".");
+        }
+
+        Properties props = new Properties();
+        configsToBeAdded.stream()
+            .forEach(pair -> props.setProperty(pair.get(0).trim(), pair.get(1).trim()));
+        LogConfig.validate(props);
+        if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) {
+            System.out.println("WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " +
+                "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " +
+                "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.");
+        }
+        return props;
+    }
+
+    // It is possible for a reassignment to complete between the time we have fetched its state and the time
+    // we fetch partition metadata. In this case, we ignore the reassignment when determining replication factor.
+    public static boolean isReassignmentInProgress(TopicPartitionInfo tpi, PartitionReassignment ra) {
+        // Reassignment is still in progress as long as the removing and adding replicas are still present
+        Set<Integer> allReplicaIds = tpi.replicas().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> changingReplicaIds = new HashSet<>();
+        if (ra != null) {
+            changingReplicaIds.addAll(ra.removingReplicas());
+            changingReplicaIds.addAll(ra.addingReplicas());
+        }
+        return allReplicaIds.stream().anyMatch(changingReplicaIds::contains);
+
+    }
+
+    private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionReassignment reassignment) {
+        return isReassignmentInProgress(tpi, reassignment) ?
+            reassignment.replicas().size() - reassignment.addingReplicas().size() :
+            tpi.replicas().size();
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopics        Topics that were found to match the requested topic name.
+     * @param requestedTopic     Name of the topic that was requested.
+     * @param requireTopicExists Indicates if the topic needs to exist for the operation to be successful.
+     *                           If set to true, the command will throw an exception if the topic with the
+     *                           requested name does not exist.
+     */
+    private static void ensureTopicExists(List<String> foundTopics, String requestedTopic, Boolean requireTopicExists) {
+        // If no topic name was mentioned, do not need to throw exception.
+        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
+            // If given topic doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
+        }
+    }
+
+    private static List<String> doGetTopics(List<String> allTopics, Optional<String> topicIncludeList, Boolean excludeInternalTopics) {
+        if (topicIncludeList.isPresent()) {
+            IncludeList topicsFilter = new IncludeList(topicIncludeList.get());
+            return allTopics.stream()
+                .filter(topic -> topicsFilter.isTopicAllowed(topic, excludeInternalTopics))
+                .collect(Collectors.toList());
+        } else {
+            return allTopics.stream()
+                .filter(topic -> !(Topic.isInternal(topic) && excludeInternalTopics))
+                .collect(Collectors.toList());
+        }
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopicIds        Topics that were found to match the requested topic id.
+     * @param requestedTopicId     Id of the topic that was requested.
+     * @param requireTopicIdExists Indicates if the topic needs to exist for the operation to be successful.
+     *                             If set to true, the command will throw an exception if the topic with the
+     *                             requested id does not exist.
+     */
+    private static void ensureTopicIdExists(List<Uuid> foundTopicIds, Uuid requestedTopicId, Boolean requireTopicIdExists) {
+        // If no topic id was mentioned, do not need to throw exception.
+        if (requestedTopicId != null && requireTopicIdExists && foundTopicIds.isEmpty()) {
+            // If given topicId doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("TopicId '%s' does not exist as expected", requestedTopicId));
+        }
+    }
+
+    static class CommandTopicPartition {
+        private final TopicCommandOptions opts;
+        private final Optional<String> name;
+        private final Optional<Integer> replicationFactor;
+        private final Optional<Integer> partitions;
+        private final Map<Integer, List<Integer>> replicaAssignment;
+        private final Properties configsToAdd;
+
+        public CommandTopicPartition(TopicCommandOptions options) {
+            opts = options;
+            name = options.topic();
+            partitions = options.partitions();
+            replicationFactor = options.replicationFactor();
+            replicaAssignment = options.replicaAssignment().orElse(Collections.emptyMap());
+            configsToAdd = parseTopicConfigsToBeAdded(options);
+        }
+
+        public Boolean hasReplicaAssignment() {
+            return !replicaAssignment.isEmpty();
+        }
+
+        public Boolean hasPartitions() {
+            return partitions.isPresent();
+        }
+
+        public Boolean ifTopicDoesntExist() {
+            return opts.ifNotExists();
+        }
+    }
+
+    static class TopicDescription {
+        private final String topic;
+        private final Uuid topicId;
+        private final Integer numPartitions;
+        private final Integer replicationFactor;
+        private final Config config;
+        private final Boolean markedForDeletion;
+
+        public TopicDescription(String topic, Uuid topicId, Integer numPartitions, Integer replicationFactor, Config config, Boolean markedForDeletion) {
+            this.topic = topic;
+            this.topicId = topicId;
+            this.numPartitions = numPartitions;
+            this.replicationFactor = replicationFactor;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+        }
+
+        public void printDescription() {
+            String configsAsString = config.entries().stream()
+                .filter(config -> !config.isDefault())
+                .map(ce -> String.format("%s=%s", ce.name(), ce.value()))
+                .collect(Collectors.joining(","));
+            System.out.print(String.format("Topic: %s", topic));
+            if (topicId != Uuid.ZERO_UUID)
+                System.out.print(String.format("\tTopicId: %s", topicId));
+            System.out.print(String.format("\tPartitionCount: %s", numPartitions));
+            System.out.print(String.format("\tReplicationFactor: %s", replicationFactor));
+            System.out.print(String.format("\tConfigs: %s", configsAsString));
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class PartitionDescription {
+        private final String topic;
+        private final TopicPartitionInfo info;
+        private final Config config;
+        private final Boolean markedForDeletion;
+        private final PartitionReassignment reassignment;
+
+        PartitionDescription(String topic,
+                             TopicPartitionInfo info,
+                             Config config,
+                             Boolean markedForDeletion,
+                             PartitionReassignment reassignment) {
+            this.topic = topic;
+            this.info = info;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+            this.reassignment = reassignment;
+        }
+
+        public Integer minIsrCount() {
+            return Integer.parseInt(config.get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value());
+        }
+
+        public Boolean isUnderReplicated() {
+            return getReplicationFactor(info, reassignment) - info.isr().size() > 0;
+        }
+
+        public boolean hasLeader() {
+            return info.leader() != null;
+        }
+
+        public Boolean isUnderMinIsr() {
+            return !hasLeader() ||  info.isr().size() < minIsrCount();
+        }
+
+        public Boolean isAtMinIsrPartitions() {
+            return minIsrCount() == info.isr().size();
+
+        }
+
+        public Boolean hasUnavailablePartitions(Set<Integer> liveBrokers) {
+            return !hasLeader() || !liveBrokers.contains(info.leader().id());
+        }
+
+        public void printDescription() {
+            System.out.print("\tTopic: " + topic);
+            System.out.print("\tPartition: " + info.partition());
+            System.out.print("\tLeader: " + (hasLeader() ? info.leader().id() : "none"));
+            System.out.print("\tReplicas: " + info.replicas().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            System.out.print("\tIsr: " + info.isr().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            if (reassignment != null) {
+                System.out.print("\tAdding Replicas: " + reassignment.addingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+                System.out.print("\tRemoving Replicas: " + reassignment.removingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+            }
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+
+    }
+
+    static class DescribeOptions {
+        private final TopicCommandOptions opts;
+        private final Set<Integer> liveBrokers;
+        private final boolean describeConfigs;
+        private final boolean describePartitions;
+
+        public DescribeOptions(TopicCommandOptions opts, Set<Integer> liveBrokers) {
+            this.opts = opts;
+            this.liveBrokers = liveBrokers;
+            this.describeConfigs = !opts.reportUnavailablePartitions() &&
+                !opts.reportUnderReplicatedPartitions() &&
+                !opts.reportUnderMinIsrPartitions() &&
+                !opts.reportAtMinIsrPartitions();
+            this.describePartitions = !opts.reportOverriddenConfigs();
+        }
+
+        private boolean shouldPrintUnderReplicatedPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderReplicatedPartitions() && partitionDescription.isUnderReplicated();
+        }
+
+        private boolean shouldPrintUnavailablePartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnavailablePartitions() && partitionDescription.hasUnavailablePartitions(liveBrokers);
+        }
+
+        private boolean shouldPrintUnderMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderMinIsrPartitions() && partitionDescription.isUnderMinIsr();
+        }
+
+        private boolean shouldPrintAtMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportAtMinIsrPartitions() && partitionDescription.isAtMinIsrPartitions();
+        }
+
+        private boolean shouldPrintTopicPartition(PartitionDescription partitionDesc) {
+            return describeConfigs ||
+                shouldPrintUnderReplicatedPartitions(partitionDesc) ||
+                shouldPrintUnavailablePartitions(partitionDesc) ||
+                shouldPrintUnderMinIsrPartitions(partitionDesc) ||
+                shouldPrintAtMinIsrPartitions(partitionDesc);
+        }
+
+        public void maybePrintPartitionDescription(PartitionDescription desc) {
+            if (shouldPrintTopicPartition(desc)) {
+                desc.printDescription();
+            }
+        }
+    }
+
+    public static class TopicService implements AutoCloseable {
+        private Admin adminClient;
+
+        public TopicService(Properties commandConfig, Optional<String> bootstrapServer) {
+            this.adminClient = createAdminClient(commandConfig, bootstrapServer);
+        }
+
+        public TopicService(Admin admin) {
+            this.adminClient = admin;
+        }
+
+        private static Admin createAdminClient(Properties commandConfig, Optional<String> bootstrapServer) {
+            if (bootstrapServer.isPresent()) {
+                commandConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+            }
+            return Admin.create(commandConfig);
+        }
+
+        public void createTopic(TopicCommandOptions opts) throws Exception {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            if (Topic.hasCollisionChars(topic.name.get())) {
+                System.out.println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could " +
+                    "collide. To avoid issues it is best to use either, but not both.");
+            }
+            createTopic(topic);
+        }
+
+        public void createTopic(CommandTopicPartition topic) throws Exception {
+            if (topic.replicationFactor.filter(rf -> rf > Short.MAX_VALUE || rf < 1).isPresent()) {
+                throw new IllegalArgumentException("The replication factor must be between 1 and " + Short.MAX_VALUE + " inclusive");
+            }
+            if (topic.partitions.filter(p -> p < 1).isPresent()) {
+                throw new IllegalArgumentException("The partitions must be greater than 0");
+            }
+
+            try {
+                NewTopic newTopic;
+                if (topic.hasReplicaAssignment()) {
+                    newTopic = new NewTopic(topic.name.get(), topic.replicaAssignment);
+                } else {
+                    newTopic = new NewTopic(topic.name.get(), topic.partitions, topic.replicationFactor.map(Integer::shortValue));
+                }
+
+                Map<String, String> configsMap = topic.configsToAdd.stringPropertyNames().stream()
+                    .collect(Collectors.toMap(name -> name, name -> topic.configsToAdd.getProperty(name)));
+
+                newTopic.configs(configsMap);
+                CreateTopicsResult createResult = adminClient.createTopics(Collections.singleton(newTopic),
+                    new CreateTopicsOptions().retryOnQuotaViolation(false));
+                createResult.all().get();
+                System.out.println("Created topic " + topic.name + ".");
+            } catch (ExecutionException e) {
+                if (e.getCause() == null) {
+                    throw e;
+                }
+                if (!(e.getCause() instanceof TopicExistsException && topic.ifTopicDoesntExist())) {
+                    throw (Exception) e.getCause();
+                }
+            }
+        }
+
+        public void listTopics(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            String results = getTopics(opts.topic(), opts.excludeInternalTopics())
+                .stream()
+                .collect(Collectors.joining("\n"));
+            System.out.println(results);
+        }
+
+        public void alterTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            CommandTopicPartition topic = new CommandTopicPartition(opts);
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+
+            if (!topics.isEmpty()) {
+                Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo = adminClient.describeTopics(topics).topicNameValues();
+                Map<String, NewPartitions> newPartitions = topics.stream()
+                    .map(topicName -> topicNewPartitions(topic, topicsInfo, topicName))
+                    .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                adminClient.createPartitions(newPartitions, new CreatePartitionsOptions().retryOnQuotaViolation(false)).all().get();
+            }
+        }
+
+        private AbstractMap.SimpleEntry<String, NewPartitions> topicNewPartitions(
+            CommandTopicPartition topic,
+            Map<String, KafkaFuture<org.apache.kafka.clients.admin.TopicDescription>> topicsInfo,
+            String topicName) {
+            if (topic.hasReplicaAssignment()) {
+                try {
+                    Integer startPartitionId = topicsInfo.get(topicName).get().partitions().size();
+                    Map<Integer, List<Integer>> replicaMap = topic.replicaAssignment.entrySet().stream()
+                        .skip(startPartitionId)
+                        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+                    List<List<Integer>> newAssignment = replicaMap.entrySet().stream()
+                        .map(Map.Entry::getValue)
+                        .collect(Collectors.toList());
+                    return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get(), newAssignment));
+                } catch (InterruptedException | ExecutionException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+            return new AbstractMap.SimpleEntry<>(topicName, NewPartitions.increaseTo(topic.partitions.get()));
+        }
+
+        public Map<TopicPartition, PartitionReassignment> listAllReassignments(Set<TopicPartition> topicPartitions) {
+            try {
+                return adminClient.listPartitionReassignments(topicPartitions).reassignments().get();
+            } catch (ExecutionException e) {
+                Throwable cause = e.getCause();
+                if (cause instanceof UnsupportedVersionException || cause instanceof ClusterAuthorizationException) {
+                    log.debug("Couldn't query reassignments through the AdminClient API: " + cause.getMessage(), cause);
+                    return Collections.emptyMap();
+                } else {
+                    throw new RuntimeException(e);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        public void describeTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            // If topicId is provided and not zero, will use topicId regardless of topic name
+            Optional<Uuid> inputTopicId = opts.topicId()
+                .map(Uuid::fromString).filter(uuid -> uuid != Uuid.ZERO_UUID);
+            Boolean useTopicId = inputTopicId.isPresent();
+
+            List<Uuid> topicIds;
+            List<String> topics;
+            if (useTopicId) {
+                topicIds = getTopicIds(inputTopicId.get(), opts.excludeInternalTopics());
+                topics = Collections.emptyList();
+            } else {
+                topicIds = Collections.emptyList();
+                topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+
+            }
+
+            // Only check topic name when topicId is not provided
+            if (useTopicId) {
+                ensureTopicIdExists(topicIds, inputTopicId.get(), !opts.ifExists());
+            } else {
+                ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            }
+            List<org.apache.kafka.clients.admin.TopicDescription> topicDescriptions = new ArrayList<>();
+
+            if (!topicIds.isEmpty()) {
+                Map<Uuid, org.apache.kafka.clients.admin.TopicDescription> descTopics =
+                    adminClient.describeTopics(TopicCollection.ofTopicIds(topicIds)).allTopicIds().get();
+                topicDescriptions = new ArrayList<>(descTopics.values());
+            }
+
+            if (!topics.isEmpty()) {
+                Map<String, org.apache.kafka.clients.admin.TopicDescription> descTopics =
+                    adminClient.describeTopics(TopicCollection.ofTopicNames(topics)).allTopicNames().get();
+                topicDescriptions = new ArrayList<>(descTopics.values());
+            }
+
+            List<String> topicNames = topicDescriptions.stream()
+                .map(org.apache.kafka.clients.admin.TopicDescription::name)
+                .collect(Collectors.toList());
+            Map<ConfigResource, KafkaFuture<Config>> allConfigs = adminClient.describeConfigs(
+                topicNames.stream()
+                    .map(name -> new ConfigResource(ConfigResource.Type.TOPIC, name))
+                    .collect(Collectors.toList())
+            ).values();
+            List<Integer> liveBrokers = adminClient.describeCluster().nodes().get().stream()
+                .map(Node::id)
+                .collect(Collectors.toList());
+            DescribeOptions describeOptions = new DescribeOptions(opts, new HashSet<>(liveBrokers));
+            Set<TopicPartition> topicPartitions = topicDescriptions
+                .stream()
+                .flatMap(td -> td.partitions().stream()
+                    .map(p -> new TopicPartition(td.name(), p.partition())))
+                .collect(Collectors.toSet());
+            Map<TopicPartition, PartitionReassignment> reassignments = listAllReassignments(topicPartitions);
+            for (org.apache.kafka.clients.admin.TopicDescription td : topicDescriptions) {
+                String topicName = td.name();
+                Uuid topicId = td.topicId();
+                Config config = allConfigs.get(new ConfigResource(ConfigResource.Type.TOPIC, topicName)).get();
+                ArrayList<TopicPartitionInfo> sortedPartitions = new ArrayList<>(td.partitions());
+                sortedPartitions.sort(Comparator.comparingInt(TopicPartitionInfo::partition));
+                printDescribeConfig(opts, describeOptions, reassignments, td, topicName, topicId, config, sortedPartitions);
+                printPartitionDescription(describeOptions, reassignments, td, topicName, config, sortedPartitions);
+            }
+        }
+
+        private void printPartitionDescription(DescribeOptions describeOptions, Map<TopicPartition, PartitionReassignment> reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Config config, ArrayList<TopicPartitionInfo> sortedPartitions) {
+            if (describeOptions.describePartitions) {
+                for (TopicPartitionInfo partition : sortedPartitions) {
+                    PartitionReassignment reassignment =
+                        reassignments.get(new TopicPartition(td.name(), partition.partition()));
+                    PartitionDescription partitionDesc = new PartitionDescription(topicName,
+                        partition, config, false, reassignment);
+                    describeOptions.maybePrintPartitionDescription(partitionDesc);
+                }
+            }
+        }
+
+        private void printDescribeConfig(TopicCommandOptions opts, DescribeOptions describeOptions, Map<TopicPartition, PartitionReassignment> reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Uuid topicId, Config config, ArrayList<TopicPartitionInfo> sortedPartitions) {
+            if (describeOptions.describeConfigs) {
+                List<ConfigEntry> entries = new ArrayList<>(config.entries());
+                boolean hasNonDefault = entries.stream().anyMatch(e -> !e.isDefault());
+                if (!opts.reportOverriddenConfigs() || hasNonDefault) {
+                    int numPartitions = td.partitions().size();
+                    TopicPartitionInfo firstPartition = sortedPartitions.get(0);
+                    PartitionReassignment reassignment =
+                        reassignments.get(new TopicPartition(td.name(), firstPartition.partition()));
+                    TopicDescription topicDesc = new TopicDescription(topicName, topicId,
+                        numPartitions, getReplicationFactor(firstPartition, reassignment),
+                        config, false);
+                    topicDesc.printDescription();
+                }
+            }
+        }
+
+        public void deleteTopic(TopicCommandOptions opts) throws ExecutionException, InterruptedException {
+            List<String> topics = getTopics(opts.topic(), opts.excludeInternalTopics());
+            ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists());
+            adminClient.deleteTopics(Collections.unmodifiableList(topics),
+                new DeleteTopicsOptions().retryOnQuotaViolation(false)
+            ).all().get();
+        }
+
+        public List<String> getTopics(Optional<String> topicIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
+            ListTopicsOptions listTopicsOptions = new ListTopicsOptions();
+            if (!excludeInternalTopics) {
+                listTopicsOptions.listInternal(true);
+            }
+
+            Set<String> allTopics = adminClient.listTopics(listTopicsOptions).names().get();
+            return doGetTopics(allTopics.stream().sorted().collect(Collectors.toList()), topicIncludeList, excludeInternalTopics);
+        }
+
+        public List<Uuid> getTopicIds(Uuid topicIdIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException {
+            ListTopicsResult allTopics = excludeInternalTopics ? adminClient.listTopics() :
+                adminClient.listTopics(new ListTopicsOptions().listInternal(true));
+            List<Uuid> allTopicIds = null;
+            allTopicIds = allTopics.listings().get().stream()
+                .map(TopicListing::topicId)
+                .sorted()
+                .collect(Collectors.toList());
+            return allTopicIds.contains(topicIdIncludeList) ?
+                Collections.singletonList(topicIdIncludeList) :
+                Collections.emptyList();
+        }
+
+        @Override
+        public void close() throws Exception {
+            adminClient.close();
+        }
+    }
+
+    public static class TopicCommandOptions extends CommandDefaultOptions {
+        private final ArgumentAcceptingOptionSpec<String> bootstrapServerOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> commandConfigOpt;
+
+        private final OptionSpecBuilder listOpt;
+
+        private final OptionSpecBuilder createOpt;
+
+        private final OptionSpecBuilder deleteOpt;
+
+        private final OptionSpecBuilder alterOpt;
+
+        private final OptionSpecBuilder describeOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> topicOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> topicIdOpt;
+
+        private final String nl;
+
+        private static final String KAFKA_CONFIGS_CLI_SUPPORTS_ALTERING_TOPIC_CONFIGS_WITH_A_BOOTSTRAP_SERVER =
+                " (the kafka-configs CLI supports altering topic configs with a --bootstrap-server option)";
+
+        private final ArgumentAcceptingOptionSpec<String> configOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> deleteConfigOpt;
+
+        private final ArgumentAcceptingOptionSpec<Integer> partitionsOpt;
+
+        private final ArgumentAcceptingOptionSpec<Integer> replicationFactorOpt;
+
+        private final ArgumentAcceptingOptionSpec<String> replicaAssignmentOpt;
+
+        private final OptionSpecBuilder reportUnderReplicatedPartitionsOpt;
+
+        private final OptionSpecBuilder reportUnavailablePartitionsOpt;
+
+        private final OptionSpecBuilder reportUnderMinIsrPartitionsOpt;
+
+        private final OptionSpecBuilder reportAtMinIsrPartitionsOpt;
+
+        private final OptionSpecBuilder topicsWithOverridesOpt;
+
+        private final OptionSpecBuilder ifExistsOpt;
+
+        private final OptionSpecBuilder ifNotExistsOpt;
+
+        private final OptionSpecBuilder excludeInternalTopicOpt;
+
+        private final Set<OptionSpec<?>> allTopicLevelOpts;
+
+        private final Set<OptionSpecBuilder> allReplicationReportOpts;
+
+        public TopicCommandOptions(String[] args) {
+            super(args);
+            bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The Kafka server to connect to.")
+                .withRequiredArg()
+                .describedAs("server to connect to")
+                .ofType(String.class);
+            commandConfigOpt = parser.accepts("command-config", "Property file containing configs to be passed to Admin Client. " +
+                    "This is used only with --bootstrap-server option for describing and altering broker configs.")
+                .withRequiredArg()
+                .describedAs("command config property file")
+                .ofType(String.class);
+
+
+            listOpt = parser.accepts("list", "List all available topics.");
+            createOpt = parser.accepts("create", "Create a new topic.");
+            deleteOpt = parser.accepts("delete", "Delete a topic");
+            alterOpt = parser.accepts("alter", "Alter the number of partitions, replica assignment, and/or configuration for the topic.");
+            describeOpt = parser.accepts("describe", "List details for the given topics.");
+            topicOpt = parser.accepts("topic", "Alter the number of partitions and replica assignment. " +
+                            "Update the configuration of an existing topic via --alter is no longer supported here" +
+                            KAFKA_CONFIGS_CLI_SUPPORTS_ALTERING_TOPIC_CONFIGS_WITH_A_BOOTSTRAP_SERVER + ".")
+                .withRequiredArg()
+                .describedAs("topic")
+                .ofType(String.class);
+            topicIdOpt = parser.accepts("topic-id", "The topic-id to describe." +
+                    "This is used only with --bootstrap-server option for describing topics.")
+                .withRequiredArg()
+                .describedAs("topic-id")
+                .ofType(String.class);
+            nl = System.getProperty("line.separator");
+
+            String configNames = LogConfig.configNames().stream()
+                .map(name -> "\t" + name)
+                .collect(Collectors.joining("nl"));
+            configOpt = parser.accepts("config", "A topic configuration override for the topic being created or altered." +

Review Comment:
   We had it in the original scala code. I removed it from the new code. 



##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);

Review Comment:
   done



##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String[] args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {
+            System.err.println(e.getMessage());
+            return 1;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        opts.checkArgs();

Review Comment:
   Moved the `checkArgs` inside `TopicCommandOptions`. This will have some impact on the test as all the tests were originally running without validating the args 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] OmniaGM commented on a diff in pull request #13201: KAFKA-14596: Move TopicCommand to tools

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1297475863


##########
tools/src/main/java/org/apache/kafka/tools/TopicCommand.java:
##########
@@ -0,0 +1,1019 @@
+/*
+ * 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;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.AdminClientConfig;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.CreatePartitionsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsOptions;
+import org.apache.kafka.clients.admin.CreateTopicsResult;
+import org.apache.kafka.clients.admin.DeleteTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsOptions;
+import org.apache.kafka.clients.admin.ListTopicsResult;
+import org.apache.kafka.clients.admin.NewPartitions;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicListing;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicCollection;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.AdminCommandFailedException;
+import org.apache.kafka.server.common.AdminOperationException;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+import org.apache.kafka.server.util.CommandLineUtils;
+import org.apache.kafka.server.util.TopicFilter.IncludeList;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class TopicCommand {
+    private static final Logger log = LoggerFactory.getLogger(TopicCommand.class);
+
+    public static void main(String[] args) {
+        Exit.exit(mainNoExit(args));
+    }
+
+    private static int mainNoExit(String... args) {
+        try {
+            execute(args);
+            return 0;
+        } catch (TerseException e) {
+            System.err.println(e.getMessage());
+            return 1;
+        } catch (Throwable e) {
+            System.err.println(e.getMessage());
+            System.err.println(Utils.stackTrace(e));
+            return 1;
+        }
+    }
+
+    static void execute(String... args) throws Exception {
+        TopicCommandOptions opts = new TopicCommandOptions(args);
+        opts.checkArgs();
+        TopicService topicService = new TopicService(opts.commandConfig(), opts.bootstrapServer());
+        int exitCode = 0;
+        try {
+            if (opts.hasCreateOption()) {
+                topicService.createTopic(opts);
+            } else if (opts.hasAlterOption()) {
+                topicService.alterTopic(opts);
+            } else if (opts.hasListOption()) {
+                topicService.listTopics(opts);
+            } else if (opts.hasDescribeOption()) {
+                topicService.describeTopic(opts);
+            } else if (opts.hasDeleteOption()) {
+                topicService.deleteTopic(opts);
+            }
+        } catch (ExecutionException e) {
+            Throwable cause = e.getCause();
+            if (cause != null) {
+                printException(cause);
+            } else {
+                printException(e);
+            }
+            exitCode = 1;
+        } catch (Throwable e) {
+            printException(e);
+            exitCode = 1;
+        } finally {
+            topicService.close();
+            Exit.exit(exitCode);
+        }
+
+    }
+
+    private static void printException(Throwable e) {
+        System.out.println("Error while executing topic command : " + e.getMessage());
+        log.error(Utils.stackTrace(e));
+    }
+
+    @SuppressWarnings("deprecation")
+    static Map<Integer, List<Integer>> parseReplicaAssignment(String replicaAssignmentList) {
+        String[] partitionList = replicaAssignmentList.split(",");
+        Map<Integer, List<Integer>> ret = new LinkedHashMap<Integer, List<Integer>>();
+        for (int i = 0; i < partitionList.length; i++) {
+            List<Integer> brokerList = Arrays.stream(partitionList[i].split(":"))
+                .map(String::trim)
+                .mapToInt(Integer::parseInt)
+                .boxed()
+                .collect(Collectors.toList());
+            Collection<Integer> duplicateBrokers = ToolsUtils.duplicates(brokerList);
+            if (!duplicateBrokers.isEmpty()) {
+                throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: " +
+                    duplicateBrokers.stream()
+                        .map(Object::toString)
+                        .collect(Collectors.joining(","))
+                );
+            }
+            ret.put(i, brokerList);
+            if (ret.get(i).size() != ret.get(0).size()) {
+                throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList);
+            }
+        }
+        return ret;
+    }
+
+    @SuppressWarnings("deprecation")
+    private static Properties parseTopicConfigsToBeAdded(TopicCommandOptions opts) {
+        List<List<String>> configsToBeAdded = opts.topicConfig().orElse(Collections.emptyList())
+            .stream()
+            .map(s -> Arrays.asList(s.split("\\s*=\\s*")))
+            .collect(Collectors.toList());
+
+        if (!configsToBeAdded.stream().allMatch(config -> config.size() == 2)) {
+            throw new IllegalArgumentException("requirement failed: Invalid topic config: all configs to be added must be in the format \"key=val\".");
+        }
+
+        Properties props = new Properties();
+        configsToBeAdded.stream()
+            .forEach(pair -> props.setProperty(pair.get(0).trim(), pair.get(1).trim()));
+        LogConfig.validate(props);
+        if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) {
+            System.out.println("WARNING: The configuration ${TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG}=${props.getProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)} is specified. " +
+                "This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker or " +
+                "if the inter.broker.protocol.version is 3.0 or newer. This configuration is deprecated and it will be removed in Apache Kafka 4.0.");
+        }
+        return props;
+    }
+
+    // It is possible for a reassignment to complete between the time we have fetched its state and the time
+    // we fetch partition metadata. In this case, we ignore the reassignment when determining replication factor.
+    public static boolean isReassignmentInProgress(TopicPartitionInfo tpi, PartitionReassignment ra) {
+        // Reassignment is still in progress as long as the removing and adding replicas are still present
+        Set<Integer> allReplicaIds = tpi.replicas().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> changingReplicaIds = new HashSet<>();
+        if (ra != null) {
+            changingReplicaIds.addAll(ra.removingReplicas());
+            changingReplicaIds.addAll(ra.addingReplicas());
+        }
+        return allReplicaIds.stream().anyMatch(changingReplicaIds::contains);
+
+    }
+
+    private static Integer getReplicationFactor(TopicPartitionInfo tpi, PartitionReassignment reassignment) {
+        return isReassignmentInProgress(tpi, reassignment) ?
+            reassignment.replicas().size() - reassignment.addingReplicas().size() :
+            tpi.replicas().size();
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopics        Topics that were found to match the requested topic name.
+     * @param requestedTopic     Name of the topic that was requested.
+     * @param requireTopicExists Indicates if the topic needs to exist for the operation to be successful.
+     *                           If set to true, the command will throw an exception if the topic with the
+     *                           requested name does not exist.
+     */
+    private static void ensureTopicExists(List<String> foundTopics, String requestedTopic, Boolean requireTopicExists) {
+        // If no topic name was mentioned, do not need to throw exception.
+        if (!(requestedTopic.isEmpty() || !Optional.ofNullable(requestedTopic).isPresent()) && requireTopicExists && foundTopics.isEmpty()) {
+            // If given topic doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("Topic '%s' does not exist as expected", requestedTopic));
+        }
+    }
+
+    private static List<String> doGetTopics(List<String> allTopics, Optional<String> topicIncludeList, Boolean excludeInternalTopics) {
+        if (topicIncludeList.isPresent()) {
+            IncludeList topicsFilter = new IncludeList(topicIncludeList.get());
+            return allTopics.stream()
+                .filter(topic -> topicsFilter.isTopicAllowed(topic, excludeInternalTopics))
+                .collect(Collectors.toList());
+        } else {
+            return allTopics.stream()
+                .filter(topic -> !(Topic.isInternal(topic) && excludeInternalTopics))
+                .collect(Collectors.toList());
+        }
+    }
+
+    /**
+     * ensures topic existence and throws exception if topic doesn't exist
+     *
+     * @param foundTopicIds        Topics that were found to match the requested topic id.
+     * @param requestedTopicId     Id of the topic that was requested.
+     * @param requireTopicIdExists Indicates if the topic needs to exist for the operation to be successful.
+     *                             If set to true, the command will throw an exception if the topic with the
+     *                             requested id does not exist.
+     */
+    private static void ensureTopicIdExists(List<Uuid> foundTopicIds, Uuid requestedTopicId, Boolean requireTopicIdExists) {
+        // If no topic id was mentioned, do not need to throw exception.
+        if (requestedTopicId != null && requireTopicIdExists && foundTopicIds.isEmpty()) {
+            // If given topicId doesn't exist then throw exception
+            throw new IllegalArgumentException(String.format("TopicId '%s' does not exist as expected", requestedTopicId));
+        }
+    }
+
+    static class CommandTopicPartition {
+        private final TopicCommandOptions opts;
+        private final Optional<String> name;
+        private final Optional<Integer> replicationFactor;
+        private final Optional<Integer> partitions;
+        private final Map<Integer, List<Integer>> replicaAssignment;
+        private final Properties configsToAdd;
+
+        public CommandTopicPartition(TopicCommandOptions options) {
+            opts = options;
+            name = options.topic();
+            partitions = options.partitions();
+            replicationFactor = options.replicationFactor();
+            replicaAssignment = options.replicaAssignment().orElse(Collections.emptyMap());
+            configsToAdd = parseTopicConfigsToBeAdded(options);
+        }
+
+        public Boolean hasReplicaAssignment() {
+            return !replicaAssignment.isEmpty();
+        }
+
+        public Boolean hasPartitions() {
+            return partitions.isPresent();
+        }
+
+        public Boolean ifTopicDoesntExist() {
+            return opts.ifNotExists();
+        }
+    }
+
+    static class TopicDescription {
+        private final String topic;
+        private final Uuid topicId;
+        private final Integer numPartitions;
+        private final Integer replicationFactor;
+        private final Config config;
+        private final Boolean markedForDeletion;
+
+        public TopicDescription(String topic, Uuid topicId, Integer numPartitions, Integer replicationFactor, Config config, Boolean markedForDeletion) {
+            this.topic = topic;
+            this.topicId = topicId;
+            this.numPartitions = numPartitions;
+            this.replicationFactor = replicationFactor;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+        }
+
+        public void printDescription() {
+            String configsAsString = config.entries().stream()
+                .filter(config -> !config.isDefault())
+                .map(ce -> String.format("%s=%s", ce.name(), ce.value()))
+                .collect(Collectors.joining(","));
+            System.out.print(String.format("Topic: %s", topic));
+            if (topicId != Uuid.ZERO_UUID)
+                System.out.print(String.format("\tTopicId: %s", topicId));
+            System.out.print(String.format("\tPartitionCount: %s", numPartitions));
+            System.out.print(String.format("\tReplicationFactor: %s", replicationFactor));
+            System.out.print(String.format("\tConfigs: %s", configsAsString));
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+    }
+
+    static class PartitionDescription {
+        private final String topic;
+        private final TopicPartitionInfo info;
+        private final Config config;
+        private final Boolean markedForDeletion;
+        private final PartitionReassignment reassignment;
+
+        PartitionDescription(String topic,
+                             TopicPartitionInfo info,
+                             Config config,
+                             Boolean markedForDeletion,
+                             PartitionReassignment reassignment) {
+            this.topic = topic;
+            this.info = info;
+            this.config = config;
+            this.markedForDeletion = markedForDeletion;
+            this.reassignment = reassignment;
+        }
+
+        public Integer minIsrCount() {
+            return Integer.parseInt(config.get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value());
+        }
+
+        public Boolean isUnderReplicated() {
+            return getReplicationFactor(info, reassignment) - info.isr().size() > 0;
+        }
+
+        public boolean hasLeader() {
+            return info.leader() != null;
+        }
+
+        public Boolean isUnderMinIsr() {
+            return !hasLeader() ||  info.isr().size() < minIsrCount();
+        }
+
+        public Boolean isAtMinIsrPartitions() {
+            return minIsrCount() == info.isr().size();
+
+        }
+
+        public Boolean hasUnavailablePartitions(Set<Integer> liveBrokers) {
+            return !hasLeader() || !liveBrokers.contains(info.leader().id());
+        }
+
+        public void printDescription() {
+            System.out.print("\tTopic: " + topic);
+            System.out.print("\tPartition: " + info.partition());
+            System.out.print("\tLeader: " + (hasLeader() ? info.leader().id() : "none"));
+            System.out.print("\tReplicas: " + info.replicas().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            System.out.print("\tIsr: " + info.isr().stream()
+                .map(node -> Integer.toString(node.id()))
+                .collect(Collectors.joining(",")));
+            if (reassignment != null) {
+                System.out.print("\tAdding Replicas: " + reassignment.addingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+                System.out.print("\tRemoving Replicas: " + reassignment.removingReplicas().stream()
+                    .map(node -> node.toString())
+                    .collect(Collectors.joining(",")));
+            }
+            System.out.print(markedForDeletion ? "\tMarkedForDeletion: true" : "");
+            System.out.println();
+        }
+
+    }
+
+    static class DescribeOptions {
+        private final TopicCommandOptions opts;
+        private final Set<Integer> liveBrokers;
+        private final boolean describeConfigs;
+        private final boolean describePartitions;
+
+        public DescribeOptions(TopicCommandOptions opts, Set<Integer> liveBrokers) {
+            this.opts = opts;
+            this.liveBrokers = liveBrokers;
+            this.describeConfigs = !opts.reportUnavailablePartitions() &&
+                !opts.reportUnderReplicatedPartitions() &&
+                !opts.reportUnderMinIsrPartitions() &&
+                !opts.reportAtMinIsrPartitions();
+            this.describePartitions = !opts.reportOverriddenConfigs();
+        }
+
+        private boolean shouldPrintUnderReplicatedPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderReplicatedPartitions() && partitionDescription.isUnderReplicated();
+        }
+
+        private boolean shouldPrintUnavailablePartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnavailablePartitions() && partitionDescription.hasUnavailablePartitions(liveBrokers);
+        }
+
+        private boolean shouldPrintUnderMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportUnderMinIsrPartitions() && partitionDescription.isUnderMinIsr();
+        }
+
+        private boolean shouldPrintAtMinIsrPartitions(PartitionDescription partitionDescription) {
+            return opts.reportAtMinIsrPartitions() && partitionDescription.isAtMinIsrPartitions();
+        }
+
+        private boolean shouldPrintTopicPartition(PartitionDescription partitionDesc) {
+            return describeConfigs ||
+                shouldPrintUnderReplicatedPartitions(partitionDesc) ||
+                shouldPrintUnavailablePartitions(partitionDesc) ||
+                shouldPrintUnderMinIsrPartitions(partitionDesc) ||
+                shouldPrintAtMinIsrPartitions(partitionDesc);
+        }
+
+        public void maybePrintPartitionDescription(PartitionDescription desc) {
+            if (shouldPrintTopicPartition(desc)) {
+                desc.printDescription();
+            }
+        }
+    }
+
+    public static class TopicService implements AutoCloseable {
+        private Admin adminClient;
+
+        public TopicService(Properties commandConfig, Optional<String> bootstrapServer) {
+            this.adminClient = createAdminClient(commandConfig, bootstrapServer);
+        }
+
+        public TopicService(Admin admin) {
+            this.adminClient = admin;
+        }
+
+        private static Admin createAdminClient(Properties commandConfig, Optional<String> bootstrapServer) {
+            if (bootstrapServer.isPresent()) {
+                commandConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);

Review Comment:
   Good catch. I updated the integration test to use --bootstrap-server. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "mimaison (via GitHub)" <gi...@apache.org>.
mimaison commented on PR #13201:
URL: https://github.com/apache/kafka/pull/13201#issuecomment-1764319701

   @OmniaGM thanks for the updates. I'm getting a test failure in TopicCommandIntegrationTest:
   ```
   Gradle Test Run :tools:test > Gradle Test Executor 31 > TopicCommandIntegrationTest > testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized(String) > testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized(String).quorum=kraft FAILED
       java.lang.RuntimeException: java.lang.IllegalArgumentException: Topic 'testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized-HbRT1wjRyV' does not exist as expected
           at org.apache.kafka.tools.TopicCommandIntegrationTest.lambda$captureDescribeTopicStandardOut$46(TopicCommandIntegrationTest.java:1006)
           at org.apache.kafka.tools.ToolsTestUtils.captureStandardStream(ToolsTestUtils.java:66)
           at org.apache.kafka.tools.ToolsTestUtils.captureStandardOut(ToolsTestUtils.java:50)
           at org.apache.kafka.tools.TopicCommandIntegrationTest.captureDescribeTopicStandardOut(TopicCommandIntegrationTest.java:1009)
           at org.apache.kafka.tools.TopicCommandIntegrationTest.testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized(TopicCommandIntegrationTest.java:943)
   
           Caused by:
           java.lang.IllegalArgumentException: Topic 'testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized-HbRT1wjRyV' does not exist as expected
               at org.apache.kafka.tools.TopicCommand.ensureTopicExists(TopicCommand.java:215)
               at org.apache.kafka.tools.TopicCommand.access$700(TopicCommand.java:78)
               at org.apache.kafka.tools.TopicCommand$TopicService.describeTopic(TopicCommand.java:559)
               at org.apache.kafka.tools.TopicCommandIntegrationTest.lambda$captureDescribeTopicStandardOut$46(TopicCommandIntegrationTest.java:1004)
               ... 4 more
   ```


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14596: Move TopicCommand to tools [kafka]

Posted by "OmniaGM (via GitHub)" <gi...@apache.org>.
OmniaGM commented on code in PR #13201:
URL: https://github.com/apache/kafka/pull/13201#discussion_r1358474161


##########
tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java:
##########
@@ -0,0 +1,1050 @@
+/*
+ * 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;
+
+import kafka.admin.RackAwareTest;
+import kafka.server.KafkaBroker;
+import kafka.server.KafkaConfig;
+import kafka.utils.Logging;
+import kafka.utils.TestUtils;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.NewPartitionReassignment;
+import org.apache.kafka.clients.admin.NewTopic;
+import org.apache.kafka.clients.admin.PartitionReassignment;
+import org.apache.kafka.clients.admin.TopicDescription;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionInfo;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.MetadataResponseData;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import scala.collection.JavaConverters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@Tag("integration")
+@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters
+public class TopicCommandIntegrationTest extends kafka.integration.KafkaServerTestHarness implements Logging, RackAwareTest {
+    private Short defaultReplicationFactor = 1;
+    private Integer numPartitions = 1;
+    private TopicCommand.TopicService topicService;
+    private Admin adminClient;
+    private String bootstrapServer;
+    private String testTopicName;
+    private Integer defaultTimeout = 10000;
+
+    /**
+     * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every
+     * test and should not reuse previous configurations unless they select their ports randomly when servers are started.
+     *
+     * Note the replica fetch max bytes is set to `1` in order to throttle the rate of replication for test
+     * `testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress`.
+     */
+    @Override
+    public scala.collection.Seq<KafkaConfig> generateConfigs() {
+        Map<Integer, String> rackInfo = new HashMap<>();
+        rackInfo.put(0, "rack1");
+        rackInfo.put(1, "rack2");
+        rackInfo.put(2, "rack2");
+        rackInfo.put(3, "rack1");
+        rackInfo.put(4, "rack3");
+        rackInfo.put(5, "rack3");
+
+        List<Properties> brokerConfigs = ToolsTestUtils
+            .createBrokerProperties(6, zkConnectOrNull(), rackInfo, numPartitions, defaultReplicationFactor);
+
+        List<KafkaConfig> configs = new ArrayList<>();
+        for (Properties props : brokerConfigs) {
+            props.put(KafkaConfig.ReplicaFetchMaxBytesProp(), "1");
+            configs.add(KafkaConfig.fromProps(props));
+        }
+        return JavaConverters.asScalaBuffer(configs).toSeq();
+    }
+
+    private TopicCommand.TopicCommandOptions buildTopicCommandOptionsWithBootstrap(String... opts) {
+        String[] finalOptions = Stream.concat(Arrays.asList(opts).stream(),
+                Arrays.asList("--bootstrap-server", bootstrapServer).stream()
+        ).toArray(String[]::new);
+        return new TopicCommand.TopicCommandOptions(finalOptions);
+    }
+
+    private void createAndWaitTopic(TopicCommand.TopicCommandOptions opts) throws Exception {
+        topicService.createTopic(opts);
+        waitForTopicCreated(opts.topic().get());
+    }
+
+    private void waitForTopicCreated(String topicName) {
+        waitForTopicCreated(topicName, defaultTimeout);
+    }
+
+    private void waitForTopicCreated(String topicName, Integer timeout) {
+        TestUtils.waitForPartitionMetadata(brokers(), topicName, 0, timeout);
+    }
+
+    @BeforeEach
+    public void setUp(TestInfo info) {
+        super.setUp(info);
+        // create adminClient
+        Properties props = new Properties();
+        bootstrapServer = bootstrapServers(listenerName());
+        props.put(org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer);
+        adminClient = Admin.create(props);
+        topicService = new TopicCommand.TopicService(props, Optional.of(bootstrapServer));
+        testTopicName = String.format("%s-%s", info.getTestMethod().get().getName(),
+            new scala.util.Random().alphanumeric().take(10).mkString());
+    }
+
+    @AfterEach
+    public void close() throws Exception {
+        if (topicService != null)
+            topicService.close();
+        if (adminClient != null)
+            adminClient.close();
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreate(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", "2", "--replication-factor", "1", "--topic", testTopicName));
+
+        adminClient.listTopics().names().get().contains(testTopicName);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithDefaults(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName));
+
+        List<TopicPartitionInfo> partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+        assertEquals(partitions.size(), numPartitions);
+        assertEquals((short) partitions.get(0).replicas().size(), defaultReplicationFactor);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithDefaultReplication(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName, "--partitions", "2"));
+
+        List<TopicPartitionInfo>  partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+        assertEquals(partitions.size(), 2);
+        assertEquals((short) partitions.get(0).replicas().size(), defaultReplicationFactor);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithDefaultPartitions(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName, "--replication-factor", "2"));
+
+        List<TopicPartitionInfo> partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+
+        assertEquals(partitions.size(), numPartitions);
+        assertEquals((short) partitions.get(0).replicas().size(), 2);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithConfigs(String quorum) throws Exception {
+        ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName);
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "2", "--replication-factor", "2", "--topic", testTopicName, "--config",
+                "delete.retention.ms=1000"));
+
+        Config configs = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource);
+        assertEquals(1000, Integer.valueOf(configs.get("delete.retention.ms").value()));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWhenAlreadyExists(String quorum) throws Exception {
+        int numPartitions = 1;
+
+        // create the topic
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", Integer.toString(numPartitions), "--replication-factor", "1",
+                "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+
+        // try to re-create the topic
+        assertThrows(TopicExistsException.class, () -> topicService.createTopic(createOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWhenAlreadyExistsWithIfNotExists(String quorum) throws Exception {
+        TopicCommand.TopicCommandOptions createOpts =
+                buildTopicCommandOptionsWithBootstrap("--create", "--topic", testTopicName, "--if-not-exists");
+        createAndWaitTopic(createOpts);
+        topicService.createTopic(createOpts);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithReplicaAssignment(String quorum) throws Exception {
+        // create the topic
+        TopicCommand.TopicCommandOptions createOpts =
+                buildTopicCommandOptionsWithBootstrap("--create", "--replica-assignment", "5:4,3:2,1:0", "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+
+        List<TopicPartitionInfo> partitions = adminClient
+            .describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames()
+            .get()
+            .get(testTopicName)
+            .partitions();
+        
+        assertEquals(3, partitions.size());
+        assertEquals(Arrays.asList(5, 4), getPartitionReplicas(partitions, 0));
+        assertEquals(Arrays.asList(3, 2), getPartitionReplicas(partitions, 1));
+        assertEquals(Arrays.asList(1, 0), getPartitionReplicas(partitions, 2));
+    }
+
+    private List<Integer> getPartitionReplicas(List<TopicPartitionInfo> partitions, int partitionNumber) {
+        return partitions.get(partitionNumber).replicas().stream().map(Node::id).collect(Collectors.toList());
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithInvalidReplicationFactor(String quorum) {
+        TopicCommand.TopicCommandOptions opts = buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "2", "--replication-factor", Integer.toString(Short.MAX_VALUE + 1),
+            "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.createTopic(opts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithNegativeReplicationFactor(String quorum) {
+        TopicCommand.TopicCommandOptions opts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "2", "--replication-factor", "-1", "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.createTopic(opts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateWithNegativePartitionCount(String quorum) {
+        TopicCommand.TopicCommandOptions opts = buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "-1", "--replication-factor", "1", "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.createTopic(opts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testInvalidTopicLevelConfig(String quorum) {
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "1", "--replication-factor", "1", "--topic", testTopicName,
+            "--config", "message.timestamp.type=boom");
+        assertThrows(ConfigException.class, () -> topicService.createTopic(createOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListTopics(String quorum) throws Exception {
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap(
+            "--create", "--partitions", "1", "--replication-factor", "1", "--topic", testTopicName));
+
+        String output = captureListTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--list"));
+        assertTrue(output.contains(testTopicName));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListTopicsWithIncludeList(String quorum) throws ExecutionException, InterruptedException {
+        String topic1 = "kafka.testTopic1";
+        String topic2 = "kafka.testTopic2";
+        String topic3 = "oooof.testTopic1";
+        adminClient.createTopics(
+                Arrays.asList(new NewTopic(topic1, 2, (short) 2),
+                    new NewTopic(topic2, 2, (short) 2),
+                    new NewTopic(topic3, 2, (short) 2)))
+            .all().get();
+        waitForTopicCreated(topic1);
+        waitForTopicCreated(topic2);
+        waitForTopicCreated(topic3);
+
+        String output = captureListTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--list", "--topic", "kafka.*"));
+
+        assertTrue(output.contains(topic1));
+        assertTrue(output.contains(topic2));
+        assertFalse(output.contains(topic3));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testListTopicsWithExcludeInternal(String quorum) throws ExecutionException, InterruptedException {
+        String topic1 = "kafka.testTopic1";
+        adminClient.createTopics(
+                Arrays.asList(new NewTopic(topic1, 2, (short) 2),
+                    new NewTopic(Topic.GROUP_METADATA_TOPIC_NAME, 2, (short) 2)))
+            .all().get();
+        waitForTopicCreated(topic1);
+
+        String output = captureListTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--list", "--exclude-internal"));
+
+        assertTrue(output.contains(topic1));
+        assertFalse(output.contains(Topic.GROUP_METADATA_TOPIC_NAME));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterPartitionCount(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Arrays.asList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter", "--topic", testTopicName, "--partitions", "3"));
+
+        kafka.utils.TestUtils.waitUntilTrue(
+            () -> brokers().forall(b -> b.metadataCache().getTopicPartitions(testTopicName).size() == 3),
+            () -> "Timeout waiting for new assignment propagating to broker", org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+        TopicDescription topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get();
+        assertTrue(topicDescription.partitions().size() == 3);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterAssignment(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter",
+            "--topic", testTopicName, "--replica-assignment", "5:3,3:1,4:2", "--partitions", "3"));
+        kafka.utils.TestUtils.waitUntilTrue(
+            () -> brokers().forall(b -> b.metadataCache().getTopicPartitions(testTopicName).size() == 3),
+            () -> "Timeout waiting for new assignment propagating to broker",
+            org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+
+        TopicDescription topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get();
+        assertTrue(topicDescription.partitions().size() == 3);
+        List<Integer> partitionReplicas = getPartitionReplicas(topicDescription.partitions(), 2);
+        assertEquals(Arrays.asList(4, 2), partitionReplicas);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterAssignmentWithMoreAssignmentThanPartitions(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Arrays.asList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        assertThrows(ExecutionException.class,
+            () -> topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter",
+                "--topic", testTopicName, "--replica-assignment", "5:3,3:1,4:2,3:2", "--partitions", "3")));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterAssignmentWithMorePartitionsThanAssignment(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Arrays.asList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        assertThrows(ExecutionException.class,
+            () -> topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter", "--topic", testTopicName,
+                "--replica-assignment", "5:3,3:1,4:2", "--partitions", "6")));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterWithInvalidPartitionCount(String quorum) throws Exception {
+        createAndWaitTopic(
+                buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "1", "--replication-factor", "1", "--topic", testTopicName)
+        );
+
+        assertThrows(ExecutionException.class,
+            () -> topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter", "--partitions", "-1", "--topic", testTopicName)));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterWhenTopicDoesntExist(String quorum) {
+        // alter a topic that does not exist without --if-exists
+        TopicCommand.TopicCommandOptions alterOpts = buildTopicCommandOptionsWithBootstrap("--alter", "--topic", testTopicName, "--partitions", "1");
+        TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient);
+        assertThrows(IllegalArgumentException.class, () -> topicService.alterTopic(alterOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testAlterWhenTopicDoesntExistWithIfExists(String quorum) throws ExecutionException, InterruptedException {
+        topicService.alterTopic(buildTopicCommandOptionsWithBootstrap("--alter", "--topic", testTopicName, "--partitions", "1", "--if-exists"));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testCreateAlterTopicWithRackAware(String quorum) throws Exception {
+        Map<Integer, String> rackInfo = new HashMap<Integer, String>();
+        rackInfo.put(0, "rack1");
+        rackInfo.put(1, "rack2");
+        rackInfo.put(2, "rack2");
+        rackInfo.put(3, "rack1");
+        rackInfo.put(4, "rack3");
+        rackInfo.put(5, "rack3");
+
+        int numPartitions = 18;
+        int replicationFactor = 3;
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", Integer.toString(numPartitions),
+            "--replication-factor", Integer.toString(replicationFactor),
+            "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+
+        Map<Integer, List<Integer>> assignment = adminClient.describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames().get().get(testTopicName).partitions()
+            .stream()
+            .collect(Collectors.toMap(
+                info -> info.partition(),
+                info -> info.replicas().stream().map(Node::id).collect(Collectors.toList())));
+        checkReplicaDistribution(assignment, rackInfo, rackInfo.size(), numPartitions,
+            replicationFactor, true, true, true);
+
+        int alteredNumPartitions = 36;
+        // verify that adding partitions will also be rack aware
+        TopicCommand.TopicCommandOptions alterOpts = buildTopicCommandOptionsWithBootstrap("--alter",
+            "--partitions", Integer.toString(alteredNumPartitions),
+            "--topic", testTopicName);
+        topicService.alterTopic(alterOpts);
+
+        kafka.utils.TestUtils.waitUntilTrue(
+            () -> brokers().forall(p -> p.metadataCache().getTopicPartitions(testTopicName).size() == alteredNumPartitions),
+            () -> "Timeout waiting for new assignment propagating to broker",
+            org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+
+        assignment = adminClient.describeTopics(Collections.singletonList(testTopicName))
+            .allTopicNames().get().get(testTopicName).partitions().stream()
+            .collect(Collectors.toMap(info -> info.partition(), info -> info.replicas().stream().map(Node::id).collect(Collectors.toList())));
+        checkReplicaDistribution(assignment, rackInfo, rackInfo.size(), alteredNumPartitions, replicationFactor,
+            true, true, true);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testConfigPreservationAcrossPartitionAlteration(String quorum) throws Exception {
+        int numPartitionsOriginal = 1;
+        String cleanupKey = "cleanup.policy";
+        String cleanupVal = "compact";
+
+        // create the topic
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", Integer.toString(numPartitionsOriginal),
+            "--replication-factor", "1",
+            "--config", cleanupKey + "=" + cleanupVal,
+            "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+        ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName);
+        Config props = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource);
+        // val props = adminZkClient.fetchEntityConfig(ConfigType.Topic, testTopicName)
+        assertNotNull(props.get(cleanupKey), "Properties after creation don't contain " + cleanupKey);
+        assertEquals(cleanupVal, props.get(cleanupKey).value(), "Properties after creation have incorrect value");
+
+        // pre-create the topic config changes path to avoid a NoNodeException
+        if (!isKRaftTest()) {
+            zkClient().makeSurePersistentPathExists(kafka.zk.ConfigEntityChangeNotificationZNode.path());
+        }
+
+        // modify the topic to add new partitions
+        int numPartitionsModified = 3;
+        TopicCommand.TopicCommandOptions alterOpts = buildTopicCommandOptionsWithBootstrap("--alter",
+            "--partitions", Integer.toString(numPartitionsModified), "--topic", testTopicName);
+        topicService.alterTopic(alterOpts);
+        Config newProps = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource);
+        assertNotNull(newProps.get(cleanupKey), "Updated properties do not contain " + cleanupKey);
+        assertEquals(cleanupVal, newProps.get(cleanupKey).value(), "Updated properties have incorrect value");
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testTopicDeletion(String quorum) throws Exception {
+        // create the NormalTopic
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "1",
+            "--replication-factor", "1",
+            "--topic", testTopicName);
+        createAndWaitTopic(createOpts);
+
+        // delete the NormalTopic
+        TopicCommand.TopicCommandOptions deleteOpts = buildTopicCommandOptionsWithBootstrap("--delete", "--topic", testTopicName);
+
+        if (!isKRaftTest()) {
+            String deletePath = kafka.zk.DeleteTopicsTopicZNode.path(testTopicName);
+            assertFalse(zkClient().pathExists(deletePath), "Delete path for topic shouldn't exist before deletion.");
+        }
+        topicService.deleteTopic(deleteOpts);
+        TestUtils.verifyTopicDeletion(zkClientOrNull(), testTopicName, 1, brokers());
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testTopicWithCollidingCharDeletionAndCreateAgain(String quorum) throws Exception {
+        // create the topic with colliding chars
+        String topicWithCollidingChar = "test.a";
+        TopicCommand.TopicCommandOptions createOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "1",
+            "--replication-factor", "1",
+            "--topic", topicWithCollidingChar);
+        createAndWaitTopic(createOpts);
+
+        // delete the topic
+        TopicCommand.TopicCommandOptions deleteOpts = buildTopicCommandOptionsWithBootstrap("--delete", "--topic", topicWithCollidingChar);
+
+        if (!isKRaftTest()) {
+            String deletePath = kafka.zk.DeleteTopicsTopicZNode.path(topicWithCollidingChar);
+            assertFalse(zkClient().pathExists(deletePath), "Delete path for topic shouldn't exist before deletion.");
+        }
+        topicService.deleteTopic(deleteOpts);
+        TestUtils.verifyTopicDeletion(zkClientOrNull(), topicWithCollidingChar, 1, brokers());
+        assertDoesNotThrow(() -> createAndWaitTopic(createOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteInternalTopic(String quorum) throws Exception {
+        // create the offset topic
+        TopicCommand.TopicCommandOptions createOffsetTopicOpts = buildTopicCommandOptionsWithBootstrap("--create",
+            "--partitions", "1",
+            "--replication-factor", "1",
+            "--topic", Topic.GROUP_METADATA_TOPIC_NAME);
+        createAndWaitTopic(createOffsetTopicOpts);
+
+        // Try to delete the Topic.GROUP_METADATA_TOPIC_NAME which is allowed by default.
+        // This is a difference between the new and the old command as the old one didn't allow internal topic deletion.
+        // If deleting internal topics is not desired, ACLS should be used to control it.
+        TopicCommand.TopicCommandOptions deleteOffsetTopicOpts =
+                buildTopicCommandOptionsWithBootstrap("--delete", "--topic", Topic.GROUP_METADATA_TOPIC_NAME);
+        String deleteOffsetTopicPath = kafka.zk.DeleteTopicsTopicZNode.path(Topic.GROUP_METADATA_TOPIC_NAME);
+        if (!isKRaftTest()) {
+            assertFalse(zkClient().pathExists(deleteOffsetTopicPath), "Delete path for topic shouldn't exist before deletion.");
+        }
+        topicService.deleteTopic(deleteOffsetTopicOpts);
+        TestUtils.verifyTopicDeletion(zkClientOrNull(), Topic.GROUP_METADATA_TOPIC_NAME, 1, brokers());
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteWhenTopicDoesntExist(String quorum) {
+        // delete a topic that does not exist
+        TopicCommand.TopicCommandOptions deleteOpts = buildTopicCommandOptionsWithBootstrap("--delete", "--topic", testTopicName);
+        assertThrows(IllegalArgumentException.class, () -> topicService.deleteTopic(deleteOpts));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDeleteWhenTopicDoesntExistWithIfExists(String quorum) throws ExecutionException, InterruptedException {
+        topicService.deleteTopic(buildTopicCommandOptionsWithBootstrap("--delete", "--topic", testTopicName, "--if-exists"));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribe(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 2, (short) 2))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName));
+        String[] rows = output.split("\n");
+        assertEquals(3, rows.length);
+        assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeWhenTopicDoesntExist(String quorum) {
+        assertThrows(IllegalArgumentException.class,
+            () -> topicService.describeTopic(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName)));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeWhenTopicDoesntExistWithIfExists(String quorum) throws ExecutionException, InterruptedException {
+        topicService.describeTopic(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName, "--if-exists"));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeUnavailablePartitions(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 6, (short) 1))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        try {
+            // check which partition is on broker 0 which we'll kill
+            TopicDescription testTopicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName))
+                .allTopicNames().get().get(testTopicName);
+            int partitionOnBroker0 = testTopicDescription.partitions().stream()
+                .filter(partition -> partition.leader().id() == 0)
+                .findFirst().get().partition();
+
+            killBroker(0);
+
+            // wait until the topic metadata for the test topic is propagated to each alive broker
+            kafka.utils.TestUtils.waitUntilTrue(
+                () -> {
+                    boolean result = true;
+                    for (KafkaBroker server : JavaConverters.asJavaCollection(brokers())) {
+                        if (server.config().brokerId() != 0) {
+                            Set<String> topicNames = Collections.singleton(testTopicName);
+                            Collection<MetadataResponseData.MetadataResponseTopic> topicMetadatas =
+                                JavaConverters.asJavaCollection(server.dataPlaneRequestProcessor().metadataCache()
+                                .getTopicMetadata(JavaConverters.asScalaSetConverter(topicNames).asScala().toSet(),
+                                    ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
+                                    false, false)
+                                );
+                            Optional<MetadataResponseData.MetadataResponseTopic> testTopicMetadata = topicMetadatas.stream()
+                                .filter(metadata -> metadata.name().equals(testTopicName))
+                                .findFirst();
+                            if (!testTopicMetadata.isPresent()) {
+                                throw new AssertionError("Topic metadata is not found in metadata cache");
+                            }
+                            Optional<MetadataResponseData.MetadataResponsePartition> testPartitionMetadata = testTopicMetadata.get().partitions().stream()
+                                .filter(metadata -> metadata.partitionIndex() == partitionOnBroker0)
+                                .findFirst();
+                            if (!testPartitionMetadata.isPresent()) {
+                                throw new AssertionError("Partition metadata is not found in metadata cache");
+                            }
+                            result = result && testPartitionMetadata.get().errorCode() == Errors.LEADER_NOT_AVAILABLE.code();
+                        }
+                    }
+                    return result;
+                },
+                () -> String.format("Partition metadata for %s is not propagated", testTopicName),
+                org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+
+            // grab the console output and assert
+            String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName, "--unavailable-partitions"));
+            String[] rows = output.split("\n");
+            assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)));
+            assertTrue(rows[0].contains("Leader: none\tReplicas: 0\tIsr:"));
+        } finally {
+            restartDeadBrokers(false);
+        }
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeUnderReplicatedPartitions(String quorum) throws ExecutionException, InterruptedException {
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 1, (short) 6))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        try {
+            killBroker(0);
+            if (isKRaftTest()) {
+                ensureConsistentKRaftMetadata();
+            } else {
+                TestUtils.waitForPartitionMetadata(aliveBrokers(), testTopicName, 0, org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS);
+            }
+            String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--under-replicated-partitions"));
+            String[] rows = output.split("\n");
+            assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)), String.format("Unexpected output: %s", rows[0]));
+        } finally {
+            restartDeadBrokers(false);
+        }
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeUnderMinIsrPartitions(String quorum) throws ExecutionException, InterruptedException {
+        Map<String, String> configMap = new HashMap<>();
+        configMap.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "6");
+
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 1, (short) 6).configs(configMap))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        try {
+            killBroker(0);
+            if (isKRaftTest()) {
+                ensureConsistentKRaftMetadata();
+            } else {
+                kafka.utils.TestUtils.waitUntilTrue(
+                    () -> aliveBrokers().forall(b -> b.metadataCache().getPartitionInfo(testTopicName, 0).get().isr().size() == 5),
+                    () -> String.format("Timeout waiting for partition metadata propagating to brokers for %s topic", testTopicName),
+                    org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L
+                );
+            }
+            String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--under-min-isr-partitions"));
+            String[] rows = output.split("\n");
+            assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)));
+        } finally {
+            restartDeadBrokers(false);
+        }
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress(String quorum) throws ExecutionException, InterruptedException {
+        Map<String, String> configMap = new HashMap<>();
+        Short replicationFactor = 1;
+        int partitions = 1;
+        TopicPartition tp = new TopicPartition(testTopicName, 0);
+
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, partitions, replicationFactor).configs(configMap))
+        ).all().get();
+        waitForTopicCreated(testTopicName);
+
+        // Produce multiple batches.
+        TestUtils.generateAndProduceMessages(brokers(), testTopicName, 10, -1);
+        TestUtils.generateAndProduceMessages(brokers(), testTopicName, 10, -1);
+
+        // Enable throttling. Note the broker config sets the replica max fetch bytes to `1` upon to minimize replication
+        // throughput so the reassignment doesn't complete quickly.
+        List<Integer> brokerIds = JavaConverters.seqAsJavaList(brokers()).stream()
+            .map(broker -> broker.config().brokerId()).collect(Collectors.toList());
+
+        ToolsTestUtils.setReplicationThrottleForPartitions(adminClient, brokerIds, Collections.singleton(tp), 1);
+
+        TopicDescription testTopicDesc = adminClient.describeTopics(Collections.singleton(testTopicName)).allTopicNames().get().get(testTopicName);
+        TopicPartitionInfo firstPartition = testTopicDesc.partitions().get(0);
+
+        List<Integer> replicasOfFirstPartition = firstPartition.replicas().stream().map(Node::id).collect(Collectors.toList());
+        List<Integer> replicasDiff = new ArrayList<>(brokerIds);
+        replicasDiff.removeAll(replicasOfFirstPartition);
+        Integer targetReplica = replicasDiff.get(0);
+
+        adminClient.alterPartitionReassignments(Collections.singletonMap(tp,
+            Optional.of(new NewPartitionReassignment(Collections.singletonList(targetReplica))))).all().get();
+
+        // let's wait until the LAIR is propagated
+        kafka.utils.TestUtils.waitUntilTrue(
+            () -> {
+                try {
+                    return !adminClient.listPartitionReassignments(Collections.singleton(tp)).reassignments().get()
+                        .get(tp).addingReplicas().isEmpty();
+                } catch (InterruptedException | ExecutionException e) {
+                    throw new RuntimeException(e);
+                }
+            },
+            () -> "Reassignment didn't add the second node",
+            org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+
+        // describe the topic and test if it's under-replicated
+        String simpleDescribeOutput = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName));
+        String[] simpleDescribeOutputRows = simpleDescribeOutput.split("\n");
+        assertTrue(simpleDescribeOutputRows[0].startsWith(String.format("Topic: %s", testTopicName)));
+        assertEquals(2, simpleDescribeOutputRows.length);
+
+        String underReplicatedOutput = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--under-replicated-partitions"));
+        assertEquals("", underReplicatedOutput,
+            String.format("--under-replicated-partitions shouldn't return anything: '%s'", underReplicatedOutput));
+
+        // Verify reassignment is still ongoing.
+        PartitionReassignment reassignments = adminClient.listPartitionReassignments(Collections.singleton(tp)).reassignments().get().get(tp);
+        assertFalse(reassignments.addingReplicas().isEmpty());
+
+        ToolsTestUtils.removeReplicationThrottleForPartitions(adminClient, brokerIds, Collections.singleton(tp));
+        TestUtils.waitForAllReassignmentsToComplete(adminClient, 100L);
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeAtMinIsrPartitions(String quorum) throws ExecutionException, InterruptedException {
+        Map<String, String> configMap = new HashMap<>();
+        configMap.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "4");
+
+        adminClient.createTopics(
+            Collections.singletonList(new NewTopic(testTopicName, 1, (short) 6).configs(configMap))).all().get();
+        waitForTopicCreated(testTopicName);
+
+        try {
+            killBroker(0);
+            killBroker(1);
+
+            if (isKRaftTest()) {
+                ensureConsistentKRaftMetadata();
+            } else {
+                kafka.utils.TestUtils.waitUntilTrue(
+                    () -> aliveBrokers().forall(broker -> broker.metadataCache().getPartitionInfo(testTopicName, 0).get().isr().size() == 4),
+                    () -> String.format("Timeout waiting for partition metadata propagating to brokers for %s topic", testTopicName),
+                    org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L
+                );
+            }
+
+            String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--at-min-isr-partitions"));
+            String[] rows = output.split("\n");
+            assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)));
+            assertEquals(1, rows.length);
+        } finally {
+            restartDeadBrokers(false);
+        }
+    }
+
+    /**
+     * Test describe --under-min-isr-partitions option with four topics:
+     *   (1) topic with partition under the configured min ISR count
+     *   (2) topic with under-replicated partition (but not under min ISR count)
+     *   (3) topic with offline partition
+     *   (4) topic with fully replicated partition
+     *
+     * Output should only display the (1) topic with partition under min ISR count and (3) topic with offline partition
+     */
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeUnderMinIsrPartitionsMixed(String quorum) throws ExecutionException, InterruptedException {
+        String underMinIsrTopic = "under-min-isr-topic";
+        String notUnderMinIsrTopic = "not-under-min-isr-topic";
+        String offlineTopic = "offline-topic";
+        String fullyReplicatedTopic = "fully-replicated-topic";
+
+        Map<String, String> configMap = new HashMap<>();
+        configMap.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "6");
+
+        adminClient.createTopics(
+            java.util.Arrays.asList(
+                new NewTopic(underMinIsrTopic, 1, (short) 6).configs(configMap),
+                new NewTopic(notUnderMinIsrTopic, 1, (short) 6),
+                new NewTopic(offlineTopic, Collections.singletonMap(0, Collections.singletonList(0))),
+                new NewTopic(fullyReplicatedTopic, Collections.singletonMap(0, java.util.Arrays.asList(1, 2, 3))))
+        ).all().get();
+
+        waitForTopicCreated(underMinIsrTopic);
+        waitForTopicCreated(notUnderMinIsrTopic);
+        waitForTopicCreated(offlineTopic);
+        waitForTopicCreated(fullyReplicatedTopic);
+
+        try {
+            killBroker(0);
+            if (isKRaftTest()) {
+                ensureConsistentKRaftMetadata();
+            } else {
+                kafka.utils.TestUtils.waitUntilTrue(
+                    () -> aliveBrokers().forall(broker ->
+                        broker.metadataCache().getPartitionInfo(underMinIsrTopic, 0).get().isr().size() < 6 &&
+                            broker.metadataCache().getPartitionInfo(offlineTopic, 0).get().leader() == MetadataResponse.NO_LEADER_ID),
+                    () -> "Timeout waiting for partition metadata propagating to brokers for underMinIsrTopic topic",
+                    org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS, 100L);
+            }
+            String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--under-min-isr-partitions"));
+            String[] rows = output.split("\n");
+            assertTrue(rows[0].startsWith(String.format("Topic: %s", underMinIsrTopic)));
+            assertTrue(rows[1].startsWith(String.format("\tTopic: %s", offlineTopic)));
+            assertEquals(2, rows.length);
+        } finally {
+            restartDeadBrokers(false);
+        }
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeReportOverriddenConfigs(String quorum) throws Exception {
+        String config = "file.delete.delay.ms=1000";
+        createAndWaitTopic(buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "2",
+            "--replication-factor", "2", "--topic", testTopicName, "--config", config));
+        String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe"));
+        assertTrue(output.contains(config), String.format("Describe output should have contained %s", config));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeAndListTopicsWithoutInternalTopics(String quorum) throws Exception {
+        createAndWaitTopic(
+            buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "1", "--replication-factor", "1", "--topic", testTopicName));
+        // create a internal topic
+        createAndWaitTopic(
+            buildTopicCommandOptionsWithBootstrap("--create", "--partitions", "1", "--replication-factor", "1", "--topic", Topic.GROUP_METADATA_TOPIC_NAME));
+
+        // test describe
+        String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--describe", "--exclude-internal"));
+        assertTrue(output.contains(testTopicName),
+            String.format("Output should have contained %s", testTopicName));
+        assertFalse(output.contains(Topic.GROUP_METADATA_TOPIC_NAME));
+
+        // test list
+        output = captureListTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--list", "--exclude-internal"));
+        assertTrue(output.contains(testTopicName));
+        assertFalse(output.contains(Topic.GROUP_METADATA_TOPIC_NAME));
+    }
+
+    @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME)
+    @ValueSource(strings = {"zk", "kraft"})
+    public void testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized(String quorum) throws ExecutionException, InterruptedException {

Review Comment:
   good catch I updated the test. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org