From 9af1e74b5e94536ebf2526af82d9d585a0e1e820 Mon Sep 17 00:00:00 2001 From: "Omnia G.H Ibrahim" Date: Tue, 17 Oct 2023 10:40:15 +0100 Subject: [PATCH] KAFKA-14596: Move TopicCommand to tools (#13201) Reviewers: Mickael Maison , Federico Valeri --- bin/kafka-topics.sh | 2 +- bin/windows/kafka-topics.bat | 2 +- build.gradle | 5 +- checkstyle/import-control.xml | 3 + .../main/scala/kafka/admin/TopicCommand.scala | 649 ---------- .../admin/TopicCommandIntegrationTest.scala | 831 ------------- .../unit/kafka/admin/TopicCommandTest.scala | 246 ---- .../org/apache/kafka/tools/ToolsUtils.java | 17 + .../org/apache/kafka/tools/TopicCommand.java | 1003 ++++++++++++++++ .../apache/kafka/tools/ToolsTestUtils.java | 143 +++ .../tools/TopicCommandIntegrationTest.java | 1065 +++++++++++++++++ .../apache/kafka/tools/TopicCommandTest.java | 283 +++++ 12 files changed, 2517 insertions(+), 1732 deletions(-) delete mode 100755 core/src/main/scala/kafka/admin/TopicCommand.scala delete mode 100644 core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala delete mode 100644 core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala create mode 100644 tools/src/main/java/org/apache/kafka/tools/TopicCommand.java create mode 100644 tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java create mode 100644 tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java diff --git a/bin/kafka-topics.sh b/bin/kafka-topics.sh index ad6a2d4d2a0..05069f45cc4 100755 --- a/bin/kafka-topics.sh +++ b/bin/kafka-topics.sh @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -exec $(dirname $0)/kafka-run-class.sh kafka.admin.TopicCommand "$@" +exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.tools.TopicCommand "$@" diff --git a/bin/windows/kafka-topics.bat b/bin/windows/kafka-topics.bat index 677b09d077d..82a2e96f70d 100644 --- a/bin/windows/kafka-topics.bat +++ b/bin/windows/kafka-topics.bat @@ -14,4 +14,4 @@ rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. rem See the License for the specific language governing permissions and rem limitations under the License. -"%~dp0kafka-run-class.bat" kafka.admin.TopicCommand %* +"%~dp0kafka-run-class.bat" org.apache.kafka.tools.TopicCommand %* diff --git a/build.gradle b/build.gradle index d8c9857cf46..c88ac7b189d 100644 --- a/build.gradle +++ b/build.gradle @@ -1882,17 +1882,14 @@ project(':tools:tools-api') { project(':tools') { archivesBaseName = "kafka-tools" - dependencies { implementation project(':clients') + implementation project(':storage') implementation project(':server-common') - implementation project(':connect:api') implementation project(':connect:runtime') implementation project(':log4j-appender') implementation project(':tools:tools-api') implementation libs.argparse4j - implementation libs.jacksonDatabind - implementation libs.jacksonJDK8Datatypes implementation libs.slf4jApi implementation libs.log4j implementation libs.joptSimple diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 42488c3225f..22195e78ee6 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -273,6 +273,9 @@ + + + diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala deleted file mode 100755 index 568dd1798a1..00000000000 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ /dev/null @@ -1,649 +0,0 @@ -/** - * 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 kafka.admin - -import java.util -import java.util.{Collections, Optional, Properties} -import joptsimple._ -import kafka.utils._ -import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.clients.admin.CreatePartitionsOptions -import org.apache.kafka.clients.admin.CreateTopicsOptions -import org.apache.kafka.clients.admin.DeleteTopicsOptions -import org.apache.kafka.clients.admin.{Admin, ListTopicsOptions, NewPartitions, NewTopic, PartitionReassignment, Config => JConfig} -import org.apache.kafka.common.{TopicCollection, TopicPartition, TopicPartitionInfo, Uuid} -import org.apache.kafka.common.config.ConfigResource.Type -import org.apache.kafka.common.config.{ConfigResource, TopicConfig} -import org.apache.kafka.common.errors.{ClusterAuthorizationException, TopicExistsException, UnsupportedVersionException} -import org.apache.kafka.common.internals.Topic -import org.apache.kafka.common.utils.Utils -import org.apache.kafka.server.common.{AdminCommandFailedException, AdminOperationException} -import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils} -import org.apache.kafka.storage.internals.log.LogConfig -import org.apache.kafka.server.util.TopicFilter.IncludeList - -import scala.annotation.nowarn -import scala.jdk.CollectionConverters._ -import scala.collection._ -import scala.compat.java8.OptionConverters._ -import scala.concurrent.ExecutionException - -object TopicCommand extends Logging { - - def main(args: Array[String]): Unit = { - val opts = new TopicCommandOptions(args) - opts.checkArgs() - - val topicService = TopicService(opts.commandConfig, opts.bootstrapServer) - - var 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 { - case e: ExecutionException => - if (e.getCause != null) - printException(e.getCause) - else - printException(e) - exitCode = 1 - case e: Throwable => - printException(e) - exitCode = 1 - } finally { - topicService.close() - Exit.exit(exitCode) - } - } - - private def printException(e: Throwable): Unit = { - println("Error while executing topic command : " + e.getMessage) - error(Utils.stackTrace(e)) - } - - class CommandTopicPartition(opts: TopicCommandOptions) { - val name = opts.topic.get - val partitions = opts.partitions - val replicationFactor = opts.replicationFactor - val replicaAssignment = opts.replicaAssignment - val configsToAdd = parseTopicConfigsToBeAdded(opts) - - def hasReplicaAssignment: Boolean = replicaAssignment.isDefined - def ifTopicDoesntExist(): Boolean = opts.ifNotExists - } - - case class TopicDescription(topic: String, - topicId: Uuid, - numPartitions: Int, - replicationFactor: Int, - config: JConfig, - markedForDeletion: Boolean) { - - def printDescription(): Unit = { - val configsAsString = config.entries.asScala.filterNot(_.isDefault).map { ce => s"${ce.name}=${ce.value}" }.mkString(",") - print(s"Topic: $topic") - if(topicId != Uuid.ZERO_UUID) print(s"\tTopicId: $topicId") - print(s"\tPartitionCount: $numPartitions") - print(s"\tReplicationFactor: $replicationFactor") - print(s"\tConfigs: $configsAsString") - print(if (markedForDeletion) "\tMarkedForDeletion: true" else "") - println() - } - } - - case class PartitionDescription(topic: String, - info: TopicPartitionInfo, - config: Option[JConfig], - markedForDeletion: Boolean, - reassignment: Option[PartitionReassignment]) { - - private def minIsrCount: Option[Int] = { - config.map(_.get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value.toInt) - } - - def isUnderReplicated: Boolean = { - getReplicationFactor(info, reassignment) - info.isr.size > 0 - } - - private def hasLeader: Boolean = { - info.leader != null - } - - def isUnderMinIsr: Boolean = { - !hasLeader || minIsrCount.exists(info.isr.size < _) - } - - def isAtMinIsrPartitions: Boolean = { - minIsrCount.contains(info.isr.size) - } - - def hasUnavailablePartitions(liveBrokers: Set[Int]): Boolean = { - !hasLeader || !liveBrokers.contains(info.leader.id) - } - - def printDescription(): Unit = { - print("\tTopic: " + topic) - print("\tPartition: " + info.partition) - print("\tLeader: " + (if (hasLeader) info.leader.id else "none")) - print("\tReplicas: " + info.replicas.asScala.map(_.id).mkString(",")) - print("\tIsr: " + info.isr.asScala.map(_.id).mkString(",")) - if (reassignment.nonEmpty) { - print("\tAdding Replicas: " + reassignment.get.addingReplicas().asScala.mkString(",")) - print("\tRemoving Replicas: " + reassignment.get.removingReplicas().asScala.mkString(",")) - } - print(if (markedForDeletion) "\tMarkedForDeletion: true" else "") - println() - } - - } - - class DescribeOptions(opts: TopicCommandOptions, liveBrokers: Set[Int]) { - val describeConfigs = - !opts.reportUnavailablePartitions && - !opts.reportUnderReplicatedPartitions && - !opts.reportUnderMinIsrPartitions && - !opts.reportAtMinIsrPartitions - val describePartitions = !opts.reportOverriddenConfigs - - private def shouldPrintUnderReplicatedPartitions(partitionDescription: PartitionDescription): Boolean = { - opts.reportUnderReplicatedPartitions && partitionDescription.isUnderReplicated - } - private def shouldPrintUnavailablePartitions(partitionDescription: PartitionDescription): Boolean = { - opts.reportUnavailablePartitions && partitionDescription.hasUnavailablePartitions(liveBrokers) - } - private def shouldPrintUnderMinIsrPartitions(partitionDescription: PartitionDescription): Boolean = { - opts.reportUnderMinIsrPartitions && partitionDescription.isUnderMinIsr - } - private def shouldPrintAtMinIsrPartitions(partitionDescription: PartitionDescription): Boolean = { - opts.reportAtMinIsrPartitions && partitionDescription.isAtMinIsrPartitions - } - - private def shouldPrintTopicPartition(partitionDesc: PartitionDescription): Boolean = { - describeConfigs || - shouldPrintUnderReplicatedPartitions(partitionDesc) || - shouldPrintUnavailablePartitions(partitionDesc) || - shouldPrintUnderMinIsrPartitions(partitionDesc) || - shouldPrintAtMinIsrPartitions(partitionDesc) - } - - def maybePrintPartitionDescription(desc: PartitionDescription): Unit = { - if (shouldPrintTopicPartition(desc)) - desc.printDescription() - } - } - - object TopicService { - def createAdminClient(commandConfig: Properties, bootstrapServer: Option[String]): Admin = { - bootstrapServer match { - case Some(serverList) => commandConfig.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, serverList) - case None => - } - Admin.create(commandConfig) - } - - def apply(commandConfig: Properties, bootstrapServer: Option[String]): TopicService = - new TopicService(createAdminClient(commandConfig, bootstrapServer)) - } - - case class TopicService private (adminClient: Admin) extends AutoCloseable { - - def createTopic(opts: TopicCommandOptions): Unit = { - val topic = new CommandTopicPartition(opts) - if (Topic.hasCollisionChars(topic.name)) - 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) - } - - def createTopic(topic: CommandTopicPartition): Unit = { - if (topic.replicationFactor.exists(rf => rf > Short.MaxValue || rf < 1)) - throw new IllegalArgumentException(s"The replication factor must be between 1 and ${Short.MaxValue} inclusive") - if (topic.partitions.exists(partitions => partitions < 1)) - throw new IllegalArgumentException(s"The partitions must be greater than 0") - - try { - val newTopic = if (topic.hasReplicaAssignment) - new NewTopic(topic.name, asJavaReplicaReassignment(topic.replicaAssignment.get)) - else { - new NewTopic( - topic.name, - topic.partitions.asJava, - topic.replicationFactor.map(_.toShort).map(Short.box).asJava) - } - - val configsMap = topic.configsToAdd.stringPropertyNames() - .asScala - .map(name => name -> topic.configsToAdd.getProperty(name)) - .toMap.asJava - - newTopic.configs(configsMap) - val createResult = adminClient.createTopics(Collections.singleton(newTopic), - new CreateTopicsOptions().retryOnQuotaViolation(false)) - createResult.all().get() - println(s"Created topic ${topic.name}.") - } catch { - case e : ExecutionException => - if (e.getCause == null) - throw e - if (!(e.getCause.isInstanceOf[TopicExistsException] && topic.ifTopicDoesntExist())) - throw e.getCause - } - } - - def listTopics(opts: TopicCommandOptions): Unit = { - println(getTopics(opts.topic, opts.excludeInternalTopics).mkString("\n")) - } - - def alterTopic(opts: TopicCommandOptions): Unit = { - val topic = new CommandTopicPartition(opts) - val topics = getTopics(opts.topic, opts.excludeInternalTopics) - ensureTopicExists(topics, opts.topic, !opts.ifExists) - - if (topics.nonEmpty) { - val topicsInfo = adminClient.describeTopics(topics.asJavaCollection).topicNameValues() - val newPartitions = topics.map { topicName => - if (topic.hasReplicaAssignment) { - val startPartitionId = topicsInfo.get(topicName).get().partitions().size() - val newAssignment = { - val replicaMap = topic.replicaAssignment.get.drop(startPartitionId) - new util.ArrayList(replicaMap.map(p => p._2.asJava).asJavaCollection).asInstanceOf[util.List[util.List[Integer]]] - } - topicName -> NewPartitions.increaseTo(topic.partitions.get, newAssignment) - } else { - topicName -> NewPartitions.increaseTo(topic.partitions.get) - } - }.toMap - adminClient.createPartitions(newPartitions.asJava, - new CreatePartitionsOptions().retryOnQuotaViolation(false)).all().get() - } - } - - def listAllReassignments(topicPartitions: util.Set[TopicPartition]): Map[TopicPartition, PartitionReassignment] = { - try { - adminClient.listPartitionReassignments(topicPartitions).reassignments().get().asScala - } catch { - case e: ExecutionException => - e.getCause match { - case ex @ (_: UnsupportedVersionException | _: ClusterAuthorizationException) => - logger.debug(s"Couldn't query reassignments through the AdminClient API: ${ex.getMessage}", ex) - Map() - case t => throw t - } - } - } - - def describeTopic(opts: TopicCommandOptions): Unit = { - // If topicId is provided and not zero, will use topicId regardless of topic name - val inputTopicId = opts.topicId.map(Uuid.fromString).filter(uuid => uuid != Uuid.ZERO_UUID) - val useTopicId = inputTopicId.nonEmpty - - val (topicIds, topics) = if (useTopicId) - (getTopicIds(inputTopicId, opts.excludeInternalTopics), Seq()) - else - (Seq(), getTopics(opts.topic, opts.excludeInternalTopics)) - - // Only check topic name when topicId is not provided - if (useTopicId) - ensureTopicIdExists(topicIds, inputTopicId, !opts.ifExists) - else - ensureTopicExists(topics, opts.topic, !opts.ifExists) - - val topicDescriptions = if (topicIds.nonEmpty) { - adminClient.describeTopics(TopicCollection.ofTopicIds(topicIds.toSeq.asJavaCollection)).allTopicIds().get().values().asScala - } else if (topics.nonEmpty) { - adminClient.describeTopics(TopicCollection.ofTopicNames(topics.asJavaCollection)).allTopicNames().get().values().asScala - } else { - Seq() - } - - val topicNames = topicDescriptions.map(_.name()) - val allConfigs = adminClient.describeConfigs(topicNames.map(new ConfigResource(Type.TOPIC, _)).asJavaCollection).values() - val liveBrokers = adminClient.describeCluster().nodes().get().asScala.map(_.id()) - val describeOptions = new DescribeOptions(opts, liveBrokers.toSet) - val topicPartitions = topicDescriptions - .flatMap(td => td.partitions.iterator().asScala.map(p => new TopicPartition(td.name(), p.partition()))) - .toSet.asJava - val reassignments = listAllReassignments(topicPartitions) - - for (td <- topicDescriptions) { - val topicName = td.name - val topicId = td.topicId() - val config = allConfigs.get(new ConfigResource(Type.TOPIC, topicName)).get() - val sortedPartitions = td.partitions.asScala.sortBy(_.partition) - - if (describeOptions.describeConfigs) { - val hasNonDefault = config.entries().asScala.exists(!_.isDefault) - if (!opts.reportOverriddenConfigs || hasNonDefault) { - val numPartitions = td.partitions().size - val firstPartition = td.partitions.iterator.next() - val reassignment = reassignments.get(new TopicPartition(td.name, firstPartition.partition)) - val topicDesc = TopicDescription(topicName, topicId, numPartitions, getReplicationFactor(firstPartition, reassignment), config, markedForDeletion = false) - topicDesc.printDescription() - } - } - - if (describeOptions.describePartitions) { - for (partition <- sortedPartitions) { - val reassignment = reassignments.get(new TopicPartition(td.name, partition.partition)) - val partitionDesc = PartitionDescription(topicName, partition, Some(config), markedForDeletion = false, reassignment) - describeOptions.maybePrintPartitionDescription(partitionDesc) - } - } - } - } - - def deleteTopic(opts: TopicCommandOptions): Unit = { - val topics = getTopics(opts.topic, opts.excludeInternalTopics) - ensureTopicExists(topics, opts.topic, !opts.ifExists) - adminClient.deleteTopics(topics.asJavaCollection, new DeleteTopicsOptions().retryOnQuotaViolation(false)) - .all().get() - } - - def getTopics(topicIncludeList: Option[String], excludeInternalTopics: Boolean = false): Seq[String] = { - val allTopics = if (excludeInternalTopics) { - adminClient.listTopics() - } else { - adminClient.listTopics(new ListTopicsOptions().listInternal(true)) - } - doGetTopics(allTopics.names().get().asScala.toSeq.sorted, topicIncludeList, excludeInternalTopics) - } - - def getTopicIds(topicIdIncludeList: Option[Uuid], excludeInternalTopics: Boolean = false): Seq[Uuid] = { - val allTopics = if (excludeInternalTopics) { - adminClient.listTopics() - } else { - adminClient.listTopics(new ListTopicsOptions().listInternal(true)) - } - val allTopicIds = allTopics.listings().get().asScala.map(_.topicId()).toSeq.sorted - topicIdIncludeList.filter(allTopicIds.contains).toSeq - } - - def close(): Unit = adminClient.close() - } - - /** - * 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 def ensureTopicExists(foundTopics: Seq[String], requestedTopic: Option[String], requireTopicExists: Boolean): Unit = { - // If no topic name was mentioned, do not need to throw exception. - if (requestedTopic.isDefined && requireTopicExists && foundTopics.isEmpty) { - // If given topic doesn't exist then throw exception - throw new IllegalArgumentException(s"Topic '${requestedTopic.get}' does not exist as expected") - } - } - - /** - * 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 def ensureTopicIdExists(foundTopicIds: Seq[Uuid], requestedTopicId: Option[Uuid], requireTopicIdExists: Boolean): Unit = { - // If no topic id was mentioned, do not need to throw exception. - if (requestedTopicId.isDefined && requireTopicIdExists && foundTopicIds.isEmpty) { - // If given topicId doesn't exist then throw exception - throw new IllegalArgumentException(s"TopicId '${requestedTopicId.get}' does not exist as expected") - } - } - - private def doGetTopics(allTopics: Seq[String], topicIncludeList: Option[String], excludeInternalTopics: Boolean): Seq[String] = { - if (topicIncludeList.isDefined) { - val topicsFilter = new IncludeList(topicIncludeList.get) - allTopics.filter(topicsFilter.isTopicAllowed(_, excludeInternalTopics)) - } else - allTopics.filterNot(Topic.isInternal(_) && excludeInternalTopics) - } - - @nowarn("cat=deprecation") - def parseTopicConfigsToBeAdded(opts: TopicCommandOptions): Properties = { - val configsToBeAdded = opts.topicConfig.getOrElse(Collections.emptyList()).asScala.map(_.split("""\s*=\s*""")) - require(configsToBeAdded.forall(config => config.length == 2), - "Invalid topic config: all configs to be added must be in the format \"key=val\".") - val props = new Properties - configsToBeAdded.foreach(pair => props.setProperty(pair(0).trim, pair(1).trim)) - LogConfig.validate(props) - if (props.containsKey(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG)) { - println(s"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.") - } - props - } - - def parseReplicaAssignment(replicaAssignmentList: String): Map[Int, List[Int]] = { - val partitionList = replicaAssignmentList.split(",") - val ret = new mutable.LinkedHashMap[Int, List[Int]]() - for (i <- 0 until partitionList.size) { - val brokerList = partitionList(i).split(":").map(s => s.trim().toInt) - val duplicateBrokers = CoreUtils.duplicates(brokerList) - if (duplicateBrokers.nonEmpty) - throw new AdminCommandFailedException(s"Partition replica lists may not contain duplicate entries: ${duplicateBrokers.mkString(",")}") - ret.put(i, brokerList.toList) - if (ret(i).size != ret(0).size) - throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList) - } - ret - } - - def asJavaReplicaReassignment(original: Map[Int, List[Int]]): util.Map[Integer, util.List[Integer]] = { - original.map(f => Integer.valueOf(f._1) -> f._2.map(e => Integer.valueOf(e)).asJava).asJava - } - - private def getReplicationFactor(tpi: TopicPartitionInfo, reassignment: Option[PartitionReassignment]): Int = { - // 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. - def isReassignmentInProgress(ra: PartitionReassignment): Boolean = { - // Reassignment is still in progress as long as the removing and adding replicas are still present - val allReplicaIds = tpi.replicas.asScala.map(_.id).toSet - val changingReplicaIds = ra.removingReplicas.asScala.map(_.intValue).toSet ++ ra.addingReplicas.asScala.map(_.intValue).toSet - allReplicaIds.exists(changingReplicaIds.contains) - } - - reassignment match { - case Some(ra) if isReassignmentInProgress(ra) => ra.replicas.asScala.diff(ra.addingReplicas.asScala).size - case _=> tpi.replicas.size - } - } - - class TopicCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) { - private val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The Kafka server to connect to.") - .withRequiredArg - .describedAs("server to connect to") - .ofType(classOf[String]) - - private val 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(classOf[String]) - - private val kafkaConfigsCanAlterTopicConfigsViaBootstrapServer = - " (the kafka-configs CLI supports altering topic configs with a --bootstrap-server option)" - - private val listOpt = parser.accepts("list", "List all available topics.") - private val createOpt = parser.accepts("create", "Create a new topic.") - private val deleteOpt = parser.accepts("delete", "Delete a topic") - private val alterOpt = parser.accepts("alter", "Alter the number of partitions and replica assignment. " + - "Update the configuration of an existing topic via --alter is no longer supported here" + kafkaConfigsCanAlterTopicConfigsViaBootstrapServer + ".") - private val describeOpt = parser.accepts("describe", "List details for the given topics.") - private val topicOpt = parser.accepts("topic", "The topic to create, alter, describe or delete. It also accepts a regular " + - "expression, except for --create option. Put topic name in double quotes and use the '\\' prefix " + - "to escape regular expression symbols; e.g. \"test\\.topic\".") - .withRequiredArg - .describedAs("topic") - .ofType(classOf[String]) - private val 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(classOf[String]) - 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 + ".") - .withRequiredArg - .describedAs("name=value") - .ofType(classOf[String]) - private val deleteConfigOpt = parser.accepts("delete-config", "A topic configuration override to be removed for an existing topic (see the list of configurations under the --config option). " + - "Not supported with the --bootstrap-server option.") - .withRequiredArg - .describedAs("name") - .ofType(classOf[String]) - private val partitionsOpt = parser.accepts("partitions", "The number of partitions for the topic being created or " + - "altered (WARNING: If partitions are increased for a topic that has a key, the partition logic or ordering of the messages will be affected). If not supplied for create, defaults to the cluster default.") - .withRequiredArg - .describedAs("# of partitions") - .ofType(classOf[java.lang.Integer]) - private val replicationFactorOpt = parser.accepts("replication-factor", "The replication factor for each partition in the topic being created. If not supplied, defaults to the cluster default.") - .withRequiredArg - .describedAs("replication factor") - .ofType(classOf[java.lang.Integer]) - private val replicaAssignmentOpt = parser.accepts("replica-assignment", "A list of manual partition-to-broker assignments for the topic being created or altered.") - .withRequiredArg - .describedAs("broker_id_for_part1_replica1 : broker_id_for_part1_replica2 , " + - "broker_id_for_part2_replica1 : broker_id_for_part2_replica2 , ...") - .ofType(classOf[String]) - private val reportUnderReplicatedPartitionsOpt = parser.accepts("under-replicated-partitions", - "if set when describing topics, only show under replicated partitions") - private val reportUnavailablePartitionsOpt = parser.accepts("unavailable-partitions", - "if set when describing topics, only show partitions whose leader is not available") - private val reportUnderMinIsrPartitionsOpt = parser.accepts("under-min-isr-partitions", - "if set when describing topics, only show partitions whose isr count is less than the configured minimum.") - private val reportAtMinIsrPartitionsOpt = parser.accepts("at-min-isr-partitions", - "if set when describing topics, only show partitions whose isr count is equal to the configured minimum.") - private val topicsWithOverridesOpt = parser.accepts("topics-with-overrides", - "if set when describing topics, only show topics that have overridden configs") - private val ifExistsOpt = parser.accepts("if-exists", - "if set when altering or deleting or describing topics, the action will only execute if the topic exists.") - private val ifNotExistsOpt = parser.accepts("if-not-exists", - "if set when creating topics, the action will only execute if the topic does not already exist.") - - private val excludeInternalTopicOpt = parser.accepts("exclude-internal", - "exclude internal topics when running list or describe command. The internal topics will be listed by default") - - options = parser.parse(args : _*) - - private val allTopicLevelOpts = immutable.Set[OptionSpec[_]](alterOpt, createOpt, describeOpt, listOpt, deleteOpt) - - private val allReplicationReportOpts = Set(reportUnderReplicatedPartitionsOpt, reportUnderMinIsrPartitionsOpt, reportAtMinIsrPartitionsOpt, reportUnavailablePartitionsOpt) - - def has(builder: OptionSpec[_]): Boolean = options.has(builder) - def valueAsOption[A](option: OptionSpec[A], defaultValue: Option[A] = None): Option[A] = if (has(option)) Some(options.valueOf(option)) else defaultValue - def valuesAsOption[A](option: OptionSpec[A], defaultValue: Option[util.List[A]] = None): Option[util.List[A]] = if (has(option)) Some(options.valuesOf(option)) else defaultValue - - def hasCreateOption: Boolean = has(createOpt) - def hasAlterOption: Boolean = has(alterOpt) - def hasListOption: Boolean = has(listOpt) - def hasDescribeOption: Boolean = has(describeOpt) - def hasDeleteOption: Boolean = has(deleteOpt) - - def bootstrapServer: Option[String] = valueAsOption(bootstrapServerOpt) - def commandConfig: Properties = if (has(commandConfigOpt)) Utils.loadProps(options.valueOf(commandConfigOpt)) else new Properties() - def topic: Option[String] = valueAsOption(topicOpt) - def topicId: Option[String] = valueAsOption(topicIdOpt) - def partitions: Option[Integer] = valueAsOption(partitionsOpt) - def replicationFactor: Option[Integer] = valueAsOption(replicationFactorOpt) - def replicaAssignment: Option[Map[Int, List[Int]]] = - if (has(replicaAssignmentOpt) && Option(options.valueOf(replicaAssignmentOpt)).getOrElse("").nonEmpty) - Some(parseReplicaAssignment(options.valueOf(replicaAssignmentOpt))) - else - None - def reportUnderReplicatedPartitions: Boolean = has(reportUnderReplicatedPartitionsOpt) - def reportUnavailablePartitions: Boolean = has(reportUnavailablePartitionsOpt) - def reportUnderMinIsrPartitions: Boolean = has(reportUnderMinIsrPartitionsOpt) - def reportAtMinIsrPartitions: Boolean = has(reportAtMinIsrPartitionsOpt) - def reportOverriddenConfigs: Boolean = has(topicsWithOverridesOpt) - def ifExists: Boolean = has(ifExistsOpt) - def ifNotExists: Boolean = has(ifNotExistsOpt) - def excludeInternalTopics: Boolean = has(excludeInternalTopicOpt) - def topicConfig: Option[util.List[String]] = valuesAsOption(configOpt) - def configsToDelete: Option[util.List[String]] = valuesAsOption(deleteConfigOpt) - - def checkArgs(): Unit = { - if (args.isEmpty) - CommandLineUtils.printUsageAndExit(parser, "Create, delete, describe, or change a topic.") - - CommandLineUtils.maybePrintHelpOrVersion(this, "This tool helps to create, delete, describe, or change a topic.") - - // should have exactly one action - val actions = Seq(createOpt, listOpt, alterOpt, describeOpt, deleteOpt).count(options.has) - if (actions != 1) - CommandLineUtils.printUsageAndExit(parser, "Command must include exactly one action: --list, --describe, --create, --alter or --delete") - - // check required args - if (!has(bootstrapServerOpt)) - throw new IllegalArgumentException("--bootstrap-server must be specified") - if (has(describeOpt) && has(ifExistsOpt)) { - if (!has(topicOpt) && !has(topicIdOpt)) - CommandLineUtils.printUsageAndExit(parser, "--topic or --topic-id is required to describe a topic") - if (has(topicOpt) && has(topicIdOpt)) - println("Only topic id will be used when both --topic and --topic-id are specified and topicId is not Uuid.ZERO_UUID") - } - if (!has(listOpt) && !has(describeOpt)) - CommandLineUtils.checkRequiredArgs(parser, options, topicOpt) - if (has(alterOpt)) { - val usedOptions = immutable.Set[OptionSpec[_]](bootstrapServerOpt, configOpt) - val invalidOptions = immutable.Set[OptionSpec[_]](alterOpt) - CommandLineUtils.checkInvalidArgsSet(parser, options, usedOptions.asJava, invalidOptions.asJava, Optional.of(kafkaConfigsCanAlterTopicConfigsViaBootstrapServer)) - CommandLineUtils.checkRequiredArgs(parser, options, partitionsOpt) - } - - // check invalid args - CommandLineUtils.checkInvalidArgs(parser, options, configOpt, (allTopicLevelOpts -- Set(alterOpt, createOpt)).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, deleteConfigOpt, (allTopicLevelOpts -- Set(alterOpt) ++ Set(bootstrapServerOpt)).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, partitionsOpt, (allTopicLevelOpts -- Set(alterOpt, createOpt)).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, replicationFactorOpt, (allTopicLevelOpts -- Set(createOpt)).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, (allTopicLevelOpts -- Set(createOpt,alterOpt)).asJava) - if (options.has(createOpt)) - CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, partitionsOpt, replicationFactorOpt) - CommandLineUtils.checkInvalidArgs(parser, options, reportUnderReplicatedPartitionsOpt, - (allTopicLevelOpts -- Set(describeOpt) ++ allReplicationReportOpts - reportUnderReplicatedPartitionsOpt + topicsWithOverridesOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, reportUnderMinIsrPartitionsOpt, - (allTopicLevelOpts -- Set(describeOpt) ++ allReplicationReportOpts - reportUnderMinIsrPartitionsOpt + topicsWithOverridesOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, reportAtMinIsrPartitionsOpt, - (allTopicLevelOpts -- Set(describeOpt) ++ allReplicationReportOpts - reportAtMinIsrPartitionsOpt + topicsWithOverridesOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, reportUnavailablePartitionsOpt, - (allTopicLevelOpts -- Set(describeOpt) ++ allReplicationReportOpts - reportUnavailablePartitionsOpt + topicsWithOverridesOpt).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, topicsWithOverridesOpt, - (allTopicLevelOpts -- Set(describeOpt) ++ allReplicationReportOpts).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, ifExistsOpt, (allTopicLevelOpts -- Set(alterOpt, deleteOpt, describeOpt)).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, ifNotExistsOpt, (allTopicLevelOpts -- Set(createOpt)).asJava) - CommandLineUtils.checkInvalidArgs(parser, options, excludeInternalTopicOpt, (allTopicLevelOpts -- Set(listOpt, describeOpt)).asJava) - } - } -} - diff --git a/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala b/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala deleted file mode 100644 index 9e1beaf6adf..00000000000 --- a/core/src/test/scala/integration/kafka/admin/TopicCommandIntegrationTest.scala +++ /dev/null @@ -1,831 +0,0 @@ -/** - * 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 kafka.admin - -import java.util.{Collections, Optional, Properties} -import kafka.admin.TopicCommand.{TopicCommandOptions, TopicService} -import kafka.integration.KafkaServerTestHarness -import kafka.server.KafkaConfig -import kafka.utils.{Logging, TestInfoUtils, TestUtils} -import kafka.zk.{ConfigEntityChangeNotificationZNode, DeleteTopicsTopicZNode} -import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.clients.admin._ -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.config.{ConfigException, ConfigResource, TopicConfig} -import org.apache.kafka.common.errors.{ClusterAuthorizationException, InvalidTopicException, TopicExistsException} -import org.apache.kafka.common.internals.Topic -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.Assertions._ -import org.junit.jupiter.api.function.Executable -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource -import org.mockito.Mockito._ - -import scala.collection.Seq -import scala.concurrent.ExecutionException -import scala.jdk.CollectionConverters._ -import scala.util.Random - -class TopicCommandIntegrationTest extends KafkaServerTestHarness with Logging with RackAwareTest { - - /** - * 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 def generateConfigs: Seq[KafkaConfig] = TestUtils.createBrokerConfigs( - numConfigs = numBrokers, - zkConnect = zkConnectOrNull, - rackInfo = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack1", 4 -> "rack3", 5 -> "rack3"), - numPartitions = numPartitions, - defaultReplicationFactor = defaultReplicationFactor, - ).map { props => - props.put(KafkaConfig.ReplicaFetchMaxBytesProp, "1") - KafkaConfig.fromProps(props) - } - - private val numPartitions = 1 - private val defaultReplicationFactor = 1.toShort - private val numBrokers = 6 - private val lineSeparator = System.lineSeparator() - - private var topicService: TopicService = _ - private var adminClient: Admin = _ - private var testTopicName: String = _ - - private[this] def createAndWaitTopic(opts: TopicCommandOptions): Unit = { - topicService.createTopic(opts) - waitForTopicCreated(opts.topic.get) - } - - private[this] def waitForTopicCreated(topicName: String, timeout: Int = 10000): Unit = { - TestUtils.waitForPartitionMetadata(brokers, topicName, partition = 0, timeout) - } - - @BeforeEach - override def setUp(info: TestInfo): Unit = { - super.setUp(info) - - // create adminClient - val props = new Properties() - props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()) - adminClient = Admin.create(props) - topicService = TopicService(adminClient) - testTopicName = s"${info.getTestMethod.get().getName}-${Random.alphanumeric.take(10).mkString}" - } - - @AfterEach - def close(): Unit = { - // adminClient is closed by topicService - if (topicService != null) - topicService.close() - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testCreate(quorum: String): Unit = { - createAndWaitTopic(new TopicCommandOptions( - Array("--partitions", "2", "--replication-factor", "1", "--topic", testTopicName))) - - adminClient.listTopics().names().get().contains(testTopicName) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testCreateWithDefaults(quorum: String): Unit = { - createAndWaitTopic(new TopicCommandOptions(Array("--topic", testTopicName))) - - val partitions = adminClient - .describeTopics(Collections.singletonList(testTopicName)) - .allTopicNames() - .get() - .get(testTopicName) - .partitions() - assertEquals(partitions.size(), numPartitions) - assertEquals(partitions.get(0).replicas().size(), defaultReplicationFactor) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testCreateWithDefaultReplication(quorum: String): Unit = { - createAndWaitTopic(new TopicCommandOptions( - Array("--topic", testTopicName, "--partitions", "2"))) - - val partitions = adminClient - .describeTopics(Collections.singletonList(testTopicName)) - .allTopicNames() - .get() - .get(testTopicName) - .partitions() - assertEquals(partitions.size(), 2) - assertEquals(partitions.get(0).replicas().size(), defaultReplicationFactor) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testCreateWithDefaultPartitions(quorum: String): Unit = { - createAndWaitTopic(new TopicCommandOptions( - Array("--topic", testTopicName, "--replication-factor", "2"))) - - val partitions = adminClient - .describeTopics(Collections.singletonList(testTopicName)) - .allTopicNames() - .get() - .get(testTopicName) - .partitions() - - assertEquals(partitions.size(), numPartitions) - assertEquals(partitions.get(0).replicas().size(), 2) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testCreateWithConfigs(quorum: String): Unit = { - val configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName) - createAndWaitTopic(new TopicCommandOptions( - Array("--partitions", "2", "--replication-factor", "2", "--topic", testTopicName, "--config", "delete.retention.ms=1000"))) - - val configs = adminClient - .describeConfigs(Collections.singleton(configResource)) - .all().get().get(configResource) - assertEquals(1000, Integer.valueOf(configs.get("delete.retention.ms").value())) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testCreateWhenAlreadyExists(quorum: String): Unit = { - val numPartitions = 1 - - // create the topic - val createOpts = new TopicCommandOptions( - Array("--partitions", numPartitions.toString, "--replication-factor", "1", "--topic", testTopicName)) - createAndWaitTopic(createOpts) - - // try to re-create the topic - assertThrows(classOf[TopicExistsException], () => topicService.createTopic(createOpts)) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testCreateWhenAlreadyExistsWithIfNotExists(quorum: String): Unit = { - val createOpts = new TopicCommandOptions(Array("--topic", testTopicName, "--if-not-exists")) - createAndWaitTopic(createOpts) - topicService.createTopic(createOpts) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testCreateWithReplicaAssignment(quorum: String): Unit = { - // create the topic - val createOpts = new TopicCommandOptions( - Array("--replica-assignment", "5:4,3:2,1:0", "--topic", testTopicName)) - createAndWaitTopic(createOpts) - - val partitions = adminClient - .describeTopics(Collections.singletonList(testTopicName)) - .allTopicNames() - .get() - .get(testTopicName) - .partitions() - assertEquals(3, partitions.size()) - assertEquals(List(5, 4), partitions.get(0).replicas().asScala.map(_.id())) - assertEquals(List(3, 2), partitions.get(1).replicas().asScala.map(_.id())) - assertEquals(List(1, 0), partitions.get(2).replicas().asScala.map(_.id())) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testCreateWithInvalidReplicationFactor(quorum: String): Unit = { - assertThrows(classOf[IllegalArgumentException], - () => topicService.createTopic(new TopicCommandOptions( - Array("--partitions", "2", "--replication-factor", (Short.MaxValue+1).toString, "--topic", testTopicName)))) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testCreateWithNegativeReplicationFactor(quorum: String): Unit = { - assertThrows(classOf[IllegalArgumentException], - () => topicService.createTopic(new TopicCommandOptions( - Array("--partitions", "2", "--replication-factor", "-1", "--topic", testTopicName)))) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testCreateWithNegativePartitionCount(quorum: String): Unit = { - assertThrows(classOf[IllegalArgumentException], - () => topicService.createTopic(new TopicCommandOptions( - Array("--partitions", "-1", "--replication-factor", "1", "--topic", testTopicName)))) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testInvalidTopicLevelConfig(quorum: String): Unit = { - val createOpts = new TopicCommandOptions( - Array("--partitions", "1", "--replication-factor", "1", "--topic", testTopicName, - "--config", "message.timestamp.type=boom")) - assertThrows(classOf[ConfigException], () => topicService.createTopic(createOpts)) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testListTopics(quorum: String): Unit = { - createAndWaitTopic(new TopicCommandOptions( - Array("--partitions", "1", "--replication-factor", "1", "--topic", testTopicName))) - - val output = TestUtils.grabConsoleOutput( - topicService.listTopics(new TopicCommandOptions(Array()))) - - assertTrue(output.contains(testTopicName), s"Unexpected output: $output") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testListTopicsWithIncludeList(quorum: String): Unit = { - val topic1 = "kafka.testTopic1" - val topic2 = "kafka.testTopic2" - val topic3 = "oooof.testTopic1" - TestUtils.createTopicWithAdmin(adminClient, topic1, brokers, 2, 2) - TestUtils.createTopicWithAdmin(adminClient, topic2, brokers, 2, 2) - TestUtils.createTopicWithAdmin(adminClient, topic3, brokers, 2, 2) - - val output = TestUtils.grabConsoleOutput( - topicService.listTopics(new TopicCommandOptions(Array("--topic", "kafka.*")))) - - assertTrue(output.contains(topic1), s"Unexpected output: $output") - assertTrue(output.contains(topic2), s"Unexpected output: $output") - assertFalse(output.contains(topic3), s"Unexpected output: $output") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testListTopicsWithExcludeInternal(quorum: String): Unit = { - val topic1 = "kafka.testTopic1" - TestUtils.createTopicWithAdmin(adminClient, topic1, brokers, 2, 2) - TestUtils.createTopicWithAdmin(adminClient, Topic.GROUP_METADATA_TOPIC_NAME, brokers, 2, 2) - - val output = TestUtils.grabConsoleOutput( - topicService.listTopics(new TopicCommandOptions(Array("--exclude-internal")))) - - assertTrue(output.contains(topic1), s"Unexpected output: $output") - assertFalse(output.contains(Topic.GROUP_METADATA_TOPIC_NAME), s"Unexpected output: $output") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testAlterPartitionCount(quorum: String): Unit = { - TestUtils.createTopicWithAdmin(adminClient, testTopicName, brokers, 2, 2) - - topicService.alterTopic(new TopicCommandOptions( - Array("--topic", testTopicName, "--partitions", "3"))) - - TestUtils.waitUntilTrue( - () => brokers.forall(_.metadataCache.getTopicPartitions(testTopicName).size == 3), - "Timeout waiting for new assignment propagating to broker") - val topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get() - assertTrue(topicDescription.partitions().size() == 3) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testAlterAssignment(quorum: String): Unit = { - TestUtils.createTopicWithAdmin(adminClient, testTopicName, brokers, 2, 2) - - topicService.alterTopic(new TopicCommandOptions( - Array("--topic", testTopicName, "--replica-assignment", "5:3,3:1,4:2", "--partitions", "3"))) - TestUtils.waitUntilTrue( - () => brokers.forall(_.metadataCache.getTopicPartitions(testTopicName).size == 3), - "Timeout waiting for new assignment propagating to broker") - - val topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get() - assertTrue(topicDescription.partitions().size() == 3) - assertEquals(List(4,2), topicDescription.partitions().get(2).replicas().asScala.map(_.id())) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testAlterAssignmentWithMoreAssignmentThanPartitions(quorum: String): Unit = { - TestUtils.createTopicWithAdmin(adminClient, testTopicName, brokers, 2, 2) - - assertThrows(classOf[ExecutionException], - () => topicService.alterTopic(new TopicCommandOptions( - Array("--topic", testTopicName, "--replica-assignment", "5:3,3:1,4:2,3:2", "--partitions", "3")))) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testAlterAssignmentWithMorePartitionsThanAssignment(quorum: String): Unit = { - TestUtils.createTopicWithAdmin(adminClient, testTopicName, brokers, 2, 2) - - assertThrows(classOf[ExecutionException], - () => topicService.alterTopic(new TopicCommandOptions( - Array("--topic", testTopicName, "--replica-assignment", "5:3,3:1,4:2", "--partitions", "6")))) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testAlterWithInvalidPartitionCount(quorum: String): Unit = { - createAndWaitTopic(new TopicCommandOptions( - Array("--partitions", "1", "--replication-factor", "1", "--topic", testTopicName))) - - assertThrows(classOf[ExecutionException], - () => topicService.alterTopic(new TopicCommandOptions( - Array("--partitions", "-1", "--topic", testTopicName)))) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testAlterWhenTopicDoesntExist(quorum: String): Unit = { - // alter a topic that does not exist without --if-exists - val alterOpts = new TopicCommandOptions(Array("--topic", testTopicName, "--partitions", "1")) - val topicService = TopicService(adminClient) - assertThrows(classOf[IllegalArgumentException], () => topicService.alterTopic(alterOpts)) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testAlterWhenTopicDoesntExistWithIfExists(quorum: String): Unit = { - topicService.alterTopic(new TopicCommandOptions( - Array("--topic", testTopicName, "--partitions", "1", "--if-exists"))) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testCreateAlterTopicWithRackAware(quorum: String): Unit = { - val rackInfo = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack1", 4 -> "rack3", 5 -> "rack3") - - val numPartitions = 18 - val replicationFactor = 3 - val createOpts = new TopicCommandOptions(Array( - "--partitions", numPartitions.toString, - "--replication-factor", replicationFactor.toString, - "--topic", testTopicName)) - createAndWaitTopic(createOpts) - - var assignment = adminClient.describeTopics(Collections.singletonList(testTopicName)) - .allTopicNames().get().get(testTopicName).partitions() - .asScala.map(info => info.partition() -> info.replicas().asScala.map(_.id())).toMap - checkReplicaDistribution(assignment, rackInfo, rackInfo.size, numPartitions, replicationFactor) - - val alteredNumPartitions = 36 - // verify that adding partitions will also be rack aware - val alterOpts = new TopicCommandOptions(Array( - "--partitions", alteredNumPartitions.toString, - "--topic", testTopicName)) - topicService.alterTopic(alterOpts) - - TestUtils.waitUntilTrue( - () => brokers.forall(_.metadataCache.getTopicPartitions(testTopicName).size == alteredNumPartitions), - "Timeout waiting for new assignment propagating to broker") - assignment = adminClient.describeTopics(Collections.singletonList(testTopicName)) - .allTopicNames().get().get(testTopicName).partitions() - .asScala.map(info => info.partition() -> info.replicas().asScala.map(_.id())).toMap - checkReplicaDistribution(assignment, rackInfo, rackInfo.size, alteredNumPartitions, replicationFactor) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testConfigPreservationAcrossPartitionAlteration(quorum: String): Unit = { - val numPartitionsOriginal = 1 - val cleanupKey = "cleanup.policy" - val cleanupVal = "compact" - - // create the topic - val createOpts = new TopicCommandOptions(Array( - "--partitions", numPartitionsOriginal.toString, - "--replication-factor", "1", - "--config", cleanupKey + "=" + cleanupVal, - "--topic", testTopicName)) - createAndWaitTopic(createOpts) - val configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName) - val 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(ConfigEntityChangeNotificationZNode.path) - } - - // modify the topic to add new partitions - val numPartitionsModified = 3 - val alterOpts = new TopicCommandOptions( - Array("--partitions", numPartitionsModified.toString, "--topic", testTopicName)) - topicService.alterTopic(alterOpts) - val 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 = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testTopicDeletion(quorum: String): Unit = { - // create the NormalTopic - val createOpts = new TopicCommandOptions(Array("--partitions", "1", - "--replication-factor", "1", - "--topic", testTopicName)) - createAndWaitTopic(createOpts) - - // delete the NormalTopic - val deleteOpts = new TopicCommandOptions(Array("--topic", testTopicName)) - - if (!isKRaftTest()) { - val deletePath = 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 = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testTopicWithCollidingCharDeletionAndCreateAgain(quorum: String): Unit = { - // create the topic with colliding chars - val topicWithCollidingChar = "test.a" - val createOpts = new TopicCommandOptions(Array("--partitions", "1", - "--replication-factor", "1", - "--topic", topicWithCollidingChar)) - createAndWaitTopic(createOpts) - - // delete the topic - val deleteOpts = new TopicCommandOptions(Array("--topic", topicWithCollidingChar)) - - if (!isKRaftTest()) { - val deletePath = 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) - - val createTopic: Executable = () => createAndWaitTopic(createOpts) - assertDoesNotThrow(createTopic) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDeleteInternalTopic(quorum: String): Unit = { - // create the offset topic - val createOffsetTopicOpts = new TopicCommandOptions(Array("--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. - val deleteOffsetTopicOpts = new TopicCommandOptions( - Array("--topic", Topic.GROUP_METADATA_TOPIC_NAME)) - val deleteOffsetTopicPath = 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 = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDeleteWhenTopicDoesntExist(quorum: String): Unit = { - // delete a topic that does not exist - val deleteOpts = new TopicCommandOptions(Array("--topic", testTopicName)) - assertThrows(classOf[IllegalArgumentException], () => topicService.deleteTopic(deleteOpts)) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDeleteWhenTopicDoesntExistWithIfExists(quorum: String): Unit = { - topicService.deleteTopic(new TopicCommandOptions(Array("--topic", testTopicName, "--if-exists"))) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribe(quorum: String): Unit = { - TestUtils.createTopicWithAdmin(adminClient, testTopicName, brokers, 2, 2) - - val output = TestUtils.grabConsoleOutput( - topicService.describeTopic(new TopicCommandOptions(Array("--topic", testTopicName)))) - val rows = output.split(lineSeparator) - assertEquals(3, rows.size, s"Unexpected output: $output") - assertTrue(rows(0).startsWith(s"Topic: $testTopicName"), s"Unexpected output: ${rows(0)}") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeWhenTopicDoesntExist(quorum: String): Unit = { - assertThrows(classOf[IllegalArgumentException], - () => topicService.describeTopic(new TopicCommandOptions(Array("--topic", testTopicName)))) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeWhenTopicDoesntExistWithIfExists(quorum: String): Unit = { - topicService.describeTopic(new TopicCommandOptions(Array("--topic", testTopicName, "--if-exists"))) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeUnavailablePartitions(quorum: String): Unit = { - TestUtils.createTopicWithAdmin(adminClient, testTopicName, brokers, numBrokers, 1) - - try { - // check which partition is on broker 0 which we'll kill - val testTopicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)) - .allTopicNames().get().asScala(testTopicName) - val partitionOnBroker0 = testTopicDescription.partitions().asScala.find(_.leader().id() == 0).get.partition() - - killBroker(0) - - // wait until the topic metadata for the test topic is propagated to each alive broker - TestUtils.waitUntilTrue(() => { - brokers - .filterNot(_.config.brokerId == 0) - .foldLeft(true) { - (result, server) => { - val topicMetadatas = server.dataPlaneRequestProcessor.metadataCache - .getTopicMetadata(Set(testTopicName), ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)) - val testPartitionMetadata = topicMetadatas.find(_.name.equals(testTopicName)).get.partitions.asScala.find(_.partitionIndex == partitionOnBroker0) - testPartitionMetadata match { - case None => throw new AssertionError(s"Partition metadata is not found in metadata cache") - case Some(metadata) => result && metadata.errorCode == Errors.LEADER_NOT_AVAILABLE.code - } - } - } - }, s"Partition metadata for $testTopicName is not propagated") - - // grab the console output and assert - val output = TestUtils.grabConsoleOutput( - topicService.describeTopic(new TopicCommandOptions( - Array("--topic", testTopicName, "--unavailable-partitions")))) - val rows = output.split(lineSeparator) - assertTrue(rows(0).startsWith(s"\tTopic: $testTopicName"), s"Unexpected output: ${rows(0)}") - assertTrue(rows(0).contains("Leader: none\tReplicas: 0\tIsr:"), s"Unexpected output: ${rows(0)}") - } finally { - restartDeadBrokers() - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeUnderReplicatedPartitions(quorum: String): Unit = { - TestUtils.createTopicWithAdmin(adminClient, testTopicName, brokers, 1, numBrokers) - - try { - killBroker(0) - if (isKRaftTest()) { - ensureConsistentKRaftMetadata() - } else { - TestUtils.waitForPartitionMetadata(aliveBrokers, testTopicName, 0) - } - val output = TestUtils.grabConsoleOutput( - topicService.describeTopic(new TopicCommandOptions(Array("--under-replicated-partitions")))) - val rows = output.split(lineSeparator) - assertTrue(rows(0).startsWith(s"\tTopic: $testTopicName"), s"Unexpected output: ${rows(0)}") - } finally { - restartDeadBrokers() - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeUnderMinIsrPartitions(quorum: String): Unit = { - val topicProps = new Properties() - topicProps.setProperty(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, numBrokers.toString) - - // create topic - TestUtils.createTopicWithAdmin(adminClient, testTopicName, brokers, 1, numBrokers, topicConfig = topicProps) - - try { - killBroker(0) - if (isKRaftTest()) { - ensureConsistentKRaftMetadata() - } else { - TestUtils.waitUntilTrue( - () => aliveBrokers.forall(_.metadataCache.getPartitionInfo(testTopicName, 0).get.isr().size() == 5), - s"Timeout waiting for partition metadata propagating to brokers for $testTopicName topic" - ) - } - val output = TestUtils.grabConsoleOutput( - topicService.describeTopic(new TopicCommandOptions(Array("--under-min-isr-partitions")))) - val rows = output.split(lineSeparator) - assertTrue(rows(0).startsWith(s"\tTopic: $testTopicName"), s"Unexpected output: ${rows(0)}") - } finally { - restartDeadBrokers() - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress(quorum: String): Unit = { - val tp = new TopicPartition(testTopicName, 0) - - TestUtils.createTopicWithAdmin(adminClient, testTopicName, brokers, 1, 1) - - // Produce multiple batches. - TestUtils.generateAndProduceMessages(brokers, testTopicName, numMessages = 10, acks = -1) - TestUtils.generateAndProduceMessages(brokers, testTopicName, numMessages = 10, acks = -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. - val brokerIds = brokers.map(_.config.brokerId) - TestUtils.setReplicationThrottleForPartitions(adminClient, brokerIds, Set(tp), throttleBytes = 1) - - val testTopicDesc = adminClient.describeTopics(Collections.singleton(testTopicName)).allTopicNames().get().get(testTopicName) - val firstPartition = testTopicDesc.partitions().asScala.head - - val replicasOfFirstPartition = firstPartition.replicas().asScala.map(_.id()) - val targetReplica = brokerIds.diff(replicasOfFirstPartition).head - - adminClient.alterPartitionReassignments(Collections.singletonMap(tp, - Optional.of(new NewPartitionReassignment(Collections.singletonList(targetReplica))))).all().get() - - // let's wait until the LAIR is propagated - TestUtils.waitUntilTrue(() => { - val reassignments = adminClient.listPartitionReassignments(Collections.singleton(tp)).reassignments().get() - !reassignments.get(tp).addingReplicas().isEmpty - }, "Reassignment didn't add the second node") - - // describe the topic and test if it's under-replicated - val simpleDescribeOutput = TestUtils.grabConsoleOutput( - topicService.describeTopic(new TopicCommandOptions(Array("--topic", testTopicName)))) - val simpleDescribeOutputRows = simpleDescribeOutput.split(lineSeparator) - assertTrue(simpleDescribeOutputRows(0).startsWith(s"Topic: $testTopicName"), s"Unexpected output: ${simpleDescribeOutputRows(0)}") - assertEquals(2, simpleDescribeOutputRows.size, s"Unexpected output: $simpleDescribeOutput") - - val underReplicatedOutput = TestUtils.grabConsoleOutput( - topicService.describeTopic(new TopicCommandOptions(Array("--under-replicated-partitions")))) - assertEquals("", underReplicatedOutput, s"--under-replicated-partitions shouldn't return anything: '$underReplicatedOutput'") - - // Verify reassignment is still ongoing. - val reassignments = adminClient.listPartitionReassignments(Collections.singleton(tp)).reassignments.get().get(tp) - assertFalse(Option(reassignments).forall(_.addingReplicas.isEmpty)) - - TestUtils.removeReplicationThrottleForPartitions(adminClient, brokerIds, Set(tp)) - TestUtils.waitForAllReassignmentsToComplete(adminClient) - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeAtMinIsrPartitions(quorum: String): Unit = { - val topicProps = new Properties() - topicProps.setProperty(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "4") - - // create topic - TestUtils.createTopicWithAdmin(adminClient, testTopicName, brokers, 1, numBrokers, topicConfig = topicProps) - - try { - killBroker(0) - killBroker(1) - - if (isKRaftTest()) { - ensureConsistentKRaftMetadata() - } else { - TestUtils.waitUntilTrue( - () => aliveBrokers.forall(_.metadataCache.getPartitionInfo(testTopicName, 0).get.isr().size() == 4), - s"Timeout waiting for partition metadata propagating to brokers for $testTopicName topic" - ) - } - - val output = TestUtils.grabConsoleOutput( - topicService.describeTopic(new TopicCommandOptions(Array("--at-min-isr-partitions")))) - val rows = output.split(lineSeparator) - assertTrue(rows(0).startsWith(s"\tTopic: $testTopicName"), s"Unexpected output: ${rows(0)}") - assertEquals(1, rows.length, s"Unexpected output: $output") - } finally { - restartDeadBrokers() - } - } - - /** - * 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 = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeUnderMinIsrPartitionsMixed(quorum: String): Unit = { - val underMinIsrTopic = "under-min-isr-topic" - val notUnderMinIsrTopic = "not-under-min-isr-topic" - val offlineTopic = "offline-topic" - val fullyReplicatedTopic = "fully-replicated-topic" - val topicProps = new Properties() - topicProps.setProperty(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, numBrokers.toString) - - // create topic - TestUtils.createTopicWithAdmin(adminClient, underMinIsrTopic, brokers, 1, numBrokers, topicConfig = topicProps) - TestUtils.createTopicWithAdmin(adminClient, notUnderMinIsrTopic, brokers, 1, numBrokers) - TestUtils.createTopicWithAdmin(adminClient, offlineTopic, brokers, 1, replicaAssignment = Map(0 -> Seq(0))) - TestUtils.createTopicWithAdmin(adminClient, fullyReplicatedTopic, brokers, 1, replicaAssignment = Map(0 -> Seq(1, 2, 3))) - - try { - killBroker(0) - if (isKRaftTest()) { - ensureConsistentKRaftMetadata() - } else { - TestUtils.waitUntilTrue( - () => aliveBrokers.forall( - broker => - broker.metadataCache.getPartitionInfo(underMinIsrTopic, 0).get.isr().size() < numBrokers && - broker.metadataCache.getPartitionInfo(offlineTopic, 0).get.leader() == MetadataResponse.NO_LEADER_ID), - "Timeout waiting for partition metadata propagating to brokers for underMinIsrTopic topic" - ) - } - val output = TestUtils.grabConsoleOutput( - topicService.describeTopic(new TopicCommandOptions(Array("--under-min-isr-partitions")))) - val rows = output.split(lineSeparator) - assertTrue(rows(0).startsWith(s"\tTopic: $underMinIsrTopic"), s"Unexpected output: ${rows(0)}") - assertTrue(rows(1).startsWith(s"\tTopic: $offlineTopic"), s"Unexpected output: ${rows(1)}") - assertEquals(2, rows.length, s"Unexpected output: $output") - } finally { - restartDeadBrokers() - } - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeReportOverriddenConfigs(quorum: String): Unit = { - val config = "file.delete.delay.ms=1000" - createAndWaitTopic(new TopicCommandOptions( - Array("--partitions", "2", "--replication-factor", "2", "--topic", testTopicName, "--config", config))) - val output = TestUtils.grabConsoleOutput( - topicService.describeTopic(new TopicCommandOptions(Array()))) - assertTrue(output.contains(config), s"Describe output should have contained $config") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeAndListTopicsWithoutInternalTopics(quorum: String): Unit = { - createAndWaitTopic( - new TopicCommandOptions(Array("--partitions", "1", "--replication-factor", "1", "--topic", testTopicName))) - // create a internal topic - createAndWaitTopic( - new TopicCommandOptions(Array("--partitions", "1", "--replication-factor", "1", "--topic", Topic.GROUP_METADATA_TOPIC_NAME))) - - // test describe - var output = TestUtils.grabConsoleOutput(topicService.describeTopic(new TopicCommandOptions( - Array("--describe", "--exclude-internal")))) - assertTrue(output.contains(testTopicName), s"Output should have contained $testTopicName") - assertFalse(output.contains(Topic.GROUP_METADATA_TOPIC_NAME), s"Unexpected output: $output") - - // test list - output = TestUtils.grabConsoleOutput(topicService.listTopics(new TopicCommandOptions(Array("--list", "--exclude-internal")))) - assertTrue(output.contains(testTopicName), s"Unexpected output: $output") - assertFalse(output.contains(Topic.GROUP_METADATA_TOPIC_NAME), s"Unexpected output: $output") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeDoesNotFailWhenListingReassignmentIsUnauthorized(quorum: String): Unit = { - adminClient = spy(adminClient) - topicService = TopicService(adminClient) - - val result = AdminClientTestUtils.listPartitionReassignmentsResult( - new ClusterAuthorizationException("Unauthorized")) - - // Passing `null` here to help the compiler disambiguate the `doReturn` methods, - // compilation for scala 2.12 fails otherwise. - doReturn(result, null).when(adminClient).listPartitionReassignments( - Set(new TopicPartition(testTopicName, 0)).asJava - ) - - TestUtils.createTopicWithAdmin(adminClient, testTopicName, brokers, 1, 1) - - val output = TestUtils.grabConsoleOutput( - topicService.describeTopic(new TopicCommandOptions(Array("--topic", testTopicName)))) - val rows = output.split(lineSeparator) - assertEquals(2, rows.size, s"Unexpected output: $output") - assertTrue(rows(0).startsWith(s"Topic: $testTopicName"), s"Unexpected output: ${rows(0)}") - } - - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) - @ValueSource(strings = Array("zk", "kraft")) - def testCreateWithTopicNameCollision(quorum: String): Unit = { - TestUtils.createTopicWithAdmin(adminClient, "foo_bar", brokers, 1, numBrokers) - - assertThrows(classOf[InvalidTopicException], - () => topicService.createTopic(new TopicCommandOptions(Array("--topic", "foo.bar")))) - } -} diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala deleted file mode 100644 index 088c9dd0205..00000000000 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ /dev/null @@ -1,246 +0,0 @@ -/** - * 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 kafka.admin - -import kafka.admin.TopicCommand.{PartitionDescription, TopicCommandOptions, TopicService} -import kafka.utils.Exit -import org.apache.kafka.clients.admin.{Admin, AdminClientTestUtils, CreatePartitionsOptions, CreateTopicsOptions, DeleteTopicsOptions, NewPartitions, NewTopic, PartitionReassignment, 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, AdminOperationException} -import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test -import org.mockito.ArgumentMatcher -import org.mockito.ArgumentMatchers.{any, argThat, eq => eqThat} -import org.mockito.Mockito.{mock, times, verify, when} - -import java.util.{Collection, Collections, Optional} -import scala.collection.Seq -import scala.concurrent.ExecutionException -import scala.jdk.CollectionConverters._ - -class TopicCommandTest { - - private[this] val brokerList = "localhost:9092" - private[this] val topicName = "topicName" - - @Test - def testIsNotUnderReplicatedWhenAdding(): Unit = { - val replicaIds = List(1, 2) - val replicas = replicaIds.map { id => - new Node(id, "localhost", 9090 + id) - } - - val partitionDescription = PartitionDescription( - "test-topic", - new TopicPartitionInfo( - 0, - new Node(1, "localhost", 9091), - replicas.asJava, - List(new Node(1, "localhost", 9091)).asJava - ), - None, - markedForDeletion = false, - Some( - new PartitionReassignment( - replicaIds.map(id => id: java.lang.Integer).asJava, - List(2: java.lang.Integer).asJava, - List.empty.asJava - ) - ) - ) - - assertFalse(partitionDescription.isUnderReplicated) - } - - @Test - def testAlterWithUnspecifiedPartitionCount(): Unit = { - assertCheckArgsExitCode(1, new TopicCommandOptions( - Array("--bootstrap-server", brokerList ,"--alter", "--topic", topicName))) - } - - @Test - def testConfigOptWithBootstrapServers(): Unit = { - assertCheckArgsExitCode(1, - new TopicCommandOptions(Array("--bootstrap-server", brokerList ,"--alter", "--topic", topicName, "--partitions", "3", "--config", "cleanup.policy=compact"))) - assertCheckArgsExitCode(1, - new TopicCommandOptions(Array("--bootstrap-server", brokerList ,"--alter", "--topic", topicName, "--partitions", "3", "--delete-config", "cleanup.policy"))) - val opts = - new TopicCommandOptions(Array("--bootstrap-server", brokerList ,"--create", "--topic", topicName, "--partitions", "3", "--replication-factor", "3", "--config", "cleanup.policy=compact")) - opts.checkArgs() - assertTrue(opts.hasCreateOption) - assertEquals(brokerList, opts.bootstrapServer.get) - assertEquals("cleanup.policy=compact", opts.topicConfig.get.get(0)) - } - - @Test - def testCreateWithPartitionCountWithoutReplicationFactorShouldSucceed(): Unit = { - val opts = new TopicCommandOptions( - Array("--bootstrap-server", brokerList, - "--create", - "--partitions", "2", - "--topic", topicName)) - opts.checkArgs() - } - - @Test - def testCreateWithReplicationFactorWithoutPartitionCountShouldSucceed(): Unit = { - val opts = new TopicCommandOptions( - Array("--bootstrap-server", brokerList, - "--create", - "--replication-factor", "3", - "--topic", topicName)) - opts.checkArgs() - } - - @Test - def testCreateWithAssignmentAndPartitionCount(): Unit = { - assertCheckArgsExitCode(1, - new TopicCommandOptions( - Array("--bootstrap-server", brokerList, - "--create", - "--replica-assignment", "3:0,5:1", - "--partitions", "2", - "--topic", topicName))) - } - - @Test - def testCreateWithAssignmentAndReplicationFactor(): Unit = { - assertCheckArgsExitCode(1, - new TopicCommandOptions( - Array("--bootstrap-server", brokerList, - "--create", - "--replica-assignment", "3:0,5:1", - "--replication-factor", "2", - "--topic", topicName))) - } - - @Test - def testCreateWithoutPartitionCountAndReplicationFactorShouldSucceed(): Unit = { - val opts = new TopicCommandOptions( - Array("--bootstrap-server", brokerList, - "--create", - "--topic", topicName)) - opts.checkArgs() - } - - @Test - def testDescribeShouldSucceed(): Unit = { - val opts = new TopicCommandOptions( - Array("--bootstrap-server", brokerList, - "--describe", - "--topic", topicName)) - opts.checkArgs() - } - - - @Test - def testParseAssignmentDuplicateEntries(): Unit = { - assertThrows(classOf[AdminCommandFailedException], () => TopicCommand.parseReplicaAssignment("5:5")) - } - - @Test - def testParseAssignmentPartitionsOfDifferentSize(): Unit = { - assertThrows(classOf[AdminOperationException], () => TopicCommand.parseReplicaAssignment("5:4:3,2:1")) - } - - @Test - def testParseAssignment(): Unit = { - val actualAssignment = TopicCommand.parseReplicaAssignment("5:4,3:2,1:0") - val expectedAssignment = Map(0 -> List(5, 4), 1 -> List(3, 2), 2 -> List(1, 0)) - assertEquals(expectedAssignment, actualAssignment) - } - - @Test - def testCreateTopicDoesNotRetryThrottlingQuotaExceededException(): Unit = { - val adminClient = mock(classOf[Admin]) - val topicService = TopicService(adminClient) - - val result = AdminClientTestUtils.createTopicsResult(topicName, Errors.THROTTLING_QUOTA_EXCEEDED.exception()) - when(adminClient.createTopics(any(), any())).thenReturn(result) - - assertThrows(classOf[ThrottlingQuotaExceededException], - () => topicService.createTopic(new TopicCommandOptions(Array("--topic", topicName)))) - - val expectedNewTopic = new NewTopic(topicName, Optional.empty[Integer](), Optional.empty[java.lang.Short]()) - .configs(Map.empty[String, String].asJava) - - verify(adminClient, times(1)).createTopics( - eqThat(Set(expectedNewTopic).asJava), - argThat((_.shouldRetryOnQuotaViolation() == false): ArgumentMatcher[CreateTopicsOptions]) - ) - } - - @Test - def testDeleteTopicDoesNotRetryThrottlingQuotaExceededException(): Unit = { - val adminClient = mock(classOf[Admin]) - val topicService = TopicService(adminClient) - - val listResult = AdminClientTestUtils.listTopicsResult(topicName) - when(adminClient.listTopics(any())).thenReturn(listResult) - - val result = AdminClientTestUtils.deleteTopicsResult(topicName, Errors.THROTTLING_QUOTA_EXCEEDED.exception()) - when(adminClient.deleteTopics(any[Collection[String]](), any())).thenReturn(result) - - val exception = assertThrows(classOf[ExecutionException], - () => topicService.deleteTopic(new TopicCommandOptions(Array("--topic", topicName)))) - assertTrue(exception.getCause.isInstanceOf[ThrottlingQuotaExceededException]) - - verify(adminClient).deleteTopics( - argThat((topics: java.util.Collection[String]) => topics.asScala.toBuffer.equals(Seq(topicName))), - argThat((options: DeleteTopicsOptions) => !options.shouldRetryOnQuotaViolation) - ) - } - - @Test - def testCreatePartitionsDoesNotRetryThrottlingQuotaExceededException(): Unit = { - val adminClient = mock(classOf[Admin]) - val topicService = TopicService(adminClient) - - val listResult = AdminClientTestUtils.listTopicsResult(topicName) - when(adminClient.listTopics(any())).thenReturn(listResult) - - val topicPartitionInfo = new TopicPartitionInfo(0, new Node(0, "", 0), - Collections.emptyList(), Collections.emptyList()) - val describeResult = AdminClientTestUtils.describeTopicsResult(topicName, new TopicDescription( - topicName, false, Collections.singletonList(topicPartitionInfo))) - when(adminClient.describeTopics(any(classOf[java.util.Collection[String]]))).thenReturn(describeResult) - - val result = AdminClientTestUtils.createPartitionsResult(topicName, Errors.THROTTLING_QUOTA_EXCEEDED.exception()) - when(adminClient.createPartitions(any(), any())).thenReturn(result) - - val exception = assertThrows(classOf[ExecutionException], - () => topicService.alterTopic(new TopicCommandOptions(Array("--topic", topicName, "--partitions", "3")))) - assertTrue(exception.getCause.isInstanceOf[ThrottlingQuotaExceededException]) - - verify(adminClient, times(1)).createPartitions( - argThat((_.get(topicName).totalCount() == 3): ArgumentMatcher[java.util.Map[String, NewPartitions]]), - argThat((_.shouldRetryOnQuotaViolation() == false): ArgumentMatcher[CreatePartitionsOptions]) - ) - } - - private[this] def assertCheckArgsExitCode(expected: Int, options: TopicCommandOptions): Unit = { - Exit.setExitProcedure { - (exitCode: Int, _: Option[String]) => - assertEquals(expected, exitCode) - throw new RuntimeException - } - try assertThrows(classOf[RuntimeException], () => options.checkArgs()) finally Exit.resetExitProcedure() - } -} diff --git a/tools/src/main/java/org/apache/kafka/tools/ToolsUtils.java b/tools/src/main/java/org/apache/kafka/tools/ToolsUtils.java index 794f1022293..a3eb4ab4bcb 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ToolsUtils.java +++ b/tools/src/main/java/org/apache/kafka/tools/ToolsUtils.java @@ -22,8 +22,10 @@ import org.apache.kafka.common.utils.Utils; import java.io.PrintStream; import java.util.Arrays; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TreeMap; import java.util.stream.Collectors; @@ -123,4 +125,19 @@ public class ToolsUtils { throw new IllegalArgumentException("Please provide valid host:port like host1:9091,host2:9092\n"); } } + + /** + * Return all duplicates in a list. A duplicated element will appear only once. + */ + public static Set duplicates(List s) { + Set set = new HashSet<>(); + Set duplicates = new HashSet<>(); + + s.forEach(element -> { + if (!set.add(element)) { + duplicates.add(element); + } + }); + return duplicates; + } } diff --git a/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java b/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java new file mode 100644 index 00000000000..e490007bd23 --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java @@ -0,0 +1,1003 @@ +/* + * 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)); + } + + static Map> parseReplicaAssignment(String replicaAssignmentList) { + String[] partitionList = replicaAssignmentList.split(","); + Map> ret = new LinkedHashMap<>(); + for (int i = 0; i < partitionList.length; i++) { + List brokerList = Arrays.stream(partitionList[i].split(":")) + .map(String::trim) + .mapToInt(Integer::parseInt) + .boxed() + .collect(Collectors.toList()); + Collection 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> 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 allReplicaIds = tpi.replicas().stream().map(Node::id).collect(Collectors.toSet()); + Set 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 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 doGetTopics(List allTopics, Optional 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 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 name; + private final Optional partitions; + private final Optional replicationFactor; + private final Map> 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 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 liveBrokers; + private final boolean describeConfigs; + private final boolean describePartitions; + + public DescribeOptions(TopicCommandOptions opts, Set 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 bootstrapServer) { + this.adminClient = createAdminClient(commandConfig, bootstrapServer); + } + + public TopicService(Admin admin) { + this.adminClient = admin; + } + + private static Admin createAdminClient(Properties commandConfig, Optional 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 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 topics = getTopics(opts.topic(), opts.excludeInternalTopics()); + ensureTopicExists(topics, opts.topic().orElse(""), !opts.ifExists()); + + if (!topics.isEmpty()) { + Map> topicsInfo = adminClient.describeTopics(topics).topicNameValues(); + Map 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 topicNewPartitions( + CommandTopicPartition topic, + Map> topicsInfo, + String topicName) { + if (topic.hasReplicaAssignment()) { + try { + Integer startPartitionId = topicsInfo.get(topicName).get().partitions().size(); + Map> replicaMap = topic.replicaAssignment.entrySet().stream() + .skip(startPartitionId) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + List> newAssignment = new ArrayList<>(replicaMap.values()); + 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 listAllReassignments(Set 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 inputTopicId = opts.topicId() + .map(Uuid::fromString).filter(uuid -> uuid != Uuid.ZERO_UUID); + Boolean useTopicId = inputTopicId.isPresent(); + + List topicIds; + List 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 topicDescriptions = new ArrayList<>(); + + if (!topicIds.isEmpty()) { + Map descTopics = + adminClient.describeTopics(TopicCollection.ofTopicIds(topicIds)).allTopicIds().get(); + topicDescriptions = new ArrayList<>(descTopics.values()); + } + + if (!topics.isEmpty()) { + Map descTopics = + adminClient.describeTopics(TopicCollection.ofTopicNames(topics)).allTopicNames().get(); + topicDescriptions = new ArrayList<>(descTopics.values()); + } + + List topicNames = topicDescriptions.stream() + .map(org.apache.kafka.clients.admin.TopicDescription::name) + .collect(Collectors.toList()); + Map> allConfigs = adminClient.describeConfigs( + topicNames.stream() + .map(name -> new ConfigResource(ConfigResource.Type.TOPIC, name)) + .collect(Collectors.toList()) + ).values(); + List liveBrokers = adminClient.describeCluster().nodes().get().stream() + .map(Node::id) + .collect(Collectors.toList()); + DescribeOptions describeOptions = new DescribeOptions(opts, new HashSet<>(liveBrokers)); + Set topicPartitions = topicDescriptions + .stream() + .flatMap(td -> td.partitions().stream() + .map(p -> new TopicPartition(td.name(), p.partition()))) + .collect(Collectors.toSet()); + Map 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 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 reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Config config, ArrayList 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 reassignments, org.apache.kafka.clients.admin.TopicDescription td, String topicName, Uuid topicId, Config config, ArrayList sortedPartitions) { + if (describeOptions.describeConfigs) { + List 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 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 getTopics(Optional topicIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException { + ListTopicsOptions listTopicsOptions = new ListTopicsOptions(); + if (!excludeInternalTopics) { + listTopicsOptions.listInternal(true); + } + + Set allTopics = adminClient.listTopics(listTopicsOptions).names().get(); + return doGetTopics(allTopics.stream().sorted().collect(Collectors.toList()), topicIncludeList, excludeInternalTopics); + } + + public List getTopicIds(Uuid topicIdIncludeList, boolean excludeInternalTopics) throws ExecutionException, InterruptedException { + ListTopicsResult allTopics = excludeInternalTopics ? adminClient.listTopics() : + adminClient.listTopics(new ListTopicsOptions().listInternal(true)); + List 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 final static class TopicCommandOptions extends CommandDefaultOptions { + private final ArgumentAcceptingOptionSpec bootstrapServerOpt; + + private final ArgumentAcceptingOptionSpec commandConfigOpt; + + private final OptionSpecBuilder listOpt; + + private final OptionSpecBuilder createOpt; + + private final OptionSpecBuilder deleteOpt; + + private final OptionSpecBuilder alterOpt; + + private final OptionSpecBuilder describeOpt; + + private final ArgumentAcceptingOptionSpec topicOpt; + + private final ArgumentAcceptingOptionSpec 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 configOpt; + + private final ArgumentAcceptingOptionSpec deleteConfigOpt; + + private final ArgumentAcceptingOptionSpec partitionsOpt; + + private final ArgumentAcceptingOptionSpec replicationFactorOpt; + + private final ArgumentAcceptingOptionSpec 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> allTopicLevelOpts; + + private final Set 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); + + String kafkaConfigsCanAlterTopicConfigsViaBootstrapServer = + " (the kafka-configs CLI supports altering topic configs with a --bootstrap-server option)"; + 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 and replica assignment. " + + "Update the configuration of an existing topic via --alter is no longer supported here" + + kafkaConfigsCanAlterTopicConfigsViaBootstrapServer + "."); + describeOpt = parser.accepts("describe", "List details for the given topics."); + topicOpt = parser.accepts("topic", "The topic to create, alter, describe or delete. It also accepts a regular " + + "expression, except for --create option. Put topic name in double quotes and use the '\\' prefix " + + "to escape regular expression symbols; e.g. \"test\\.topic\".") + .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 logConfigNames = LogConfig.configNames().stream().map(config -> "\t" + config).collect(Collectors.joining(nl)); + configOpt = parser.accepts("config", "A topic configuration override for the topic being created." + + " The following is a list of valid configurations: " + nl + logConfigNames + 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 + ".") + .withRequiredArg() + .describedAs("name=value") + .ofType(String.class); + + deleteConfigOpt = parser.accepts("delete-config", "A topic configuration override to be removed for an existing topic (see the list of configurations under the --config option). " + + "Not supported with the --bootstrap-server option.") + .withRequiredArg() + .describedAs("name") + .ofType(String.class); + partitionsOpt = parser.accepts("partitions", "The number of partitions for the topic being created or " + + "altered (WARNING: If partitions are increased for a topic that has a key, the partition logic or ordering of the messages will be affected). If not supplied for create, defaults to the cluster default.") + .withRequiredArg() + .describedAs("# of partitions") + .ofType(java.lang.Integer.class); + replicationFactorOpt = parser.accepts("replication-factor", "The replication factor for each partition in the topic being created. If not supplied, defaults to the cluster default.") + .withRequiredArg() + .describedAs("replication factor") + .ofType(java.lang.Integer.class); + replicaAssignmentOpt = parser.accepts("replica-assignment", "A list of manual partition-to-broker assignments for the topic being created or altered.") + .withRequiredArg() + .describedAs("broker_id_for_part1_replica1 : broker_id_for_part1_replica2 , " + + "broker_id_for_part2_replica1 : broker_id_for_part2_replica2 , ...") + .ofType(String.class); + reportUnderReplicatedPartitionsOpt = parser.accepts("under-replicated-partitions", + "if set when describing topics, only show under replicated partitions"); + reportUnavailablePartitionsOpt = parser.accepts("unavailable-partitions", + "if set when describing topics, only show partitions whose leader is not available"); + reportUnderMinIsrPartitionsOpt = parser.accepts("under-min-isr-partitions", + "if set when describing topics, only show partitions whose isr count is less than the configured minimum."); + reportAtMinIsrPartitionsOpt = parser.accepts("at-min-isr-partitions", + "if set when describing topics, only show partitions whose isr count is equal to the configured minimum."); + topicsWithOverridesOpt = parser.accepts("topics-with-overrides", + "if set when describing topics, only show topics that have overridden configs"); + ifExistsOpt = parser.accepts("if-exists", + "if set when altering or deleting or describing topics, the action will only execute if the topic exists."); + ifNotExistsOpt = parser.accepts("if-not-exists", + "if set when creating topics, the action will only execute if the topic does not already exist."); + excludeInternalTopicOpt = parser.accepts("exclude-internal", + "exclude internal topics when running list or describe command. The internal topics will be listed by default"); + options = parser.parse(args); + + allTopicLevelOpts = new HashSet<>(Arrays.asList(alterOpt, createOpt, describeOpt, listOpt, deleteOpt)); + allReplicationReportOpts = new HashSet<>(Arrays.asList(reportUnderReplicatedPartitionsOpt, reportUnderMinIsrPartitionsOpt, reportAtMinIsrPartitionsOpt, reportUnavailablePartitionsOpt)); + + checkArgs(); + } + + public Boolean has(OptionSpec builder) { + return options.has(builder); + } + + public Optional valueAsOption(OptionSpec option) { + return valueAsOption(option, Optional.empty()); + } + + public Optional> valuesAsOption(OptionSpec option) { + return valuesAsOption(option, Collections.emptyList()); + } + + public Optional valueAsOption(OptionSpec option, Optional defaultValue) { + if (has(option)) { + return Optional.of(options.valueOf(option)); + } else { + return defaultValue; + } + } + + public Optional> valuesAsOption(OptionSpec option, List defaultValue) { + return options.has(option) ? Optional.of(options.valuesOf(option)) : Optional.of(defaultValue); + } + + public Boolean hasCreateOption() { + return has(createOpt); + } + + public Boolean hasAlterOption() { + return has(alterOpt); + } + + public Boolean hasListOption() { + return has(listOpt); + } + + public Boolean hasDescribeOption() { + return has(describeOpt); + } + + public Boolean hasDeleteOption() { + return has(deleteOpt); + } + + public Optional bootstrapServer() { + return valueAsOption(bootstrapServerOpt); + } + + public Properties commandConfig() throws IOException { + if (has(commandConfigOpt)) { + return Utils.loadProps(options.valueOf(commandConfigOpt)); + } else { + return new Properties(); + } + } + + public Optional topic() { + return valueAsOption(topicOpt); + } + + public Optional topicId() { + return valueAsOption(topicIdOpt); + } + + public Optional partitions() { + return valueAsOption(partitionsOpt); + } + + public Optional replicationFactor() { + return valueAsOption(replicationFactorOpt); + } + + public Optional>> replicaAssignment() { + if (has(replicaAssignmentOpt) && !Optional.of(options.valueOf(replicaAssignmentOpt)).orElse("").isEmpty()) + return Optional.of(parseReplicaAssignment(options.valueOf(replicaAssignmentOpt))); + else + return Optional.empty(); + } + + public Boolean reportUnderReplicatedPartitions() { + return has(reportUnderReplicatedPartitionsOpt); + } + + public Boolean reportUnavailablePartitions() { + return has(reportUnavailablePartitionsOpt); + } + + public Boolean reportUnderMinIsrPartitions() { + return has(reportUnderMinIsrPartitionsOpt); + } + + public Boolean reportAtMinIsrPartitions() { + return has(reportAtMinIsrPartitionsOpt); + } + + public Boolean reportOverriddenConfigs() { + return has(topicsWithOverridesOpt); + } + + public Boolean ifExists() { + return has(ifExistsOpt); + } + + public Boolean ifNotExists() { + return has(ifNotExistsOpt); + } + + public Boolean excludeInternalTopics() { + return has(excludeInternalTopicOpt); + } + + public Optional> topicConfig() { + return valuesAsOption(configOpt); + } + + public void checkArgs() { + if (args.length == 0) + CommandLineUtils.printUsageAndExit(parser, "Create, delete, describe, or change a topic."); + + CommandLineUtils.maybePrintHelpOrVersion(this, "This tool helps to create, delete, describe, or change a topic."); + + // should have exactly one action + long actions = + Arrays.asList(createOpt, listOpt, alterOpt, describeOpt, deleteOpt) + .stream().filter(options::has) + .count(); + if (actions != 1) + CommandLineUtils.printUsageAndExit(parser, "Command must include exactly one action: --list, --describe, --create, --alter or --delete"); + + checkRequiredArgs(); + checkInvalidArgs(); + } + + private void checkRequiredArgs() { + // check required args + if (!has(bootstrapServerOpt)) + throw new IllegalArgumentException("--bootstrap-server must be specified"); + if (has(describeOpt) && has(ifExistsOpt)) { + if (!has(topicOpt) && !has(topicIdOpt)) + CommandLineUtils.printUsageAndExit(parser, "--topic or --topic-id is required to describe a topic"); + if (has(topicOpt) && has(topicIdOpt)) + System.out.println("Only topic id will be used when both --topic and --topic-id are specified and topicId is not Uuid.ZERO_UUID"); + } + if (!has(listOpt) && !has(describeOpt)) + CommandLineUtils.checkRequiredArgs(parser, options, topicOpt); + if (has(alterOpt)) { + Set> usedOptions = new HashSet<>(Arrays.asList(bootstrapServerOpt, configOpt)); + Set> invalidOptions = new HashSet<>(Arrays.asList(alterOpt)); + CommandLineUtils.checkInvalidArgsSet(parser, options, usedOptions, invalidOptions, Optional.of(KAFKA_CONFIGS_CLI_SUPPORTS_ALTERING_TOPIC_CONFIGS_WITH_A_BOOTSTRAP_SERVER)); + CommandLineUtils.checkRequiredArgs(parser, options, partitionsOpt); + } + } + + private void checkInvalidArgs() { + // check invalid args + CommandLineUtils.checkInvalidArgs(parser, options, configOpt, invalidOptions(Arrays.asList(alterOpt, createOpt))); + CommandLineUtils.checkInvalidArgs(parser, options, deleteConfigOpt, + invalidOptions(new HashSet<>(Arrays.asList(bootstrapServerOpt)), Arrays.asList(alterOpt))); + CommandLineUtils.checkInvalidArgs(parser, options, partitionsOpt, invalidOptions(Arrays.asList(alterOpt, createOpt))); + CommandLineUtils.checkInvalidArgs(parser, options, replicationFactorOpt, invalidOptions(Arrays.asList(createOpt))); + CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, invalidOptions(Arrays.asList(alterOpt, createOpt))); + if (options.has(createOpt)) { + CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, partitionsOpt, replicationFactorOpt); + } + + + CommandLineUtils.checkInvalidArgs(parser, options, reportUnderReplicatedPartitionsOpt, + invalidOptions(Collections.singleton(topicsWithOverridesOpt), Arrays.asList(describeOpt, reportUnderReplicatedPartitionsOpt))); + CommandLineUtils.checkInvalidArgs(parser, options, reportUnderMinIsrPartitionsOpt, + invalidOptions(Collections.singleton(topicsWithOverridesOpt), Arrays.asList(describeOpt, reportUnderMinIsrPartitionsOpt))); + CommandLineUtils.checkInvalidArgs(parser, options, reportAtMinIsrPartitionsOpt, + invalidOptions(Collections.singleton(topicsWithOverridesOpt), Arrays.asList(describeOpt, reportAtMinIsrPartitionsOpt))); + CommandLineUtils.checkInvalidArgs(parser, options, reportUnavailablePartitionsOpt, + invalidOptions(Collections.singleton(topicsWithOverridesOpt), Arrays.asList(describeOpt, reportUnavailablePartitionsOpt))); + CommandLineUtils.checkInvalidArgs(parser, options, topicsWithOverridesOpt, + invalidOptions(new HashSet<>(allReplicationReportOpts), Arrays.asList(describeOpt))); + CommandLineUtils.checkInvalidArgs(parser, options, ifExistsOpt, + invalidOptions(Arrays.asList(alterOpt, deleteOpt, describeOpt))); + CommandLineUtils.checkInvalidArgs(parser, options, ifNotExistsOpt, invalidOptions(Arrays.asList(createOpt))); + CommandLineUtils.checkInvalidArgs(parser, options, excludeInternalTopicOpt, invalidOptions(Arrays.asList(listOpt, describeOpt))); + } + + private Set> invalidOptions(List> removeOptions) { + return invalidOptions(new HashSet<>(), removeOptions); + } + + private LinkedHashSet> invalidOptions(Set> addOptions, List> removeOptions) { + LinkedHashSet> finalOptions = new LinkedHashSet<>(allTopicLevelOpts); + finalOptions.removeAll(removeOptions); + finalOptions.addAll(addOptions); + return finalOptions; + } + } +} diff --git a/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java b/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java index 36e4e12d816..ac379bcb49e 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java +++ b/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java @@ -17,15 +17,35 @@ package org.apache.kafka.tools; import kafka.utils.TestInfoUtils; +import kafka.server.DynamicConfig; +import kafka.utils.TestUtils; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AlterConfigOp; +import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.storage.internals.log.LogConfig; import java.io.ByteArrayOutputStream; import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; public class ToolsTestUtils { /** @see TestInfoUtils#TestWithParameterizedQuorumName() */ public static final String TEST_WITH_PARAMETERIZED_QUORUM_NAME = "{displayName}.quorum={0}"; + private static int randomPort = 0; + public static String captureStandardOut(Runnable runnable) { return captureStandardStream(false, runnable); } @@ -55,6 +75,129 @@ public class ToolsTestUtils { } } + public static List createBrokerProperties(int numConfigs, String zkConnect, + Map rackInfo, + int numPartitions, + short defaultReplicationFactor) { + + return createBrokerProperties(numConfigs, zkConnect, rackInfo, 1, false, numPartitions, + defaultReplicationFactor, 0); + } + + /** + * Create a test config for the provided parameters. + * + * Note that if `interBrokerSecurityProtocol` is defined, the listener for the `SecurityProtocol` will be enabled. + */ + public static List createBrokerProperties(int numConfigs, String zkConnect, + Map rackInfo, int logDirCount, + boolean enableToken, int numPartitions, short defaultReplicationFactor, + int startingIdNumber) { + List result = new ArrayList<>(); + int endingIdNumber = startingIdNumber + numConfigs - 1; + for (int node = startingIdNumber; node <= endingIdNumber; node++) { + result.add(TestUtils.createBrokerConfig(node, zkConnect, true, true, randomPort, + scala.Option.empty(), + scala.Option.empty(), + scala.Option.empty(), + true, false, randomPort, false, randomPort, false, randomPort, + scala.Option.apply(rackInfo.get(node)), + logDirCount, enableToken, numPartitions, defaultReplicationFactor, false)); + } + return result; + } + + /** + * Set broker replication quotas and enable throttling for a set of partitions. This + * will override any previous replication quotas, but will leave the throttling status + * of other partitions unaffected. + */ + public static void setReplicationThrottleForPartitions(Admin admin, + List brokerIds, + Set partitions, + int throttleBytes) throws ExecutionException, InterruptedException { + throttleAllBrokersReplication(admin, brokerIds, throttleBytes); + assignThrottledPartitionReplicas(admin, partitions.stream().collect(Collectors.toMap(p -> p, p -> brokerIds))); + } + + /** + * Throttles all replication across the cluster. + * @param adminClient is the adminClient to use for making connection with the cluster + * @param brokerIds all broker ids in the cluster + * @param throttleBytes is the target throttle + */ + public static void throttleAllBrokersReplication(Admin adminClient, List brokerIds, int throttleBytes) throws ExecutionException, InterruptedException { + List throttleConfigs = new ArrayList<>(); + throttleConfigs.add(new AlterConfigOp(new ConfigEntry(DynamicConfig.Broker$.MODULE$.LeaderReplicationThrottledRateProp(), + Integer.toString(throttleBytes)), AlterConfigOp.OpType.SET)); + throttleConfigs.add(new AlterConfigOp(new ConfigEntry(DynamicConfig.Broker$.MODULE$.FollowerReplicationThrottledRateProp(), + Integer.toString(throttleBytes)), AlterConfigOp.OpType.SET)); + + Map> configs = new HashMap<>(); + for (int brokerId : brokerIds) { + ConfigResource configResource = new ConfigResource(ConfigResource.Type.BROKER, Integer.toString(brokerId)); + configs.put(configResource, throttleConfigs); + } + + adminClient.incrementalAlterConfigs(configs).all().get(); + } + + /** + * Remove a set of throttled partitions and reset the overall replication quota. + */ + public static void removeReplicationThrottleForPartitions(Admin admin, List brokerIds, Set partitions) throws ExecutionException, InterruptedException { + removePartitionReplicaThrottles(admin, partitions); + resetBrokersThrottle(admin, brokerIds); + } + + public static void assignThrottledPartitionReplicas(Admin adminClient, Map> allReplicasByPartition) throws InterruptedException, ExecutionException { + Map>>> configResourceToPartitionReplicas = + allReplicasByPartition.entrySet().stream() + .collect(Collectors.groupingBy( + topicPartitionListEntry -> new ConfigResource(ConfigResource.Type.TOPIC, topicPartitionListEntry.getKey().topic())) + ); + + Map> throttles = configResourceToPartitionReplicas.entrySet().stream() + .collect( + Collectors.toMap(Map.Entry::getKey, entry -> { + List alterConfigOps = new ArrayList<>(); + Map> replicaThrottle = + entry.getValue().stream().collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + alterConfigOps.add(new AlterConfigOp( + new ConfigEntry(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, formatReplicaThrottles(replicaThrottle)), + AlterConfigOp.OpType.SET)); + alterConfigOps.add(new AlterConfigOp( + new ConfigEntry(LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, formatReplicaThrottles(replicaThrottle)), + AlterConfigOp.OpType.SET)); + return alterConfigOps; + } + )); + adminClient.incrementalAlterConfigs(new HashMap<>(throttles)).all().get(); + } + + public static void resetBrokersThrottle(Admin adminClient, List brokerIds) throws ExecutionException, InterruptedException { + throttleAllBrokersReplication(adminClient, brokerIds, Integer.MAX_VALUE); + } + + public static void removePartitionReplicaThrottles(Admin adminClient, Set partitions) throws ExecutionException, InterruptedException { + Map> throttles = partitions.stream().collect(Collectors.toMap( + tp -> new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()), + tp -> Arrays.asList( + new AlterConfigOp(new ConfigEntry(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, ""), + AlterConfigOp.OpType.DELETE), + new AlterConfigOp(new ConfigEntry(LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, ""), + AlterConfigOp.OpType.DELETE)) + )); + + adminClient.incrementalAlterConfigs(throttles).all().get(); + } + + public static String formatReplicaThrottles(Map> moves) { + return moves.entrySet().stream() + .flatMap(entry -> entry.getValue().stream().map(replicaId -> entry.getKey().partition() + ":" + replicaId)) + .collect(Collectors.joining(",")); + } + public static class MockExitProcedure implements Exit.Procedure { private boolean hasExited = false; private int statusCode; diff --git a/tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java new file mode 100644 index 00000000000..b934e04012c --- /dev/null +++ b/tools/src/test/java/org/apache/kafka/tools/TopicCommandIntegrationTest.java @@ -0,0 +1,1065 @@ +/* + * 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.CommonClientConfigs; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AdminClientTestUtils; +import org.apache.kafka.clients.admin.Config; +import org.apache.kafka.clients.admin.ListPartitionReassignmentsResult; +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.ClusterAuthorizationException; +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; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; + +@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 int numPartitions = 1; + private TopicCommand.TopicService topicService; + private Admin adminClient; + private String bootstrapServer; + private String testTopicName; + private long 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 generateConfigs() { + Map 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 brokerConfigs = ToolsTestUtils + .createBrokerProperties(6, zkConnectOrNull(), rackInfo, numPartitions, defaultReplicationFactor); + + List 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, long 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(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(), TestUtils.randomString(10)); + } + @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)); + + assertTrue(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 partitions = adminClient + .describeTopics(Collections.singletonList(testTopicName)) + .allTopicNames() + .get() + .get(testTopicName) + .partitions(); + assertEquals(numPartitions, partitions.size()); + assertEquals(defaultReplicationFactor, (short) partitions.get(0).replicas().size()); + } + + @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 partitions = adminClient + .describeTopics(Collections.singletonList(testTopicName)) + .allTopicNames() + .get() + .get(testTopicName) + .partitions(); + assertEquals(2, partitions.size()); + assertEquals(defaultReplicationFactor, (short) partitions.get(0).replicas().size()); + } + + @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 partitions = adminClient + .describeTopics(Collections.singletonList(testTopicName)) + .allTopicNames() + .get() + .get(testTopicName) + .partitions(); + + assertEquals(numPartitions, partitions.size()); + assertEquals(2, (short) partitions.get(0).replicas().size()); + } + + @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 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 getPartitionReplicas(List 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(); + assertEquals(3, topicDescription.partitions().size()); + } + + @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 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 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"); + + 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> 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 topicNames = Collections.singleton(testTopicName); + Collection topicMetadatas = + JavaConverters.asJavaCollection(server.dataPlaneRequestProcessor().metadataCache() + .getTopicMetadata(JavaConverters.asScalaSetConverter(topicNames).asScala().toSet(), + ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT), + false, false) + ); + Optional 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 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 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 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 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 replicasOfFirstPartition = firstPartition.replicas().stream().map(Node::id).collect(Collectors.toList()); + List 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 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 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 Exception { + adminClient = spy(adminClient); + topicService.close(); // need to be closed before initializing a new one with the spy version of adminclient, otherwise will have the extra adminclient(s) not closed. + ListPartitionReassignmentsResult result = AdminClientTestUtils.listPartitionReassignmentsResult( + new ClusterAuthorizationException("Unauthorized")); + + doReturn(result).when(adminClient).listPartitionReassignments( + Collections.singleton(new TopicPartition(testTopicName, 0)) + ); + + topicService = new TopicCommand.TopicService(adminClient); + + adminClient.createTopics( + Collections.singletonList(new NewTopic(testTopicName, 1, (short) 1)) + ).all().get(); + waitForTopicCreated(testTopicName); + + String output = captureDescribeTopicStandardOut(buildTopicCommandOptionsWithBootstrap("--describe", "--topic", testTopicName)); + String[] rows = output.split("\n"); + assertEquals(2, rows.length, "Unexpected output: " + output); + assertTrue(rows[0].startsWith(String.format("Topic: %s", testTopicName)), "Unexpected output: " + rows[0]); + } + + @ParameterizedTest(name = ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME) + @ValueSource(strings = {"zk", "kraft"}) + public void testCreateWithTopicNameCollision(String quorum) throws ExecutionException, InterruptedException { + adminClient.createTopics( + Collections.singletonList(new NewTopic("foo_bar", 1, (short) 6))).all().get(); + waitForTopicCreated("foo_bar"); + + assertThrows(InvalidTopicException.class, + () -> topicService.createTopic(buildTopicCommandOptionsWithBootstrap("--create", "--topic", "foo.bar"))); + } + + private void checkReplicaDistribution(Map> assignment, + Map brokerRackMapping, + Integer numBrokers, + Integer numPartitions, + Integer replicationFactor, + Boolean verifyRackAware, + Boolean verifyLeaderDistribution, + Boolean verifyReplicasDistribution) { + // always verify that no broker will be assigned for more than one replica + assignment.entrySet().stream() + .forEach(entry -> assertEquals(new HashSet<>(entry.getValue()).size(), entry.getValue().size(), + "More than one replica is assigned to same broker for the same partition")); + + ReplicaDistributions distribution = getReplicaDistribution(assignment, brokerRackMapping); + + if (verifyRackAware) { + Map> partitionRackMap = distribution.partitionRacks; + + List partitionRackMapValueSize = partitionRackMap.values().stream() + .map(value -> (int) value.stream().distinct().count()) + .collect(Collectors.toList()); + + List expected = Collections.nCopies(numPartitions, replicationFactor); + assertEquals(expected, partitionRackMapValueSize, "More than one replica of the same partition is assigned to the same rack"); + } + + if (verifyLeaderDistribution) { + Map leaderCount = distribution.brokerLeaderCount; + int leaderCountPerBroker = numPartitions / numBrokers; + List expected = Collections.nCopies(numBrokers, leaderCountPerBroker); + assertEquals(expected, new ArrayList<>(leaderCount.values()), "Preferred leader count is not even for brokers"); + } + + if (verifyReplicasDistribution) { + Map replicasCount = distribution.brokerReplicasCount; + int numReplicasPerBroker = numPartitions * replicationFactor / numBrokers; + List expected = Collections.nCopies(numBrokers, numReplicasPerBroker); + assertEquals(expected, new ArrayList<>(replicasCount.values()), "Replica count is not even for broker"); + } + } + + private String captureDescribeTopicStandardOut(TopicCommand.TopicCommandOptions opts) { + Runnable runnable = () -> { + try { + topicService.describeTopic(opts); + } catch (Exception e) { + throw new RuntimeException(e); + } + }; + return ToolsTestUtils.captureStandardOut(runnable); + } + + private String captureListTopicStandardOut(TopicCommand.TopicCommandOptions opts) { + Runnable runnable = () -> { + try { + topicService.listTopics(opts); + } catch (Exception e) { + throw new RuntimeException(e); + } + }; + return ToolsTestUtils.captureStandardOut(runnable); + } + + private static ReplicaDistributions getReplicaDistribution(Map> assignment, Map brokerRackMapping) { + Map leaderCount = new HashMap<>(); + Map partitionCount = new HashMap<>(); + Map> partitionRackMap = new HashMap<>(); + + assignment.entrySet().stream().forEach(entry -> { + Integer partitionId = entry.getKey(); + List replicaList = entry.getValue(); + Integer leader = replicaList.get(0); + leaderCount.put(leader, leaderCount.getOrDefault(leader, 0) + 1); + replicaList.stream().forEach(brokerId -> { + partitionCount.put(brokerId, partitionCount.getOrDefault(brokerId, 0) + 1); + String rack; + if (brokerRackMapping.containsKey(brokerId)) { + rack = brokerRackMapping.get(brokerId); + List partitionRackValues = Stream.of(Collections.singletonList(rack), partitionRackMap.getOrDefault(partitionId, Collections.emptyList())) + .flatMap(List::stream) + .collect(Collectors.toList()); + partitionRackMap.put(partitionId, partitionRackValues); + } else { + System.err.println(String.format("No mapping found for %s in `brokerRackMapping`", brokerId)); + } + }); + }); + return new ReplicaDistributions(partitionRackMap, leaderCount, partitionCount); + } + + private static class ReplicaDistributions { + private final Map> partitionRacks; + private final Map brokerLeaderCount; + private final Map brokerReplicasCount; + + public ReplicaDistributions(Map> partitionRacks, + Map brokerLeaderCount, + Map brokerReplicasCount) { + this.partitionRacks = partitionRacks; + this.brokerLeaderCount = brokerLeaderCount; + this.brokerReplicasCount = brokerReplicasCount; + } + } + +} diff --git a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java new file mode 100644 index 00000000000..6831b0bae4c --- /dev/null +++ b/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 replicaIds = Arrays.asList(1, 2); + List 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> actualAssignment = TopicCommand.parseReplicaAssignment("5:4,3:2,1:0"); + Map> expectedAssignment = new HashMap<>(); + expectedAssignment.put(0, Arrays.asList(5, 4)); + expectedAssignment.put(1, Arrays.asList(3, 2)); + expectedAssignment.put(2, Arrays.asList(1, 0)); + assertEquals(expectedAssignment, actualAssignment); + } + + @Test + public void testCreateTopicDoesNotRetryThrottlingQuotaExceededException() { + Admin adminClient = mock(Admin.class); + TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient); + + CreateTopicsResult result = AdminClientTestUtils.createTopicsResult(topicName, Errors.THROTTLING_QUOTA_EXCEEDED.exception()); + when(adminClient.createTopics(any(), any())).thenReturn(result); + + assertThrows(ThrottlingQuotaExceededException.class, + () -> topicService.createTopic(new TopicCommand.TopicCommandOptions(new String[]{ + "--bootstrap-server", bootstrapServer, + "--create", "--topic", topicName + }))); + + NewTopic expectedNewTopic = new NewTopic(topicName, Optional.empty(), Optional.empty()) + .configs(Collections.emptyMap()); + + verify(adminClient, times(1)).createTopics( + eq(new HashSet<>(Arrays.asList(expectedNewTopic))), + argThat(exception -> !exception.shouldRetryOnQuotaViolation()) + ); + } + + @Test + public void testDeleteTopicDoesNotRetryThrottlingQuotaExceededException() { + Admin adminClient = mock(Admin.class); + TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient); + + ListTopicsResult listResult = AdminClientTestUtils.listTopicsResult(topicName); + when(adminClient.listTopics(any())).thenReturn(listResult); + + DeleteTopicsResult result = AdminClientTestUtils.deleteTopicsResult(topicName, Errors.THROTTLING_QUOTA_EXCEEDED.exception()); + when(adminClient.deleteTopics(anyCollection(), any())).thenReturn(result); + + ExecutionException exception = assertThrows(ExecutionException.class, + () -> topicService.deleteTopic(new TopicCommand.TopicCommandOptions(new String[]{ + "--bootstrap-server", bootstrapServer, + "--delete", "--topic", topicName + }))); + + assertTrue(exception.getCause() instanceof ThrottlingQuotaExceededException); + + verify(adminClient).deleteTopics( + argThat((Collection topics) -> topics.equals(Arrays.asList(topicName))), + argThat((DeleteTopicsOptions options) -> !options.shouldRetryOnQuotaViolation())); + } + + @Test + public void testCreatePartitionsDoesNotRetryThrottlingQuotaExceededException() { + Admin adminClient = mock(Admin.class); + TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient); + + ListTopicsResult listResult = AdminClientTestUtils.listTopicsResult(topicName); + when(adminClient.listTopics(any())).thenReturn(listResult); + + TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, new Node(0, "", 0), + Collections.emptyList(), Collections.emptyList()); + DescribeTopicsResult describeResult = AdminClientTestUtils.describeTopicsResult(topicName, + new TopicDescription(topicName, false, Collections.singletonList(topicPartitionInfo))); + when(adminClient.describeTopics(anyCollection())).thenReturn(describeResult); + + CreatePartitionsResult result = AdminClientTestUtils.createPartitionsResult(topicName, Errors.THROTTLING_QUOTA_EXCEEDED.exception()); + when(adminClient.createPartitions(any(), any())).thenReturn(result); + + Exception exception = assertThrows(ExecutionException.class, + () -> topicService.alterTopic(new TopicCommand.TopicCommandOptions(new String[]{ + "--alter", "--topic", topicName, "--partitions", "3", + "--bootstrap-server", bootstrapServer + }))); + assertTrue(exception.getCause() instanceof ThrottlingQuotaExceededException); + + verify(adminClient, times(1)).createPartitions( + argThat(newPartitions -> newPartitions.get(topicName).totalCount() == 3), + argThat(createPartitionOption -> !createPartitionOption.shouldRetryOnQuotaViolation())); + } + + public void assertInitializeInvalidOptionsExitCode(int expected, String[] options) { + Exit.setExitProcedure((exitCode, message) -> { + assertEquals(expected, exitCode); + throw new RuntimeException(); + }); + try { + assertThrows(RuntimeException.class, () -> new TopicCommand.TopicCommandOptions(options)); + } finally { + Exit.resetExitProcedure(); + } + } +}