From 76b1b50b644149e77ee1ec42d882e2cb80742bdf Mon Sep 17 00:00:00 2001 From: Nikolay Date: Wed, 1 Nov 2023 03:29:05 +0300 Subject: [PATCH] KAFKA-14595 Move ReassignPartitionsCommand to java (#13247) This PR contains changes required to move PartitionReassignmentState class to java code. Reviewers: Mickael Maison , Justine Olshan , Federico Valeri , Taras Ledkov Taras Ledkov , Kamal Chandraprakash, --- bin/kafka-reassign-partitions.sh | 2 +- bin/windows/kafka-reassign-partitions.bat | 2 +- .../admin/ReassignPartitionsCommand.scala | 1501 ----------------- .../kafka/ReplicationQuotasTestRig.scala | 340 ---- .../reassign/ReassignPartitionsCommand.java | 1499 ++++++++++++++++ .../ReassignPartitionsCommandOptions.java | 64 - .../apache/kafka/tools/reassign/Tuple2.java | 48 + .../ReassignPartitionsCommandArgsTest.java | 11 +- .../ReassignPartitionsIntegrationTest.java | 235 +-- .../reassign/ReassignPartitionsUnitTest.java | 461 +++-- 10 files changed, 1863 insertions(+), 2300 deletions(-) delete mode 100755 core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala delete mode 100644 core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala create mode 100644 tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java create mode 100644 tools/src/main/java/org/apache/kafka/tools/reassign/Tuple2.java diff --git a/bin/kafka-reassign-partitions.sh b/bin/kafka-reassign-partitions.sh index 4c7f1bc35e0..65310f90bfe 100755 --- a/bin/kafka-reassign-partitions.sh +++ b/bin/kafka-reassign-partitions.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.ReassignPartitionsCommand "$@" +exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.tools.reassign.ReassignPartitionsCommand "$@" diff --git a/bin/windows/kafka-reassign-partitions.bat b/bin/windows/kafka-reassign-partitions.bat index 62b710d36e0..58176714513 100644 --- a/bin/windows/kafka-reassign-partitions.bat +++ b/bin/windows/kafka-reassign-partitions.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.ReassignPartitionsCommand %* +"%~dp0kafka-run-class.bat" org.apache.kafka.tools.reassign.ReassignPartitionsCommand %* diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala deleted file mode 100755 index 205eb9ded93..00000000000 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ /dev/null @@ -1,1501 +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.Optional -import java.util.concurrent.ExecutionException -import kafka.server.DynamicConfig -import kafka.utils.{CoreUtils, Exit, Json, Logging} -import kafka.utils.Implicits._ -import kafka.utils.json.JsonValue -import org.apache.kafka.admin.{AdminUtils, BrokerMetadata} -import org.apache.kafka.clients.admin.AlterConfigOp.OpType -import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp, ConfigEntry, NewPartitionReassignment, PartitionReassignment, TopicDescription} -import org.apache.kafka.common.config.ConfigResource -import org.apache.kafka.common.errors.{ReplicaNotAvailableException, UnknownTopicOrPartitionException} -import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.common.{KafkaException, KafkaFuture, TopicPartition, TopicPartitionReplica} -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 scala.jdk.CollectionConverters._ -import scala.collection.{Map, Seq, mutable} -import scala.math.Ordered.orderingToOrdered - - -object ReassignPartitionsCommand extends Logging { - private[admin] val AnyLogDir = "any" - - val helpText = "This tool helps to move topic partitions between replicas." - - /** - * The earliest version of the partition reassignment JSON. We will default to this - * version if no other version number is given. - */ - private[admin] val EarliestVersion = 1 - - /** - * The earliest version of the JSON for each partition reassignment topic. We will - * default to this version if no other version number is given. - */ - private[admin] val EarliestTopicsJsonVersion = 1 - - // Throttles that are set at the level of an individual broker. - val brokerLevelLeaderThrottle = - DynamicConfig.Broker.LeaderReplicationThrottledRateProp - val brokerLevelFollowerThrottle = - DynamicConfig.Broker.FollowerReplicationThrottledRateProp - val brokerLevelLogDirThrottle = - DynamicConfig.Broker.ReplicaAlterLogDirsIoMaxBytesPerSecondProp - val brokerLevelThrottles = Seq( - brokerLevelLeaderThrottle, - brokerLevelFollowerThrottle, - brokerLevelLogDirThrottle - ) - - // Throttles that are set at the level of an individual topic. - val topicLevelLeaderThrottle = - LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG - val topicLevelFollowerThrottle = - LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG - val topicLevelThrottles = Seq( - topicLevelLeaderThrottle, - topicLevelFollowerThrottle - ) - - private[admin] val cannotExecuteBecauseOfExistingMessage = "Cannot execute because " + - "there is an existing partition assignment. Use --additional to override this and " + - "create a new partition assignment in addition to the existing one. The --additional " + - "flag can also be used to change the throttle by resubmitting the current reassignment." - - private[admin] val youMustRunVerifyPeriodicallyMessage = "Warning: You must run " + - "--verify periodically, until the reassignment completes, to ensure the throttle " + - "is removed." - - /** - * A map from topic names to partition movements. - */ - type MoveMap = mutable.Map[String, mutable.Map[Int, PartitionMove]] - - /** - * A partition movement. The source and destination brokers may overlap. - * - * @param sources The source brokers. - * @param destinations The destination brokers. - */ - sealed case class PartitionMove(sources: mutable.Set[Int], - destinations: mutable.Set[Int]) { } - - /** - * The state of a partition reassignment. The current replicas and target replicas - * may overlap. - * - * @param currentReplicas The current replicas. - * @param targetReplicas The target replicas. - * @param done True if the reassignment is done. - */ - sealed case class PartitionReassignmentState(currentReplicas: Seq[Int], - targetReplicas: Seq[Int], - done: Boolean) {} - - /** - * The state of a replica log directory movement. - */ - sealed trait LogDirMoveState { - /** - * True if the move is done without errors. - */ - def done: Boolean - } - - /** - * A replica log directory move state where the source log directory is missing. - * - * @param targetLogDir The log directory that we wanted the replica to move to. - */ - sealed case class MissingReplicaMoveState(targetLogDir: String) - extends LogDirMoveState { - override def done = false - } - - /** - * A replica log directory move state where the source replica is missing. - * - * @param targetLogDir The log directory that we wanted the replica to move to. - */ - sealed case class MissingLogDirMoveState(targetLogDir: String) - extends LogDirMoveState { - override def done = false - } - - /** - * A replica log directory move state where the move is in progress. - * - * @param currentLogDir The current log directory. - * @param futureLogDir The log directory that the replica is moving to. - * @param targetLogDir The log directory that we wanted the replica to move to. - */ - sealed case class ActiveMoveState(currentLogDir: String, - targetLogDir: String, - futureLogDir: String) - extends LogDirMoveState { - override def done = false - } - - /** - * A replica log directory move state where there is no move in progress, but we did not - * reach the target log directory. - * - * @param currentLogDir The current log directory. - * @param targetLogDir The log directory that we wanted the replica to move to. - */ - sealed case class CancelledMoveState(currentLogDir: String, - targetLogDir: String) - extends LogDirMoveState { - override def done = true - } - - /** - * The completed replica log directory move state. - * - * @param targetLogDir The log directory that we wanted the replica to move to. - */ - sealed case class CompletedMoveState(targetLogDir: String) - extends LogDirMoveState { - override def done = true - } - - /** - * An exception thrown to indicate that the command has failed, but we don't want to - * print a stack trace. - * - * @param message The message to print out before exiting. A stack trace will not - * be printed. - */ - class TerseReassignmentFailureException(message: String) extends KafkaException(message) { - } - - def main(args: Array[String]): Unit = { - val opts = validateAndParseArgs(args) - var failed = true - var adminClient: Admin = null - - try { - val props = if (opts.options.has(opts.commandConfigOpt)) - Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt)) - else - new util.Properties() - props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt)) - props.putIfAbsent(AdminClientConfig.CLIENT_ID_CONFIG, "reassign-partitions-tool") - adminClient = Admin.create(props) - handleAction(adminClient, opts) - failed = false - } catch { - case e: TerseReassignmentFailureException => - println(e.getMessage) - case e: Throwable => - println("Error: " + e.getMessage) - println(Utils.stackTrace(e)) - } finally { - // It's good to do this after printing any error stack trace. - if (adminClient != null) { - adminClient.close() - } - } - // If the command failed, exit with a non-zero exit code. - if (failed) { - Exit.exit(1) - } - } - - private def handleAction(adminClient: Admin, - opts: ReassignPartitionsCommandOptions): Unit = { - if (opts.options.has(opts.verifyOpt)) { - verifyAssignment(adminClient, - Utils.readFileAsString(opts.options.valueOf(opts.reassignmentJsonFileOpt)), - opts.options.has(opts.preserveThrottlesOpt)) - } else if (opts.options.has(opts.generateOpt)) { - generateAssignment(adminClient, - Utils.readFileAsString(opts.options.valueOf(opts.topicsToMoveJsonFileOpt)), - opts.options.valueOf(opts.brokerListOpt), - !opts.options.has(opts.disableRackAware)) - } else if (opts.options.has(opts.executeOpt)) { - executeAssignment(adminClient, - opts.options.has(opts.additionalOpt), - Utils.readFileAsString(opts.options.valueOf(opts.reassignmentJsonFileOpt)), - opts.options.valueOf(opts.interBrokerThrottleOpt), - opts.options.valueOf(opts.replicaAlterLogDirsThrottleOpt), - opts.options.valueOf(opts.timeoutOpt)) - } else if (opts.options.has(opts.cancelOpt)) { - cancelAssignment(adminClient, - Utils.readFileAsString(opts.options.valueOf(opts.reassignmentJsonFileOpt)), - opts.options.has(opts.preserveThrottlesOpt), - opts.options.valueOf(opts.timeoutOpt)) - } else if (opts.options.has(opts.listOpt)) { - listReassignments(adminClient) - } else { - throw new RuntimeException("Unsupported action.") - } - } - - /** - * A result returned from verifyAssignment. - * - * @param partStates A map from partitions to reassignment states. - * @param partsOngoing True if there are any ongoing partition reassignments. - * @param moveStates A map from log directories to movement states. - * @param movesOngoing True if there are any ongoing moves that we know about. - */ - case class VerifyAssignmentResult(partStates: Map[TopicPartition, PartitionReassignmentState], - partsOngoing: Boolean = false, - moveStates: Map[TopicPartitionReplica, LogDirMoveState] = Map.empty, - movesOngoing: Boolean = false) - - /** - * The entry point for the --verify command. - * - * @param adminClient The AdminClient to use. - * @param jsonString The JSON string to use for the topics and partitions to verify. - * @param preserveThrottles True if we should avoid changing topic or broker throttles. - * - * @return A result that is useful for testing. - */ - def verifyAssignment(adminClient: Admin, jsonString: String, preserveThrottles: Boolean) - : VerifyAssignmentResult = { - val (targetParts, targetLogDirs) = parsePartitionReassignmentData(jsonString) - val (partStates, partsOngoing) = verifyPartitionAssignments(adminClient, targetParts) - val (moveStates, movesOngoing) = verifyReplicaMoves(adminClient, targetLogDirs) - if (!partsOngoing && !movesOngoing && !preserveThrottles) { - // If the partition assignments and replica assignments are done, clear any throttles - // that were set. We have to clear all throttles, because we don't have enough - // information to know all of the source brokers that might have been involved in the - // previous reassignments. - clearAllThrottles(adminClient, targetParts) - } - VerifyAssignmentResult(partStates, partsOngoing, moveStates, movesOngoing) - } - - /** - * Verify the partition reassignments specified by the user. - * - * @param adminClient The AdminClient to use. - * @param targets The partition reassignments specified by the user. - * - * @return A tuple of the partition reassignment states, and a - * boolean which is true if there are no ongoing - * reassignments (including reassignments not described - * in the JSON file.) - */ - def verifyPartitionAssignments(adminClient: Admin, - targets: Seq[(TopicPartition, Seq[Int])]) - : (Map[TopicPartition, PartitionReassignmentState], Boolean) = { - val (partStates, partsOngoing) = findPartitionReassignmentStates(adminClient, targets) - println(partitionReassignmentStatesToString(partStates)) - (partStates, partsOngoing) - } - - def compareTopicPartitions(a: TopicPartition, b: TopicPartition): Boolean = { - (a.topic(), a.partition()) < (b.topic(), b.partition()) - } - - def compareTopicPartitionReplicas(a: TopicPartitionReplica, b: TopicPartitionReplica): Boolean = { - (a.brokerId(), a.topic(), a.partition()) < (b.brokerId(), b.topic(), b.partition()) - } - - /** - * Convert partition reassignment states to a human-readable string. - * - * @param states A map from topic partitions to states. - * @return A string summarizing the partition reassignment states. - */ - def partitionReassignmentStatesToString(states: Map[TopicPartition, PartitionReassignmentState]) - : String = { - val bld = new mutable.ArrayBuffer[String]() - bld.append("Status of partition reassignment:") - states.keySet.toBuffer.sortWith(compareTopicPartitions).foreach { topicPartition => - val state = states(topicPartition) - if (state.done) { - if (state.currentReplicas.equals(state.targetReplicas)) { - bld.append("Reassignment of partition %s is completed.". - format(topicPartition.toString)) - } else { - bld.append(s"There is no active reassignment of partition ${topicPartition}, " + - s"but replica set is ${state.currentReplicas.mkString(",")} rather than " + - s"${state.targetReplicas.mkString(",")}.") - } - } else { - bld.append("Reassignment of partition %s is still in progress.".format(topicPartition)) - } - } - bld.mkString(System.lineSeparator()) - } - - /** - * Find the state of the specified partition reassignments. - * - * @param adminClient The Admin client to use. - * @param targetReassignments The reassignments we want to learn about. - * - * @return A tuple containing the reassignment states for each topic - * partition, plus whether there are any ongoing reassignments. - */ - def findPartitionReassignmentStates(adminClient: Admin, - targetReassignments: Seq[(TopicPartition, Seq[Int])]) - : (Map[TopicPartition, PartitionReassignmentState], Boolean) = { - val currentReassignments = adminClient. - listPartitionReassignments.reassignments.get().asScala - val (foundReassignments, notFoundReassignments) = targetReassignments.partition { - case (part, _) => currentReassignments.contains(part) - } - val foundResults = foundReassignments.map { - case (part, targetReplicas) => (part, - PartitionReassignmentState( - currentReassignments(part).replicas. - asScala.map(i => i.asInstanceOf[Int]), - targetReplicas, - false)) - } - val topicNamesToLookUp = new mutable.HashSet[String]() - notFoundReassignments.foreach { case (part, _) => - if (!currentReassignments.contains(part)) - topicNamesToLookUp.add(part.topic) - } - val topicDescriptions = adminClient. - describeTopics(topicNamesToLookUp.asJava).topicNameValues().asScala - val notFoundResults = notFoundReassignments.map { - case (part, targetReplicas) => - currentReassignments.get(part) match { - case Some(reassignment) => (part, - PartitionReassignmentState( - reassignment.replicas.asScala.map(_.asInstanceOf[Int]), - targetReplicas, - false)) - case None => - (part, topicDescriptionFutureToState(part.partition, - topicDescriptions(part.topic), targetReplicas)) - } - } - val allResults = foundResults ++ notFoundResults - (allResults.toMap, currentReassignments.nonEmpty) - } - - private def topicDescriptionFutureToState(partition: Int, - future: KafkaFuture[TopicDescription], - targetReplicas: Seq[Int]): PartitionReassignmentState = { - try { - val topicDescription = future.get() - if (topicDescription.partitions().size() < partition) { - throw new ExecutionException("Too few partitions found", new UnknownTopicOrPartitionException()) - } - PartitionReassignmentState( - topicDescription.partitions.get(partition).replicas.asScala.map(_.id), - targetReplicas, - true) - } catch { - case t: ExecutionException if t.getCause.isInstanceOf[UnknownTopicOrPartitionException] => - PartitionReassignmentState(Seq(), targetReplicas, true) - } - } - - /** - * Verify the replica reassignments specified by the user. - * - * @param adminClient The AdminClient to use. - * @param targetReassignments The replica reassignments specified by the user. - * - * @return A tuple of the replica states, and a boolean which is true - * if there are any ongoing replica moves. - * - * Note: Unlike in verifyPartitionAssignments, we will - * return false here even if there are unrelated ongoing - * reassignments. (We don't have an efficient API that - * returns all ongoing replica reassignments.) - */ - def verifyReplicaMoves(adminClient: Admin, - targetReassignments: Map[TopicPartitionReplica, String]) - : (Map[TopicPartitionReplica, LogDirMoveState], Boolean) = { - val moveStates = findLogDirMoveStates(adminClient, targetReassignments) - println(replicaMoveStatesToString(moveStates)) - (moveStates, !moveStates.values.forall(_.done)) - } - - /** - * Find the state of the specified partition reassignments. - * - * @param adminClient The AdminClient to use. - * @param targetMoves The movements we want to learn about. The map is keyed - * by TopicPartitionReplica, and its values are target log - * directories. - * - * @return The states for each replica movement. - */ - def findLogDirMoveStates(adminClient: Admin, - targetMoves: Map[TopicPartitionReplica, String]) - : Map[TopicPartitionReplica, LogDirMoveState] = { - val replicaLogDirInfos = adminClient.describeReplicaLogDirs( - targetMoves.keySet.asJava).all().get().asScala - targetMoves.map { case (replica, targetLogDir) => - val moveState = replicaLogDirInfos.get(replica) match { - case None => MissingReplicaMoveState(targetLogDir) - case Some(info) => if (info.getCurrentReplicaLogDir == null) { - MissingLogDirMoveState(targetLogDir) - } else if (info.getFutureReplicaLogDir == null) { - if (info.getCurrentReplicaLogDir.equals(targetLogDir)) { - CompletedMoveState(targetLogDir) - } else { - CancelledMoveState(info.getCurrentReplicaLogDir, targetLogDir) - } - } else { - ActiveMoveState(info.getCurrentReplicaLogDir(), - targetLogDir, - info.getFutureReplicaLogDir) - } - } - (replica, moveState) - } - } - - /** - * Convert replica move states to a human-readable string. - * - * @param states A map from topic partition replicas to states. - * @return A tuple of a summary string, and a boolean describing - * whether there are any active replica moves. - */ - def replicaMoveStatesToString(states: Map[TopicPartitionReplica, LogDirMoveState]) - : String = { - val bld = new mutable.ArrayBuffer[String] - states.keySet.toBuffer.sortWith(compareTopicPartitionReplicas).foreach { replica => - val state = states(replica) - state match { - case MissingLogDirMoveState(_) => - bld.append(s"Partition ${replica.topic}-${replica.partition} is not found " + - s"in any live log dir on broker ${replica.brokerId}. There is likely an " + - s"offline log directory on the broker.") - case MissingReplicaMoveState(_) => - bld.append(s"Partition ${replica.topic}-${replica.partition} cannot be found " + - s"in any live log directory on broker ${replica.brokerId}.") - case ActiveMoveState(_, targetLogDir, futureLogDir) => - if (targetLogDir.equals(futureLogDir)) { - bld.append(s"Reassignment of replica $replica is still in progress.") - } else { - bld.append(s"Partition ${replica.topic}-${replica.partition} on broker " + - s"${replica.brokerId} is being moved to log dir $futureLogDir " + - s"instead of $targetLogDir.") - } - case CancelledMoveState(currentLogDir, targetLogDir) => - bld.append(s"Partition ${replica.topic}-${replica.partition} on broker " + - s"${replica.brokerId} is not being moved from log dir $currentLogDir to " + - s"$targetLogDir.") - case CompletedMoveState(_) => - bld.append(s"Reassignment of replica $replica completed successfully.") - } - } - bld.mkString(System.lineSeparator()) - } - - /** - * Clear all topic-level and broker-level throttles. - * - * @param adminClient The AdminClient to use. - * @param targetParts The target partitions loaded from the JSON file. - */ - def clearAllThrottles(adminClient: Admin, - targetParts: Seq[(TopicPartition, Seq[Int])]): Unit = { - val activeBrokers = adminClient.describeCluster().nodes().get().asScala.map(_.id()).toSet - val brokers = activeBrokers ++ targetParts.flatMap(_._2).toSet - println("Clearing broker-level throttles on broker%s %s".format( - if (brokers.size == 1) "" else "s", brokers.mkString(","))) - clearBrokerLevelThrottles(adminClient, brokers) - - val topics = targetParts.map(_._1.topic()).toSet - println("Clearing topic-level throttles on topic%s %s".format( - if (topics.size == 1) "" else "s", topics.mkString(","))) - clearTopicLevelThrottles(adminClient, topics) - } - - /** - * Clear all throttles which have been set at the broker level. - * - * @param adminClient The AdminClient to use. - * @param brokers The brokers to clear the throttles for. - */ - def clearBrokerLevelThrottles(adminClient: Admin, brokers: Set[Int]): Unit = { - val configOps = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]() - brokers.foreach { brokerId => - configOps.put( - new ConfigResource(ConfigResource.Type.BROKER, brokerId.toString), - brokerLevelThrottles.map(throttle => new AlterConfigOp( - new ConfigEntry(throttle, null), OpType.DELETE)).asJava) - } - adminClient.incrementalAlterConfigs(configOps).all().get() - } - - /** - * Clear the reassignment throttles for the specified topics. - * - * @param adminClient The AdminClient to use. - * @param topics The topics to clear the throttles for. - */ - def clearTopicLevelThrottles(adminClient: Admin, topics: Set[String]): Unit = { - val configOps = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]() - topics.foreach { - topicName => configOps.put( - new ConfigResource(ConfigResource.Type.TOPIC, topicName), - topicLevelThrottles.map(throttle => new AlterConfigOp(new ConfigEntry(throttle, null), - OpType.DELETE)).asJava) - } - adminClient.incrementalAlterConfigs(configOps).all().get() - } - - /** - * The entry point for the --generate command. - * - * @param adminClient The AdminClient to use. - * @param reassignmentJson The JSON string to use for the topics to reassign. - * @param brokerListString The comma-separated string of broker IDs to use. - * @param enableRackAwareness True if rack-awareness should be enabled. - * - * @return A tuple containing the proposed assignment and the - * current assignment. - */ - def generateAssignment(adminClient: Admin, - reassignmentJson: String, - brokerListString: String, - enableRackAwareness: Boolean) - : (Map[TopicPartition, Seq[Int]], Map[TopicPartition, Seq[Int]]) = { - val (brokersToReassign, topicsToReassign) = - parseGenerateAssignmentArgs(reassignmentJson, brokerListString) - val currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign) - val brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness) - val proposedAssignments = calculateAssignment(currentAssignments, brokerMetadatas) - println("Current partition replica assignment\n%s\n". - format(formatAsReassignmentJson(currentAssignments, Map.empty))) - println("Proposed partition reassignment configuration\n%s". - format(formatAsReassignmentJson(proposedAssignments, Map.empty))) - (proposedAssignments, currentAssignments) - } - - /** - * Calculate the new partition assignments to suggest in --generate. - * - * @param currentAssignment The current partition assignments. - * @param brokerMetadatas The rack information for each broker. - * - * @return A map from partitions to the proposed assignments for each. - */ - def calculateAssignment(currentAssignment: Map[TopicPartition, Seq[Int]], - brokerMetadatas: Seq[BrokerMetadata]) - : Map[TopicPartition, Seq[Int]] = { - val groupedByTopic = currentAssignment.groupBy { case (tp, _) => tp.topic } - val proposedAssignments = mutable.Map[TopicPartition, Seq[Int]]() - groupedByTopic.forKeyValue { (topic, assignment) => - val (_, replicas) = assignment.head - val assignedReplicas = CoreUtils.replicaToBrokerAssignmentAsScala(AdminUtils. - assignReplicasToBrokers(brokerMetadatas.asJavaCollection, assignment.size, replicas.size)) - proposedAssignments ++= assignedReplicas.map { case (partition, replicas) => - new TopicPartition(topic, partition) -> replicas - } - } - proposedAssignments - } - - private def describeTopics(adminClient: Admin, - topics: Set[String]) - : Map[String, TopicDescription] = { - adminClient.describeTopics(topics.asJava).topicNameValues().asScala.map { case (topicName, topicDescriptionFuture) => - try topicName -> topicDescriptionFuture.get - catch { - case t: ExecutionException if t.getCause.isInstanceOf[UnknownTopicOrPartitionException] => - throw new ExecutionException( - new UnknownTopicOrPartitionException(s"Topic $topicName not found.")) - } - } - } - - /** - * Get the current replica assignments for some topics. - * - * @param adminClient The AdminClient to use. - * @param topics The topics to get information about. - * @return A map from partitions to broker assignments. - * If any topic can't be found, an exception will be thrown. - */ - def getReplicaAssignmentForTopics(adminClient: Admin, - topics: Seq[String]) - : Map[TopicPartition, Seq[Int]] = { - describeTopics(adminClient, topics.toSet).flatMap { - case (topicName, topicDescription) => topicDescription.partitions.asScala.map { info => - (new TopicPartition(topicName, info.partition), info.replicas.asScala.map(_.id)) - } - } - } - - /** - * Get the current replica assignments for some partitions. - * - * @param adminClient The AdminClient to use. - * @param partitions The partitions to get information about. - * @return A map from partitions to broker assignments. - * If any topic can't be found, an exception will be thrown. - */ - def getReplicaAssignmentForPartitions(adminClient: Admin, - partitions: Set[TopicPartition]) - : Map[TopicPartition, Seq[Int]] = { - describeTopics(adminClient, partitions.map(_.topic)).flatMap { - case (topicName, topicDescription) => topicDescription.partitions.asScala.flatMap { info => - val tp = new TopicPartition(topicName, info.partition) - if (partitions.contains(tp)) { - Some(tp, info.replicas.asScala.map(_.id)) - } else { - None - } - } - } - } - - /** - * Find the rack information for some brokers. - * - * @param adminClient The AdminClient object. - * @param brokers The brokers to gather metadata about. - * @param enableRackAwareness True if we should return rack information, and throw an - * exception if it is inconsistent. - * - * @return The metadata for each broker that was found. - * Brokers that were not found will be omitted. - */ - def getBrokerMetadata(adminClient: Admin, - brokers: Seq[Int], - enableRackAwareness: Boolean): Seq[BrokerMetadata] = { - val brokerSet = brokers.toSet - val results = adminClient.describeCluster().nodes.get().asScala. - filter(node => brokerSet.contains(node.id)). - map { - node => if (enableRackAwareness && node.rack != null) { - new BrokerMetadata(node.id, Optional.of(node.rack)) - } else { - new BrokerMetadata(node.id, Optional.empty()) - } - }.toSeq - val numRackless = results.count(!_.rack.isPresent) - if (enableRackAwareness && numRackless != 0 && numRackless != results.size) { - throw new AdminOperationException("Not all brokers have rack information. Add " + - "--disable-rack-aware in command line to make replica assignment without rack " + - "information.") - } - results - } - - /** - * Parse and validate data gathered from the command-line for --generate - * In particular, we parse the JSON and validate that duplicate brokers and - * topics don't appear. - * - * @param reassignmentJson The JSON passed to --generate . - * @param brokerList A list of brokers passed to --generate. - * - * @return A tuple of brokers to reassign, topics to reassign - */ - def parseGenerateAssignmentArgs(reassignmentJson: String, - brokerList: String): (Seq[Int], Seq[String]) = { - val brokerListToReassign = brokerList.split(',').map(_.toInt) - val duplicateReassignments = CoreUtils.duplicates(brokerListToReassign) - if (duplicateReassignments.nonEmpty) - throw new AdminCommandFailedException("Broker list contains duplicate entries: %s". - format(duplicateReassignments.mkString(","))) - val topicsToReassign = parseTopicsData(reassignmentJson) - val duplicateTopicsToReassign = CoreUtils.duplicates(topicsToReassign) - if (duplicateTopicsToReassign.nonEmpty) - throw new AdminCommandFailedException("List of topics to reassign contains duplicate entries: %s". - format(duplicateTopicsToReassign.mkString(","))) - (brokerListToReassign, topicsToReassign) - } - - /** - * The entry point for the --execute and --execute-additional commands. - * - * @param adminClient The AdminClient to use. - * @param additional Whether --additional was passed. - * @param reassignmentJson The JSON string to use for the topics to reassign. - * @param interBrokerThrottle The inter-broker throttle to use, or a negative - * number to skip using a throttle. - * @param logDirThrottle The replica log directory throttle to use, or a - * negative number to skip using a throttle. - * @param timeoutMs The maximum time in ms to wait for log directory - * replica assignment to begin. - * @param time The Time object to use. - */ - def executeAssignment(adminClient: Admin, - additional: Boolean, - reassignmentJson: String, - interBrokerThrottle: Long = -1L, - logDirThrottle: Long = -1L, - timeoutMs: Long = 10000L, - time: Time = Time.SYSTEM): Unit = { - val (proposedParts, proposedReplicas) = parseExecuteAssignmentArgs(reassignmentJson) - val currentReassignments = adminClient. - listPartitionReassignments().reassignments().get().asScala - // If there is an existing assignment, check for --additional before proceeding. - // This helps avoid surprising users. - if (!additional && currentReassignments.nonEmpty) { - throw new TerseReassignmentFailureException(cannotExecuteBecauseOfExistingMessage) - } - verifyBrokerIds(adminClient, proposedParts.values.flatten.toSet) - val currentParts = getReplicaAssignmentForPartitions(adminClient, proposedParts.keySet.toSet) - println(currentPartitionReplicaAssignmentToString(proposedParts, currentParts)) - - if (interBrokerThrottle >= 0 || logDirThrottle >= 0) { - println(youMustRunVerifyPeriodicallyMessage) - - if (interBrokerThrottle >= 0) { - val moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts) - modifyReassignmentThrottle(adminClient, moveMap, interBrokerThrottle) - } - - if (logDirThrottle >= 0) { - val movingBrokers = calculateMovingBrokers(proposedReplicas.keySet.toSet) - modifyLogDirThrottle(adminClient, movingBrokers, logDirThrottle) - } - } - - // Execute the partition reassignments. - val errors = alterPartitionReassignments(adminClient, proposedParts) - if (errors.nonEmpty) { - throw new TerseReassignmentFailureException( - "Error reassigning partition(s):%n%s".format( - errors.keySet.toBuffer.sortWith(compareTopicPartitions).map { part => - s"$part: ${errors(part).getMessage}" - }.mkString(System.lineSeparator()))) - } - println("Successfully started partition reassignment%s for %s".format( - if (proposedParts.size == 1) "" else "s", - proposedParts.keySet.toBuffer.sortWith(compareTopicPartitions).mkString(","))) - if (proposedReplicas.nonEmpty) { - executeMoves(adminClient, proposedReplicas, timeoutMs, time) - } - } - - /** - * Execute some partition log directory movements. - * - * @param adminClient The AdminClient to use. - * @param proposedReplicas A map from TopicPartitionReplicas to the - * directories to move them to. - * @param timeoutMs The maximum time in ms to wait for log directory - * replica assignment to begin. - * @param time The Time object to use. - */ - def executeMoves(adminClient: Admin, - proposedReplicas: Map[TopicPartitionReplica, String], - timeoutMs: Long, - time: Time): Unit = { - val startTimeMs = time.milliseconds() - val pendingReplicas = new mutable.HashMap[TopicPartitionReplica, String]() - pendingReplicas ++= proposedReplicas - var done = false - do { - val completed = alterReplicaLogDirs(adminClient, pendingReplicas) - if (completed.nonEmpty) { - println("Successfully started log directory move%s for: %s".format( - if (completed.size == 1) "" else "s", - completed.toBuffer.sortWith(compareTopicPartitionReplicas).mkString(","))) - } - pendingReplicas --= completed - if (pendingReplicas.isEmpty) { - done = true - } else if (time.milliseconds() >= startTimeMs + timeoutMs) { - throw new TerseReassignmentFailureException( - "Timed out before log directory move%s could be started for: %s".format( - if (pendingReplicas.size == 1) "" else "s", - pendingReplicas.keySet.toBuffer.sortWith(compareTopicPartitionReplicas). - mkString(","))) - } else { - // If a replica has been moved to a new host and we also specified a particular - // log directory, we will have to keep retrying the alterReplicaLogDirs - // call. It can't take effect until the replica is moved to that host. - time.sleep(100) - } - } while (!done) - } - - /** - * Entry point for the --list command. - * - * @param adminClient The AdminClient to use. - */ - def listReassignments(adminClient: Admin): Unit = { - println(curReassignmentsToString(adminClient)) - } - - /** - * Convert the current partition reassignments to text. - * - * @param adminClient The AdminClient to use. - * @return A string describing the current partition reassignments. - */ - def curReassignmentsToString(adminClient: Admin): String = { - val currentReassignments = adminClient. - listPartitionReassignments().reassignments().get().asScala - val text = currentReassignments.keySet.toBuffer.sortWith(compareTopicPartitions).map { part => - val reassignment = currentReassignments(part) - val replicas = reassignment.replicas.asScala - val addingReplicas = reassignment.addingReplicas.asScala - val removingReplicas = reassignment.removingReplicas.asScala - "%s: replicas: %s.%s%s".format(part, replicas.mkString(","), - if (addingReplicas.isEmpty) "" else - " adding: %s.".format(addingReplicas.mkString(",")), - if (removingReplicas.isEmpty) "" else - " removing: %s.".format(removingReplicas.mkString(","))) - }.mkString(System.lineSeparator()) - if (text.isEmpty) { - "No partition reassignments found." - } else { - "Current partition reassignments:%n%s".format(text) - } - } - - /** - * Verify that all the brokers in an assignment exist. - * - * @param adminClient The AdminClient to use. - * @param brokers The broker IDs to verify. - */ - def verifyBrokerIds(adminClient: Admin, brokers: Set[Int]): Unit = { - val allNodeIds = adminClient.describeCluster().nodes().get().asScala.map(_.id).toSet - brokers.find(!allNodeIds.contains(_)).map { - id => throw new AdminCommandFailedException(s"Unknown broker id ${id}") - } - } - - /** - * Return the string which we want to print to describe the current partition assignment. - * - * @param proposedParts The proposed partition assignment. - * @param currentParts The current partition assignment. - * - * @return The string to print. We will only print information about - * partitions that appear in the proposed partition assignment. - */ - def currentPartitionReplicaAssignmentToString(proposedParts: Map[TopicPartition, Seq[Int]], - currentParts: Map[TopicPartition, Seq[Int]]): String = { - "Current partition replica assignment%n%n%s%n%nSave this to use as the %s". - format(formatAsReassignmentJson(currentParts.filter { case (k, _) => proposedParts.contains(k) }.toMap, Map.empty), - "--reassignment-json-file option during rollback") - } - - /** - * Execute the given partition reassignments. - * - * @param adminClient The admin client object to use. - * @param reassignments A map from topic names to target replica assignments. - * @return A map from partition objects to error strings. - */ - def alterPartitionReassignments(adminClient: Admin, - reassignments: Map[TopicPartition, Seq[Int]]): Map[TopicPartition, Throwable] = { - val results = adminClient.alterPartitionReassignments(reassignments.map { case (part, replicas) => - (part, Optional.of(new NewPartitionReassignment(replicas.map(Integer.valueOf).asJava))) - }.asJava).values().asScala - results.flatMap { - case (part, future) => { - try { - future.get() - None - } catch { - case t: ExecutionException => Some(part, t.getCause()) - } - } - } - } - - /** - * Cancel the given partition reassignments. - * - * @param adminClient The admin client object to use. - * @param reassignments The partition reassignments to cancel. - * @return A map from partition objects to error strings. - */ - def cancelPartitionReassignments(adminClient: Admin, - reassignments: Set[TopicPartition]) - : Map[TopicPartition, Throwable] = { - val results = adminClient.alterPartitionReassignments(reassignments.map { - (_, Optional.empty[NewPartitionReassignment]()) - }.toMap.asJava).values().asScala - results.flatMap { case (part, future) => - try { - future.get() - None - } catch { - case t: ExecutionException => Some(part, t.getCause()) - } - } - } - - /** - * Compute the in progress partition move from the current reassignments. - * @param currentReassignments All replicas, adding replicas and removing replicas of target partitions - */ - private def calculateCurrentMoveMap(currentReassignments: Map[TopicPartition, PartitionReassignment]): MoveMap = { - val moveMap = new mutable.HashMap[String, mutable.Map[Int, PartitionMove]]() - // Add the current reassignments to the move map. - currentReassignments.forKeyValue { (part, reassignment) => - val allReplicas = reassignment.replicas().asScala.map(Int.unbox) - val addingReplicas = reassignment.addingReplicas.asScala.map(Int.unbox) - - // The addingReplicas is included in the replicas during reassignment - val sources = mutable.Set[Int]() ++ allReplicas.diff(addingReplicas) - val destinations = mutable.Set[Int]() ++ addingReplicas - - val partMoves = moveMap.getOrElseUpdate(part.topic, new mutable.HashMap[Int, PartitionMove]) - partMoves.put(part.partition, PartitionMove(sources, destinations)) - } - moveMap - } - - /** - * Calculate the global map of all partitions that are moving. - * - * @param currentReassignments The currently active reassignments. - * @param proposedParts The proposed location of the partitions (destinations replicas only). - * @param currentParts The current location of the partitions that we are - * proposing to move. - * @return A map from topic name to partition map. - * The partition map is keyed on partition index and contains - * the movements for that partition. - */ - def calculateProposedMoveMap(currentReassignments: Map[TopicPartition, PartitionReassignment], - proposedParts: Map[TopicPartition, Seq[Int]], - currentParts: Map[TopicPartition, Seq[Int]]): MoveMap = { - val moveMap = calculateCurrentMoveMap(currentReassignments) - - proposedParts.forKeyValue { (part, replicas) => - val partMoves = moveMap.getOrElseUpdate(part.topic, new mutable.HashMap[Int, PartitionMove]) - - // If there is a reassignment in progress, use the sources from moveMap, otherwise - // use the sources from currentParts - val sources = mutable.Set[Int]() ++ (partMoves.get(part.partition) match { - case Some(move) => move.sources.toSeq - case None => currentParts.getOrElse(part, - throw new RuntimeException(s"Trying to reassign a topic partition $part with 0 replicas")) - }) - val destinations = mutable.Set[Int]() ++ replicas.diff(sources.toSeq) - - partMoves.put(part.partition, - PartitionMove(sources, destinations)) - } - moveMap - } - - /** - * Calculate the leader throttle configurations to use. - * - * @param moveMap The movements. - * @return A map from topic names to leader throttle configurations. - */ - def calculateLeaderThrottles(moveMap: MoveMap): Map[String, String] = { - moveMap.map { - case (topicName, partMoveMap) => { - val components = new mutable.TreeSet[String] - partMoveMap.forKeyValue { (partId, move) => - move.sources.foreach(source => components.add("%d:%d".format(partId, source))) - } - (topicName, components.mkString(",")) - } - } - } - - /** - * Calculate the follower throttle configurations to use. - * - * @param moveMap The movements. - * @return A map from topic names to follower throttle configurations. - */ - def calculateFollowerThrottles(moveMap: MoveMap): Map[String, String] = { - moveMap.map { - case (topicName, partMoveMap) => { - val components = new mutable.TreeSet[String] - partMoveMap.forKeyValue { (partId, move) => - move.destinations.foreach(destination => - if (!move.sources.contains(destination)) { - components.add("%d:%d".format(partId, destination)) - }) - } - (topicName, components.mkString(",")) - } - } - } - - /** - * Calculate all the brokers which are involved in the given partition reassignments. - * - * @param moveMap The partition movements. - * @return A set of all the brokers involved. - */ - def calculateReassigningBrokers(moveMap: MoveMap): Set[Int] = { - val reassigningBrokers = new mutable.TreeSet[Int] - moveMap.values.foreach { - _.values.foreach { - partMove => - partMove.sources.foreach(reassigningBrokers.add) - partMove.destinations.foreach(reassigningBrokers.add) - } - } - reassigningBrokers.toSet - } - - /** - * Calculate all the brokers which are involved in the given directory movements. - * - * @param replicaMoves The replica movements. - * @return A set of all the brokers involved. - */ - def calculateMovingBrokers(replicaMoves: Set[TopicPartitionReplica]): Set[Int] = { - replicaMoves.map(_.brokerId()) - } - - /** - * Modify the topic configurations that control inter-broker throttling. - * - * @param adminClient The adminClient object to use. - * @param leaderThrottles A map from topic names to leader throttle configurations. - * @param followerThrottles A map from topic names to follower throttle configurations. - */ - def modifyTopicThrottles(adminClient: Admin, - leaderThrottles: Map[String, String], - followerThrottles: Map[String, String]): Unit = { - val configs = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]() - val topicNames = leaderThrottles.keySet ++ followerThrottles.keySet - topicNames.foreach { topicName => - val ops = new util.ArrayList[AlterConfigOp] - leaderThrottles.get(topicName).foreach { value => - ops.add(new AlterConfigOp(new ConfigEntry(topicLevelLeaderThrottle, value), OpType.SET)) - } - followerThrottles.get(topicName).foreach { value => - ops.add(new AlterConfigOp(new ConfigEntry(topicLevelFollowerThrottle, value), OpType.SET)) - } - if (!ops.isEmpty) { - configs.put(new ConfigResource(ConfigResource.Type.TOPIC, topicName), ops) - } - } - adminClient.incrementalAlterConfigs(configs).all().get() - } - - private def modifyReassignmentThrottle(admin: Admin, moveMap: MoveMap, interBrokerThrottle: Long): Unit = { - val leaderThrottles = calculateLeaderThrottles(moveMap) - val followerThrottles = calculateFollowerThrottles(moveMap) - modifyTopicThrottles(admin, leaderThrottles, followerThrottles) - - val reassigningBrokers = calculateReassigningBrokers(moveMap) - modifyInterBrokerThrottle(admin, reassigningBrokers, interBrokerThrottle) - } - - /** - * Modify the leader/follower replication throttles for a set of brokers. - * - * @param adminClient The Admin instance to use - * @param reassigningBrokers The set of brokers involved in the reassignment - * @param interBrokerThrottle The new throttle (ignored if less than 0) - */ - def modifyInterBrokerThrottle(adminClient: Admin, - reassigningBrokers: Set[Int], - interBrokerThrottle: Long): Unit = { - if (interBrokerThrottle >= 0) { - val configs = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]() - reassigningBrokers.foreach { brokerId => - val ops = new util.ArrayList[AlterConfigOp] - ops.add(new AlterConfigOp(new ConfigEntry(brokerLevelLeaderThrottle, - interBrokerThrottle.toString), OpType.SET)) - ops.add(new AlterConfigOp(new ConfigEntry(brokerLevelFollowerThrottle, - interBrokerThrottle.toString), OpType.SET)) - configs.put(new ConfigResource(ConfigResource.Type.BROKER, brokerId.toString), ops) - } - adminClient.incrementalAlterConfigs(configs).all().get() - println(s"The inter-broker throttle limit was set to $interBrokerThrottle B/s") - } - } - - /** - * Modify the log dir reassignment throttle for a set of brokers. - * - * @param admin The Admin instance to use - * @param movingBrokers The set of broker to alter the throttle of - * @param logDirThrottle The new throttle (ignored if less than 0) - */ - def modifyLogDirThrottle(admin: Admin, - movingBrokers: Set[Int], - logDirThrottle: Long): Unit = { - if (logDirThrottle >= 0) { - val configs = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]() - movingBrokers.foreach { brokerId => - val ops = new util.ArrayList[AlterConfigOp] - ops.add(new AlterConfigOp(new ConfigEntry(brokerLevelLogDirThrottle, logDirThrottle.toString), OpType.SET)) - configs.put(new ConfigResource(ConfigResource.Type.BROKER, brokerId.toString), ops) - } - admin.incrementalAlterConfigs(configs).all().get() - println(s"The replica-alter-dir throttle limit was set to $logDirThrottle B/s") - } - } - - /** - * Parse the reassignment JSON string passed to the --execute command. - * - * @param reassignmentJson The JSON string. - * @return A tuple of the partitions to be reassigned and the replicas - * to be reassigned. - */ - def parseExecuteAssignmentArgs(reassignmentJson: String) - : (Map[TopicPartition, Seq[Int]], Map[TopicPartitionReplica, String]) = { - val (partitionsToBeReassigned, replicaAssignment) = parsePartitionReassignmentData(reassignmentJson) - if (partitionsToBeReassigned.isEmpty) - throw new AdminCommandFailedException("Partition reassignment list cannot be empty") - if (partitionsToBeReassigned.exists(_._2.isEmpty)) { - throw new AdminCommandFailedException("Partition replica list cannot be empty") - } - val duplicateReassignedPartitions = CoreUtils.duplicates(partitionsToBeReassigned.map { case (tp, _) => tp }) - if (duplicateReassignedPartitions.nonEmpty) - throw new AdminCommandFailedException("Partition reassignment contains duplicate topic partitions: %s".format(duplicateReassignedPartitions.mkString(","))) - val duplicateEntries = partitionsToBeReassigned - .map { case (tp, replicas) => (tp, CoreUtils.duplicates(replicas))} - .filter { case (_, duplicatedReplicas) => duplicatedReplicas.nonEmpty } - if (duplicateEntries.nonEmpty) { - val duplicatesMsg = duplicateEntries - .map { case (tp, duplicateReplicas) => "%s contains multiple entries for %s".format(tp, duplicateReplicas.mkString(",")) } - .mkString(". ") - throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: %s".format(duplicatesMsg)) - } - (partitionsToBeReassigned.toMap, replicaAssignment) - } - - /** - * The entry point for the --cancel command. - * - * @param adminClient The AdminClient to use. - * @param jsonString The JSON string to use for the topics and partitions to cancel. - * @param preserveThrottles True if we should avoid changing topic or broker throttles. - * @param timeoutMs The maximum time in ms to wait for log directory - * replica assignment to begin. - * @param time The Time object to use. - * - * @return A tuple of the partition reassignments that were cancelled, - * and the replica movements that were cancelled. - */ - def cancelAssignment(adminClient: Admin, - jsonString: String, - preserveThrottles: Boolean, - timeoutMs: Long = 10000L, - time: Time = Time.SYSTEM) - : (Set[TopicPartition], Set[TopicPartitionReplica]) = { - val (targetParts, targetReplicas) = parsePartitionReassignmentData(jsonString) - val targetPartsSet = targetParts.map(_._1).toSet - val curReassigningParts = adminClient.listPartitionReassignments(targetPartsSet.asJava). - reassignments().get().asScala.flatMap { - case (part, reassignment) => if (!reassignment.addingReplicas().isEmpty || - !reassignment.removingReplicas().isEmpty) { - Some(part) - } else { - None - } - }.toSet - if (curReassigningParts.nonEmpty) { - val errors = cancelPartitionReassignments(adminClient, curReassigningParts) - if (errors.nonEmpty) { - throw new TerseReassignmentFailureException( - "Error cancelling partition reassignment%s for:%n%s".format( - if (errors.size == 1) "" else "s", - errors.keySet.toBuffer.sortWith(compareTopicPartitions).map { - part => s"${part}: ${errors(part).getMessage}" - }.mkString(System.lineSeparator()))) - } - println("Successfully cancelled partition reassignment%s for: %s".format( - if (curReassigningParts.size == 1) "" else "s", - s"${curReassigningParts.toBuffer.sortWith(compareTopicPartitions).mkString(",")}")) - } else { - println("None of the specified partition reassignments are active.") - } - val curMovingParts = findLogDirMoveStates(adminClient, targetReplicas).flatMap { - case (part, moveState) => moveState match { - case state: ActiveMoveState => Some(part, state.currentLogDir) - case _ => None - } - }.toMap - if (curMovingParts.isEmpty) { - println("None of the specified partition moves are active.") - } else { - executeMoves(adminClient, curMovingParts, timeoutMs, time) - } - if (!preserveThrottles) { - clearAllThrottles(adminClient, targetParts) - } - (curReassigningParts, curMovingParts.keySet) - } - - def formatAsReassignmentJson(partitionsToBeReassigned: Map[TopicPartition, Seq[Int]], - replicaLogDirAssignment: Map[TopicPartitionReplica, String]): String = { - Json.encodeAsString(Map( - "version" -> 1, - "partitions" -> partitionsToBeReassigned.keySet.toBuffer.sortWith(compareTopicPartitions).map { - tp => - val replicas = partitionsToBeReassigned(tp) - Map( - "topic" -> tp.topic, - "partition" -> tp.partition, - "replicas" -> replicas.asJava, - "log_dirs" -> replicas.map(r => replicaLogDirAssignment.getOrElse(new TopicPartitionReplica(tp.topic, tp.partition, r), AnyLogDir)).asJava - ).asJava - }.asJava - ).asJava) - } - - def parseTopicsData(jsonData: String): Seq[String] = { - Json.parseFull(jsonData) match { - case Some(js) => - val version = js.asJsonObject.get("version") match { - case Some(jsonValue) => jsonValue.to[Int] - case None => EarliestTopicsJsonVersion - } - parseTopicsData(version, js) - case None => throw new AdminOperationException("The input string is not a valid JSON") - } - } - - def parseTopicsData(version: Int, js: JsonValue): Seq[String] = { - version match { - case 1 => - for { - partitionsSeq <- js.asJsonObject.get("topics").toSeq - p <- partitionsSeq.asJsonArray.iterator - } yield p.asJsonObject("topic").to[String] - case _ => throw new AdminOperationException(s"Not supported version field value $version") - } - } - - def parsePartitionReassignmentData(jsonData: String): (Seq[(TopicPartition, Seq[Int])], Map[TopicPartitionReplica, String]) = { - Json.tryParseFull(jsonData) match { - case Right(js) => - val version = js.asJsonObject.get("version") match { - case Some(jsonValue) => jsonValue.to[Int] - case None => EarliestVersion - } - parsePartitionReassignmentData(version, js) - case Left(f) => - throw new AdminOperationException(f) - } - } - - // Parses without deduplicating keys so the data can be checked before allowing reassignment to proceed - def parsePartitionReassignmentData(version:Int, jsonData: JsonValue): (Seq[(TopicPartition, Seq[Int])], Map[TopicPartitionReplica, String]) = { - version match { - case 1 => - val partitionAssignment = mutable.ListBuffer.empty[(TopicPartition, Seq[Int])] - val replicaAssignment = mutable.Map.empty[TopicPartitionReplica, String] - for { - partitionsSeq <- jsonData.asJsonObject.get("partitions").toSeq - p <- partitionsSeq.asJsonArray.iterator - } { - val partitionFields = p.asJsonObject - val topic = partitionFields("topic").to[String] - val partition = partitionFields("partition").to[Int] - val newReplicas = partitionFields("replicas").to[Seq[Int]] - val newLogDirs = partitionFields.get("log_dirs") match { - case Some(jsonValue) => jsonValue.to[Seq[String]] - case None => newReplicas.map(_ => AnyLogDir) - } - if (newReplicas.size != newLogDirs.size) - throw new AdminCommandFailedException(s"Size of replicas list $newReplicas is different from " + - s"size of log dirs list $newLogDirs for partition ${new TopicPartition(topic, partition)}") - partitionAssignment += (new TopicPartition(topic, partition) -> newReplicas) - replicaAssignment ++= newReplicas.zip(newLogDirs).map { case (replica, logDir) => - new TopicPartitionReplica(topic, partition, replica) -> logDir - }.filter(_._2 != AnyLogDir) - } - (partitionAssignment, replicaAssignment) - case _ => throw new AdminOperationException(s"Not supported version field value $version") - } - } - - def validateAndParseArgs(args: Array[String]): ReassignPartitionsCommandOptions = { - val opts = new ReassignPartitionsCommandOptions(args) - - CommandLineUtils.maybePrintHelpOrVersion(opts, helpText) - - // Determine which action we should perform. - val validActions = Seq(opts.generateOpt, opts.executeOpt, opts.verifyOpt, - opts.cancelOpt, opts.listOpt) - val allActions = validActions.filter(opts.options.has _) - if (allActions.size != 1) { - CommandLineUtils.printUsageAndExit(opts.parser, "Command must include exactly one action: %s".format( - validActions.map("--" + _.options().get(0)).mkString(", "))) - } - val action = allActions.head - - if (!opts.options.has(opts.bootstrapServerOpt)) - CommandLineUtils.printUsageAndExit(opts.parser, "Please specify --bootstrap-server") - - // Make sure that we have all the required arguments for our action. - val requiredArgs = Map( - opts.verifyOpt -> collection.immutable.Seq( - opts.reassignmentJsonFileOpt - ), - opts.generateOpt -> collection.immutable.Seq( - opts.topicsToMoveJsonFileOpt, - opts.brokerListOpt - ), - opts.executeOpt -> collection.immutable.Seq( - opts.reassignmentJsonFileOpt - ), - opts.cancelOpt -> collection.immutable.Seq( - opts.reassignmentJsonFileOpt - ), - opts.listOpt -> collection.immutable.Seq.empty - ) - CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, requiredArgs(action): _*) - - // Make sure that we didn't specify any arguments that are incompatible with our chosen action. - val permittedArgs = Map( - opts.verifyOpt -> Seq( - opts.bootstrapServerOpt, - opts.commandConfigOpt, - opts.preserveThrottlesOpt, - ), - opts.generateOpt -> Seq( - opts.bootstrapServerOpt, - opts.brokerListOpt, - opts.commandConfigOpt, - opts.disableRackAware, - ), - opts.executeOpt -> Seq( - opts.additionalOpt, - opts.bootstrapServerOpt, - opts.commandConfigOpt, - opts.interBrokerThrottleOpt, - opts.replicaAlterLogDirsThrottleOpt, - opts.timeoutOpt, - ), - opts.cancelOpt -> Seq( - opts.bootstrapServerOpt, - opts.commandConfigOpt, - opts.preserveThrottlesOpt, - opts.timeoutOpt - ), - opts.listOpt -> Seq( - opts.bootstrapServerOpt, - opts.commandConfigOpt - ) - ) - opts.options.specs.forEach(opt => { - if (!opt.equals(action) && - !requiredArgs(action).contains(opt) && - !permittedArgs(action).contains(opt)) { - CommandLineUtils.printUsageAndExit(opts.parser, - """Option "%s" can't be used with action "%s"""".format(opt, action)) - } - }) - - opts - } - - def alterReplicaLogDirs(adminClient: Admin, - assignment: Map[TopicPartitionReplica, String]) - : Set[TopicPartitionReplica] = { - adminClient.alterReplicaLogDirs(assignment.asJava).values().asScala.flatMap { - case (replica, future) => { - try { - future.get() - Some(replica) - } catch { - case t: ExecutionException => - t.getCause match { - // Ignore ReplicaNotAvailableException. It is OK if the replica is not - // available at this moment. - case _: ReplicaNotAvailableException => None - case e: Throwable => - throw new AdminCommandFailedException(s"Failed to alter dir for $replica", e) - } - } - } - }.toSet - } - - sealed class ReassignPartitionsCommandOptions(args: Array[String]) extends CommandDefaultOptions(args) { - // Actions - val verifyOpt = parser.accepts("verify", "Verify if the reassignment completed as specified by the " + - "--reassignment-json-file option. If there is a throttle engaged for the replicas specified, and the rebalance has completed, the throttle will be removed") - val generateOpt = parser.accepts("generate", "Generate a candidate partition reassignment configuration." + - " Note that this only generates a candidate assignment, it does not execute it.") - val executeOpt = parser.accepts("execute", "Kick off the reassignment as specified by the --reassignment-json-file option.") - val cancelOpt = parser.accepts("cancel", "Cancel an active reassignment.") - val listOpt = parser.accepts("list", "List all active partition reassignments.") - - // Arguments - val bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: the server(s) to use for bootstrapping.") - .withRequiredArg - .describedAs("Server(s) to use for bootstrapping") - .ofType(classOf[String]) - - val commandConfigOpt = parser.accepts("command-config", "Property file containing configs to be passed to Admin Client.") - .withRequiredArg - .describedAs("Admin client property file") - .ofType(classOf[String]) - - val reassignmentJsonFileOpt = parser.accepts("reassignment-json-file", "The JSON file with the partition reassignment configuration" + - "The format to use is - \n" + - "{\"partitions\":\n\t[{\"topic\": \"foo\",\n\t \"partition\": 1,\n\t \"replicas\": [1,2,3],\n\t \"log_dirs\": [\"dir1\",\"dir2\",\"dir3\"] }],\n\"version\":1\n}\n" + - "Note that \"log_dirs\" is optional. When it is specified, its length must equal the length of the replicas list. The value in this list " + - "can be either \"any\" or the absolution path of the log directory on the broker. If absolute log directory path is specified, the replica will be moved to the specified log directory on the broker.") - .withRequiredArg - .describedAs("manual assignment json file path") - .ofType(classOf[String]) - val topicsToMoveJsonFileOpt = parser.accepts("topics-to-move-json-file", "Generate a reassignment configuration to move the partitions" + - " of the specified topics to the list of brokers specified by the --broker-list option. The format to use is - \n" + - "{\"topics\":\n\t[{\"topic\": \"foo\"},{\"topic\": \"foo1\"}],\n\"version\":1\n}") - .withRequiredArg - .describedAs("topics to reassign json file path") - .ofType(classOf[String]) - val brokerListOpt = parser.accepts("broker-list", "The list of brokers to which the partitions need to be reassigned" + - " in the form \"0,1,2\". This is required if --topics-to-move-json-file is used to generate reassignment configuration") - .withRequiredArg - .describedAs("brokerlist") - .ofType(classOf[String]) - val disableRackAware = parser.accepts("disable-rack-aware", "Disable rack aware replica assignment") - val interBrokerThrottleOpt = parser.accepts("throttle", "The movement of partitions between brokers will be throttled to this value (bytes/sec). " + - "This option can be included with --execute when a reassignment is started, and it can be altered by resubmitting the current reassignment " + - "along with the --additional flag. The throttle rate should be at least 1 KB/s.") - .withRequiredArg() - .describedAs("throttle") - .ofType(classOf[Long]) - .defaultsTo(-1) - val replicaAlterLogDirsThrottleOpt = parser.accepts("replica-alter-log-dirs-throttle", - "The movement of replicas between log directories on the same broker will be throttled to this value (bytes/sec). " + - "This option can be included with --execute when a reassignment is started, and it can be altered by resubmitting the current reassignment " + - "along with the --additional flag. The throttle rate should be at least 1 KB/s.") - .withRequiredArg() - .describedAs("replicaAlterLogDirsThrottle") - .ofType(classOf[Long]) - .defaultsTo(-1) - val timeoutOpt = parser.accepts("timeout", "The maximum time in ms to wait for log directory replica assignment to begin.") - .withRequiredArg() - .describedAs("timeout") - .ofType(classOf[Long]) - .defaultsTo(10000) - val additionalOpt = parser.accepts("additional", "Execute this reassignment in addition to any " + - "other ongoing ones. This option can also be used to change the throttle of an ongoing reassignment.") - val preserveThrottlesOpt = parser.accepts("preserve-throttles", "Do not modify broker or topic throttles.") - options = parser.parse(args : _*) - } -} diff --git a/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala b/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala deleted file mode 100644 index 584955de691..00000000000 --- a/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala +++ /dev/null @@ -1,340 +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 - -import java.io.{File, PrintWriter} -import java.nio.charset.StandardCharsets -import java.nio.file.{Files, StandardOpenOption} - -import javax.imageio.ImageIO -import kafka.admin.ReassignPartitionsCommand -import kafka.server.{KafkaConfig, KafkaServer, QuorumTestHarness, QuotaType} -import kafka.utils.TestUtils._ -import kafka.utils.{EmptyTestInfo, Exit, Logging, TestUtils} -import kafka.zk.ReassignPartitionsZNode -import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} -import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.utils.Utils -import org.jfree.chart.plot.PlotOrientation -import org.jfree.chart.{ChartFactory, ChartFrame, JFreeChart} -import org.jfree.data.xy.{XYSeries, XYSeriesCollection} - -import scala.jdk.CollectionConverters._ -import scala.collection.{Map, Seq, mutable} - -/** - * Test rig for measuring throttling performance. Configure the parameters for a set of experiments, then execute them - * and view the html output file, with charts, that are produced. You can also render the charts to the screen if - * you wish. - * - * Currently you'll need about 40GB of disk space to run these experiments (largest data written x2). Tune the msgSize - * & #partitions and throttle to adjust this. - */ -object ReplicationQuotasTestRig { - new File("Experiments").mkdir() - private val dir = "Experiments/Run" + System.currentTimeMillis().toString.substring(8) - new File(dir).mkdir() - val k = 1000 * 1000 - - - def main(args: Array[String]): Unit = { - val displayChartsOnScreen = if (args.length > 0 && args(0) == "show-gui") true else false - val journal = new Journal() - - val experiments = Seq( - //1GB total data written, will take 210s - new ExperimentDef("Experiment1", brokers = 5, partitions = 20, throttle = 1 * k, msgsPerPartition = 500, msgSize = 100 * 1000), - //5GB total data written, will take 110s - new ExperimentDef("Experiment2", brokers = 5, partitions = 50, throttle = 10 * k, msgsPerPartition = 1000, msgSize = 100 * 1000), - //5GB total data written, will take 110s - new ExperimentDef("Experiment3", brokers = 50, partitions = 50, throttle = 2 * k, msgsPerPartition = 1000, msgSize = 100 * 1000), - //10GB total data written, will take 110s - new ExperimentDef("Experiment4", brokers = 25, partitions = 100, throttle = 4 * k, msgsPerPartition = 1000, msgSize = 100 * 1000), - //10GB total data written, will take 80s - new ExperimentDef("Experiment5", brokers = 5, partitions = 50, throttle = 50 * k, msgsPerPartition = 4000, msgSize = 100 * 1000) - ) - experiments.foreach(run(_, journal, displayChartsOnScreen)) - - if (!displayChartsOnScreen) - Exit.exit(0) - } - - def run(config: ExperimentDef, journal: Journal, displayChartsOnScreen: Boolean): Unit = { - val experiment = new Experiment() - try { - experiment.setUp(new EmptyTestInfo()) - experiment.run(config, journal, displayChartsOnScreen) - journal.footer() - } - catch { - case e: Exception => e.printStackTrace() - } - finally { - experiment.tearDown() - } - } - - case class ExperimentDef(name: String, brokers: Int, partitions: Int, throttle: Long, msgsPerPartition: Int, msgSize: Int) { - val targetBytesPerBrokerMB: Long = msgsPerPartition.toLong * msgSize.toLong * partitions.toLong / brokers.toLong / 1000000 - } - - class Experiment extends QuorumTestHarness with Logging { - val topicName = "my-topic" - var experimentName = "unset" - val partitionId = 0 - var servers: Seq[KafkaServer] = _ - val leaderRates = mutable.Map[Int, Array[Double]]() - val followerRates = mutable.Map[Int, Array[Double]]() - var adminClient: Admin = _ - - def startBrokers(brokerIds: Seq[Int]): Unit = { - println("Starting Brokers") - servers = brokerIds.map(i => createBrokerConfig(i, zkConnect)) - .map(c => createServer(KafkaConfig.fromProps(c))) - - TestUtils.waitUntilBrokerMetadataIsPropagated(servers) - val brokerList = TestUtils.plaintextBootstrapServers(servers) - adminClient = Admin.create(Map[String, Object]( - AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG -> brokerList - ).asJava) - } - - override def tearDown(): Unit = { - Utils.closeQuietly(adminClient, "adminClient") - TestUtils.shutdownServers(servers) - super.tearDown() - } - - def run(config: ExperimentDef, journal: Journal, displayChartsOnScreen: Boolean): Unit = { - experimentName = config.name - val brokers = (100 to 100 + config.brokers) - var count = 0 - val shift = Math.round(config.brokers / 2f) - - def nextReplicaRoundRobin(): Int = { - count = count + 1 - 100 + (count + shift) % config.brokers - } - val replicas = (0 to config.partitions).map(partition => partition -> Seq(nextReplicaRoundRobin())).toMap - - startBrokers(brokers) - createTopic(zkClient, topicName, replicas, servers) - - println("Writing Data") - val producer = TestUtils.createProducer(TestUtils.plaintextBootstrapServers(servers), acks = 0) - (0 until config.msgsPerPartition).foreach { x => - (0 until config.partitions).foreach { partition => - producer.send(new ProducerRecord(topicName, partition, null, new Array[Byte](config.msgSize))) - } - } - - println("Generating Reassignment") - val (newAssignment, _) = ReassignPartitionsCommand.generateAssignment(adminClient, - json(topicName), brokers.mkString(","), true) - - println("Starting Reassignment") - val start = System.currentTimeMillis() - ReassignPartitionsCommand.executeAssignment(adminClient, false, - new String(ReassignPartitionsZNode.encode(newAssignment), StandardCharsets.UTF_8), - config.throttle) - - //Await completion - waitForReassignmentToComplete() - println(s"Reassignment took ${(System.currentTimeMillis() - start)/1000}s") - - validateAllOffsetsMatch(config) - - journal.appendToJournal(config) - renderChart(leaderRates, "Leader", journal, displayChartsOnScreen) - renderChart(followerRates, "Follower", journal, displayChartsOnScreen) - logOutput(config, replicas, newAssignment) - - println("Output can be found here: " + journal.path()) - } - - def validateAllOffsetsMatch(config: ExperimentDef): Unit = { - //Validate that offsets are correct in all brokers - for (broker <- servers) { - (0 until config.partitions).foreach { partitionId => - val offset = broker.getLogManager.getLog(new TopicPartition(topicName, partitionId)).map(_.logEndOffset).getOrElse(-1L) - if (offset >= 0 && offset != config.msgsPerPartition) { - throw new RuntimeException(s"Run failed as offsets did not match for partition $partitionId on broker ${broker.config.brokerId}. Expected ${config.msgsPerPartition} but was $offset.") - } - } - } - } - - def logOutput(config: ExperimentDef, replicas: Map[Int, Seq[Int]], newAssignment: Map[TopicPartition, Seq[Int]]): Unit = { - val actual = zkClient.getPartitionAssignmentForTopics(Set(topicName))(topicName) - - //Long stats - println("The replicas are " + replicas.toSeq.sortBy(_._1).map("\n" + _)) - println("This is the current replica assignment:\n" + actual.map { case (k, v) => k -> v.replicas }) - println("proposed assignment is: \n" + newAssignment) - println("This is the assignment we ended up with" + actual.map { case (k, v) => k -> v.replicas }) - - //Test Stats - println(s"numBrokers: ${config.brokers}") - println(s"numPartitions: ${config.partitions}") - println(s"throttle: ${config.throttle}") - println(s"numMessagesPerPartition: ${config.msgsPerPartition}") - println(s"msgSize: ${config.msgSize}") - println(s"We will write ${config.targetBytesPerBrokerMB}MB of data per broker") - println(s"Worst case duration is ${config.targetBytesPerBrokerMB * 1000 * 1000/ config.throttle}") - } - - def waitForReassignmentToComplete(): Unit = { - waitUntilTrue(() => { - printRateMetrics() - adminClient.listPartitionReassignments().reassignments().get().isEmpty - }, s"Partition reassignments didn't complete.", 60 * 60 * 1000, pause = 1000L) - } - - def renderChart(data: mutable.Map[Int, Array[Double]], name: String, journal: Journal, displayChartsOnScreen: Boolean): Unit = { - val dataset = addDataToChart(data) - val chart = createChart(name, dataset) - - writeToFile(name, journal, chart) - maybeDisplayOnScreen(displayChartsOnScreen, chart) - println(s"Chart generated for $name") - } - - def maybeDisplayOnScreen(displayChartsOnScreen: Boolean, chart: JFreeChart): Unit = { - if (displayChartsOnScreen) { - val frame = new ChartFrame(experimentName, chart) - frame.pack() - frame.setVisible(true) - } - } - - def writeToFile(name: String, journal: Journal, chart: JFreeChart): Unit = { - val file = new File(dir, experimentName + "-" + name + ".png") - ImageIO.write(chart.createBufferedImage(1000, 700), "png", file) - journal.appendChart(file.getAbsolutePath, name.eq("Leader")) - } - - def createChart(name: String, dataset: XYSeriesCollection): JFreeChart = { - val chart: JFreeChart = ChartFactory.createXYLineChart( - experimentName + " - " + name + " Throttling Performance", - "Time (s)", - "Throttle Throughput (B/s)", - dataset - , PlotOrientation.VERTICAL, false, true, false - ) - chart - } - - def addDataToChart(data: mutable.Map[Int, Array[Double]]): XYSeriesCollection = { - val dataset = new XYSeriesCollection - data.foreach { case (broker, values) => - val series = new XYSeries("Broker:" + broker) - var x = 0 - values.foreach { value => - series.add(x, value) - x += 1 - } - dataset.addSeries(series) - } - dataset - } - - def record(rates: mutable.Map[Int, Array[Double]], brokerId: Int, currentRate: Double) = { - var leaderRatesBroker: Array[Double] = rates.getOrElse(brokerId, Array[Double]()) - leaderRatesBroker = leaderRatesBroker ++ Array(currentRate) - rates.put(brokerId, leaderRatesBroker) - } - - def printRateMetrics(): Unit = { - for (broker <- servers) { - val leaderRate: Double = measuredRate(broker, QuotaType.LeaderReplication) - if (broker.config.brokerId == 100) - info("waiting... Leader rate on 101 is " + leaderRate) - record(leaderRates, broker.config.brokerId, leaderRate) - if (leaderRate > 0) - trace("Leader Rate on " + broker.config.brokerId + " is " + leaderRate) - - val followerRate: Double = measuredRate(broker, QuotaType.FollowerReplication) - record(followerRates, broker.config.brokerId, followerRate) - if (followerRate > 0) - trace("Follower Rate on " + broker.config.brokerId + " is " + followerRate) - } - } - - private def measuredRate(broker: KafkaServer, repType: QuotaType): Double = { - val metricName = broker.metrics.metricName("byte-rate", repType.toString) - if (broker.metrics.metrics.asScala.contains(metricName)) - broker.metrics.metrics.asScala(metricName).metricValue.asInstanceOf[Double] - else -1 - } - - def json(topic: String*): String = { - val topicStr = topic.map { - t => "{\"topic\": \"" + t + "\"}" - }.mkString(",") - s"""{"topics": [$topicStr],"version":1}""" - } - } - - class Journal { - private val log = new File(dir, "Log.html") - header() - - def appendToJournal(config: ExperimentDef): Unit = { - val message = s"\n\n

${config.name}

" + - s"

- BrokerCount: ${config.brokers}" + - s"

- PartitionCount: ${config.partitions}" + - f"

- Throttle: ${config.throttle.toDouble}%,.0f MB/s" + - f"

- MsgCount: ${config.msgsPerPartition}%,.0f " + - f"

- MsgSize: ${config.msgSize}%,.0f" + - s"

- TargetBytesPerBrokerMB: ${config.targetBytesPerBrokerMB}

" - append(message) - } - - def appendChart(path: String, first: Boolean): Unit = { - val message = new StringBuilder - if (first) - message.append("

") - message.append("\"Chart\"") - if (!first) - message.append("

") - append(message.toString()) - } - - def header(): Unit = { - append("

Replication Quotas Test Rig

") - } - - def footer(): Unit = { - append("") - } - - def append(message: String): Unit = { - val stream = Files.newOutputStream(log.toPath, StandardOpenOption.CREATE, StandardOpenOption.APPEND) - val writer = new PrintWriter(stream) - writer.append(message) - writer.close() - } - - def path(): String = { - log.getAbsolutePath - } - } - -} - diff --git a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java new file mode 100644 index 00000000000..0665a337e8b --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java @@ -0,0 +1,1499 @@ +/* + * 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.reassign; + +import com.fasterxml.jackson.core.JsonParseException; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonMappingException; +import joptsimple.OptionSpec; +import org.apache.kafka.admin.AdminUtils; +import org.apache.kafka.admin.BrokerMetadata; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.AlterConfigOp; +import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.clients.admin.DescribeReplicaLogDirsResult; +import org.apache.kafka.clients.admin.NewPartitionReassignment; +import org.apache.kafka.clients.admin.PartitionReassignment; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicPartitionReplica; +import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.errors.ReplicaNotAvailableException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Time; +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.CommandLineUtils; +import org.apache.kafka.server.util.Json; +import org.apache.kafka.server.util.json.DecodeJson; +import org.apache.kafka.server.util.json.JsonObject; +import org.apache.kafka.server.util.json.JsonValue; +import org.apache.kafka.tools.TerseException; +import org.apache.kafka.tools.ToolsUtils; + +import java.io.IOException; +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.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +@SuppressWarnings("ClassDataAbstractionCoupling") +public class ReassignPartitionsCommand { + private static final String ANY_LOG_DIR = "any"; + + static final String HELP_TEXT = "This tool helps to move topic partitions between replicas."; + + private static final DecodeJson.DecodeInteger INT = new DecodeJson.DecodeInteger(); + + private static final DecodeJson.DecodeString STRING = new DecodeJson.DecodeString(); + + private static final DecodeJson> INT_LIST = DecodeJson.decodeList(INT); + + private static final DecodeJson> STRING_LIST = DecodeJson.decodeList(STRING); + + /** + * The earliest version of the partition reassignment JSON. We will default to this + * version if no other version number is given. + */ + static final int EARLIEST_VERSION = 1; + + /** + * The earliest version of the JSON for each partition reassignment topic. We will + * default to this version if no other version number is given. + */ + static final int EARLIEST_TOPICS_JSON_VERSION = 1; + + // Throttles that are set at the level of an individual broker. + //DynamicConfig.Broker.LeaderReplicationThrottledRateProp + static final String BROKER_LEVEL_LEADER_THROTTLE = "leader.replication.throttled.rate"; + //DynamicConfig.Broker.FollowerReplicationThrottledRateProp + static final String BROKER_LEVEL_FOLLOWER_THROTTLE = "follower.replication.throttled.rate"; + //DynamicConfig.Broker.ReplicaAlterLogDirsIoMaxBytesPerSecondProp + static final String BROKER_LEVEL_LOG_DIR_THROTTLE = "replica.alter.log.dirs.io.max.bytes.per.second"; + static final List BROKER_LEVEL_THROTTLES = Arrays.asList( + BROKER_LEVEL_LEADER_THROTTLE, + BROKER_LEVEL_FOLLOWER_THROTTLE, + BROKER_LEVEL_LOG_DIR_THROTTLE + ); + + // Throttles that are set at the level of an individual topic. + //LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG + static final String TOPIC_LEVEL_LEADER_THROTTLE = "leader.replication.throttled.replicas"; + //LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG + static final String TOPIC_LEVEL_FOLLOWER_THROTTLE = "follower.replication.throttled.replicas"; + private static final List TOPIC_LEVEL_THROTTLES = Arrays.asList( + TOPIC_LEVEL_LEADER_THROTTLE, + TOPIC_LEVEL_FOLLOWER_THROTTLE + ); + + private static final String CANNOT_EXECUTE_BECAUSE_OF_EXISTING_MESSAGE = "Cannot execute because " + + "there is an existing partition assignment. Use --additional to override this and " + + "create a new partition assignment in addition to the existing one. The --additional " + + "flag can also be used to change the throttle by resubmitting the current reassignment."; + + private static final String YOU_MUST_RUN_VERIFY_PERIODICALLY_MESSAGE = "Warning: You must run " + + "--verify periodically, until the reassignment completes, to ensure the throttle " + + "is removed."; + + public static void main(String[] args) { + ReassignPartitionsCommandOptions opts = validateAndParseArgs(args); + boolean failed = true; + Admin adminClient = null; + + try { + Properties props = opts.options.has(opts.commandConfigOpt) + ? Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt)) + : new Properties(); + props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt)); + props.putIfAbsent(AdminClientConfig.CLIENT_ID_CONFIG, "reassign-partitions-tool"); + adminClient = Admin.create(props); + handleAction(adminClient, opts); + failed = false; + } catch (TerseException e) { + System.out.println(e.getMessage()); + } catch (Throwable e) { + System.out.println("Error: " + e.getMessage()); + System.out.println(Utils.stackTrace(e)); + } finally { + // It's good to do this after printing any error stack trace. + if (adminClient != null) { + adminClient.close(); + } + } + // If the command failed, exit with a non-zero exit code. + if (failed) { + Exit.exit(1); + } + } + + private static void handleAction(Admin adminClient, ReassignPartitionsCommandOptions opts) throws IOException, ExecutionException, InterruptedException, TerseException { + if (opts.options.has(opts.verifyOpt)) { + verifyAssignment(adminClient, + Utils.readFileAsString(opts.options.valueOf(opts.reassignmentJsonFileOpt)), + opts.options.has(opts.preserveThrottlesOpt)); + } else if (opts.options.has(opts.generateOpt)) { + generateAssignment(adminClient, + Utils.readFileAsString(opts.options.valueOf(opts.topicsToMoveJsonFileOpt)), + opts.options.valueOf(opts.brokerListOpt), + !opts.options.has(opts.disableRackAware)); + } else if (opts.options.has(opts.executeOpt)) { + executeAssignment(adminClient, + opts.options.has(opts.additionalOpt), + Utils.readFileAsString(opts.options.valueOf(opts.reassignmentJsonFileOpt)), + opts.options.valueOf(opts.interBrokerThrottleOpt), + opts.options.valueOf(opts.replicaAlterLogDirsThrottleOpt), + opts.options.valueOf(opts.timeoutOpt), + Time.SYSTEM); + } else if (opts.options.has(opts.cancelOpt)) { + cancelAssignment(adminClient, + Utils.readFileAsString(opts.options.valueOf(opts.reassignmentJsonFileOpt)), + opts.options.has(opts.preserveThrottlesOpt), + opts.options.valueOf(opts.timeoutOpt), + Time.SYSTEM); + } else if (opts.options.has(opts.listOpt)) { + listReassignments(adminClient); + } else { + throw new RuntimeException("Unsupported action."); + } + } + + /** + * The entry point for the --verify command. + * + * @param adminClient The AdminClient to use. + * @param jsonString The JSON string to use for the topics and partitions to verify. + * @param preserveThrottles True if we should avoid changing topic or broker throttles. + * + * @return A result that is useful for testing. + */ + static VerifyAssignmentResult verifyAssignment(Admin adminClient, + String jsonString, + Boolean preserveThrottles + ) throws ExecutionException, InterruptedException, JsonProcessingException { + Tuple2>>, Map> t0 = parsePartitionReassignmentData(jsonString); + + List>> targetParts = t0.v1; + Map targetLogDirs = t0.v2; + + Tuple2, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts); + + Map partStates = t1.v1; + Boolean partsOngoing = t1.v2; + + Tuple2, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs); + + Map moveStates = t2.v1; + Boolean movesOngoing = t2.v2; + + if (!partsOngoing && !movesOngoing && !preserveThrottles) { + // If the partition assignments and replica assignments are done, clear any throttles + // that were set. We have to clear all throttles, because we don't have enough + // information to know all of the source brokers that might have been involved in the + // previous reassignments. + clearAllThrottles(adminClient, targetParts); + } + + return new VerifyAssignmentResult(partStates, partsOngoing, moveStates, movesOngoing); + } + + /** + * Verify the partition reassignments specified by the user. + * + * @param adminClient The AdminClient to use. + * @param targets The partition reassignments specified by the user. + * + * @return A tuple of the partition reassignment states, and a + * boolean which is true if there are no ongoing + * reassignments (including reassignments not described + * in the JSON file.) + */ + private static Tuple2, Boolean> verifyPartitionAssignments(Admin adminClient, + List>> targets + ) throws ExecutionException, InterruptedException { + Tuple2, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets); + System.out.println(partitionReassignmentStatesToString(t0.v1)); + return t0; + } + + static int compareTopicPartitions(TopicPartition a, TopicPartition b) { + int topicOrder = Objects.compare(a.topic(), b.topic(), String::compareTo); + return topicOrder == 0 ? Integer.compare(a.partition(), b.partition()) : topicOrder; + } + + static int compareTopicPartitionReplicas(TopicPartitionReplica a, TopicPartitionReplica b) { + int brokerOrder = Integer.compare(a.brokerId(), b.brokerId()); + + if (brokerOrder != 0) + return brokerOrder; + + int topicOrder = Objects.compare(a.topic(), b.topic(), String::compareTo); + return topicOrder == 0 ? Integer.compare(a.partition(), b.partition()) : topicOrder; + } + + /** + * Convert partition reassignment states to a human-readable string. + * + * @param states A map from topic partitions to states. + * @return A string summarizing the partition reassignment states. + */ + static String partitionReassignmentStatesToString(Map states) { + List bld = new ArrayList<>(); + bld.add("Status of partition reassignment:"); + states.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).forEach(topicPartition -> { + PartitionReassignmentState state = states.get(topicPartition); + if (state.done) { + if (state.currentReplicas.equals(state.targetReplicas)) { + bld.add(String.format("Reassignment of partition %s is completed.", topicPartition)); + } else { + String currentReplicaStr = state.currentReplicas.stream().map(String::valueOf).collect(Collectors.joining(",")); + String targetReplicaStr = state.targetReplicas.stream().map(String::valueOf).collect(Collectors.joining(",")); + + bld.add("There is no active reassignment of partition " + topicPartition + ", " + + "but replica set is " + currentReplicaStr + " rather than " + + targetReplicaStr + "."); + } + } else { + bld.add(String.format("Reassignment of partition %s is still in progress.", topicPartition)); + } + }); + return bld.stream().collect(Collectors.joining(System.lineSeparator())); + } + + /** + * Find the state of the specified partition reassignments. + * + * @param adminClient The Admin client to use. + * @param targetReassignments The reassignments we want to learn about. + * + * @return A tuple containing the reassignment states for each topic + * partition, plus whether there are any ongoing reassignments. + */ + static Tuple2, Boolean> findPartitionReassignmentStates(Admin adminClient, + List>> targetReassignments + ) throws ExecutionException, InterruptedException { + Map currentReassignments = adminClient. + listPartitionReassignments().reassignments().get(); + + List>> foundReassignments = new ArrayList<>(); + List>> notFoundReassignments = new ArrayList<>(); + + targetReassignments.forEach(reassignment -> { + if (currentReassignments.containsKey(reassignment.v1)) + foundReassignments.add(reassignment); + else + notFoundReassignments.add(reassignment); + }); + + List> foundResults = foundReassignments.stream().map(e -> { + TopicPartition part = e.v1; + List targetReplicas = e.v2; + return new Tuple2<>(part, + new PartitionReassignmentState( + currentReassignments.get(part).replicas(), + targetReplicas, + false)); + }).collect(Collectors.toList()); + + Set topicNamesToLookUp = notFoundReassignments.stream() + .map(e -> e.v1) + .filter(part -> !currentReassignments.containsKey(part)) + .map(TopicPartition::topic) + .collect(Collectors.toSet()); + + Map> topicDescriptions = adminClient. + describeTopics(topicNamesToLookUp).topicNameValues(); + + List> notFoundResults = new ArrayList<>(); + for (Tuple2> e : notFoundReassignments) { + TopicPartition part = e.v1; + List targetReplicas = e.v2; + + if (currentReassignments.containsKey(part)) { + PartitionReassignment reassignment = currentReassignments.get(part); + notFoundResults.add(new Tuple2<>(part, new PartitionReassignmentState( + reassignment.replicas(), + targetReplicas, + false))); + } else { + notFoundResults.add(new Tuple2<>(part, topicDescriptionFutureToState(part.partition(), + topicDescriptions.get(part.topic()), targetReplicas))); + } + } + + Map allResults = new HashMap<>(); + foundResults.forEach(e -> allResults.put(e.v1, e.v2)); + notFoundResults.forEach(e -> allResults.put(e.v1, e.v2)); + + return new Tuple2<>(allResults, !currentReassignments.isEmpty()); + } + + private static PartitionReassignmentState topicDescriptionFutureToState(int partition, + KafkaFuture future, + List targetReplicas + ) throws InterruptedException, ExecutionException { + try { + TopicDescription topicDescription = future.get(); + if (topicDescription.partitions().size() < partition) { + throw new ExecutionException("Too few partitions found", new UnknownTopicOrPartitionException()); + } + return new PartitionReassignmentState( + topicDescription.partitions().get(partition).replicas().stream().map(Node::id).collect(Collectors.toList()), + targetReplicas, + true); + } catch (ExecutionException t) { + if (t.getCause() instanceof UnknownTopicOrPartitionException) + return new PartitionReassignmentState(Collections.emptyList(), targetReplicas, true); + + throw t; + } + } + + /** + * Verify the replica reassignments specified by the user. + * + * @param adminClient The AdminClient to use. + * @param targetReassignments The replica reassignments specified by the user. + * + * @return A tuple of the replica states, and a boolean which is true + * if there are any ongoing replica moves. + * + * Note: Unlike in verifyPartitionAssignments, we will + * return false here even if there are unrelated ongoing + * reassignments. (We don't have an efficient API that + * returns all ongoing replica reassignments.) + */ + private static Tuple2, Boolean> verifyReplicaMoves(Admin adminClient, + Map targetReassignments + ) throws ExecutionException, InterruptedException { + Map moveStates = findLogDirMoveStates(adminClient, targetReassignments); + System.out.println(replicaMoveStatesToString(moveStates)); + return new Tuple2<>(moveStates, !moveStates.values().stream().allMatch(LogDirMoveState::done)); + } + + /** + * Find the state of the specified partition reassignments. + * + * @param adminClient The AdminClient to use. + * @param targetMoves The movements we want to learn about. The map is keyed + * by TopicPartitionReplica, and its values are target log + * directories. + * + * @return The states for each replica movement. + */ + static Map findLogDirMoveStates(Admin adminClient, + Map targetMoves + ) throws ExecutionException, InterruptedException { + Map replicaLogDirInfos = adminClient + .describeReplicaLogDirs(targetMoves.keySet()).all().get(); + + return targetMoves.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> { + TopicPartitionReplica replica = e.getKey(); + String targetLogDir = e.getValue(); + + if (!replicaLogDirInfos.containsKey(replica)) + return new MissingReplicaMoveState(targetLogDir); + + DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica); + + if (info.getCurrentReplicaLogDir() == null) + return new MissingLogDirMoveState(targetLogDir); + + if (info.getFutureReplicaLogDir() == null) { + if (info.getCurrentReplicaLogDir().equals(targetLogDir)) + return new CompletedMoveState(targetLogDir); + + return new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir); + } + + return new ActiveMoveState(info.getCurrentReplicaLogDir(), targetLogDir, info.getFutureReplicaLogDir()); + })); + } + + /** + * Convert replica move states to a human-readable string. + * + * @param states A map from topic partition replicas to states. + * @return A tuple of a summary string, and a boolean describing + * whether there are any active replica moves. + */ + static String replicaMoveStatesToString(Map states) { + List bld = new ArrayList<>(); + states.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitionReplicas).forEach(replica -> { + LogDirMoveState state = states.get(replica); + if (state instanceof MissingLogDirMoveState) { + bld.add("Partition " + replica.topic() + "-" + replica.partition() + " is not found " + + "in any live log dir on broker " + replica.brokerId() + ". There is likely an " + + "offline log directory on the broker."); + } else if (state instanceof MissingReplicaMoveState) { + bld.add("Partition " + replica.topic() + "-" + replica.partition() + " cannot be found " + + "in any live log directory on broker " + replica.brokerId() + "."); + } else if (state instanceof ActiveMoveState) { + String targetLogDir = ((ActiveMoveState) state).targetLogDir; + String futureLogDir = ((ActiveMoveState) state).futureLogDir; + if (targetLogDir.equals(futureLogDir)) { + bld.add("Reassignment of replica " + replica + " is still in progress."); + } else { + bld.add("Partition " + replica.topic() + "-" + replica.partition() + " on broker " + + replica.brokerId() + " is being moved to log dir " + futureLogDir + " " + + "instead of " + targetLogDir + "."); + } + } else if (state instanceof CancelledMoveState) { + String targetLogDir = ((CancelledMoveState) state).targetLogDir; + String currentLogDir = ((CancelledMoveState) state).currentLogDir; + bld.add("Partition " + replica.topic() + "-" + replica.partition() + " on broker " + + replica.brokerId() + " is not being moved from log dir " + currentLogDir + " to " + + targetLogDir + "."); + } else if (state instanceof CompletedMoveState) { + bld.add("Reassignment of replica " + replica + " completed successfully."); + } + }); + + return bld.stream().collect(Collectors.joining(System.lineSeparator())); + } + + /** + * Clear all topic-level and broker-level throttles. + * + * @param adminClient The AdminClient to use. + * @param targetParts The target partitions loaded from the JSON file. + */ + private static void clearAllThrottles(Admin adminClient, + List>> targetParts + ) throws ExecutionException, InterruptedException { + Set brokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet()); + targetParts.forEach(t -> brokers.addAll(t.v2)); + + System.out.printf("Clearing broker-level throttles on broker%s %s%n", + brokers.size() == 1 ? "" : "s", Utils.join(brokers, ",")); + clearBrokerLevelThrottles(adminClient, brokers); + + Set topics = targetParts.stream().map(t -> t.v1.topic()).collect(Collectors.toSet()); + System.out.printf("Clearing topic-level throttles on topic%s %s%n", + topics.size() == 1 ? "" : "s", Utils.join(topics, ",")); + clearTopicLevelThrottles(adminClient, topics); + } + + /** + * Clear all throttles which have been set at the broker level. + * + * @param adminClient The AdminClient to use. + * @param brokers The brokers to clear the throttles for. + */ + private static void clearBrokerLevelThrottles(Admin adminClient, Set brokers) throws ExecutionException, InterruptedException { + Map> configOps = new HashMap<>(); + brokers.forEach(brokerId -> configOps.put( + new ConfigResource(ConfigResource.Type.BROKER, brokerId.toString()), + BROKER_LEVEL_THROTTLES.stream().map(throttle -> new AlterConfigOp( + new ConfigEntry(throttle, null), AlterConfigOp.OpType.DELETE)).collect(Collectors.toList()) + )); + adminClient.incrementalAlterConfigs(configOps).all().get(); + } + + /** + * Clear the reassignment throttles for the specified topics. + * + * @param adminClient The AdminClient to use. + * @param topics The topics to clear the throttles for. + */ + private static void clearTopicLevelThrottles(Admin adminClient, Set topics) throws ExecutionException, InterruptedException { + Map> configOps = topics.stream().collect(Collectors.toMap( + topicName -> new ConfigResource(ConfigResource.Type.TOPIC, topicName), + topicName -> TOPIC_LEVEL_THROTTLES.stream().map(throttle -> new AlterConfigOp(new ConfigEntry(throttle, null), + AlterConfigOp.OpType.DELETE)).collect(Collectors.toList()) + )); + + adminClient.incrementalAlterConfigs(configOps).all().get(); + } + + /** + * The entry point for the --generate command. + * + * @param adminClient The AdminClient to use. + * @param reassignmentJson The JSON string to use for the topics to reassign. + * @param brokerListString The comma-separated string of broker IDs to use. + * @param enableRackAwareness True if rack-awareness should be enabled. + * + * @return A tuple containing the proposed assignment and the + * current assignment. + */ + static Tuple2>, Map>> generateAssignment(Admin adminClient, + String reassignmentJson, + String brokerListString, + Boolean enableRackAwareness + ) throws ExecutionException, InterruptedException, JsonProcessingException { + Tuple2, List> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString); + + List brokersToReassign = t0.v1; + List topicsToReassign = t0.v2; + + Map> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign); + List brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness); + Map> proposedAssignments = calculateAssignment(currentAssignments, brokerMetadatas); + System.out.printf("Current partition replica assignment%n%s%n%n", + formatAsReassignmentJson(currentAssignments, Collections.emptyMap())); + System.out.printf("Proposed partition reassignment configuration%n%s%n", + formatAsReassignmentJson(proposedAssignments, Collections.emptyMap())); + return new Tuple2<>(proposedAssignments, currentAssignments); + } + + /** + * Calculate the new partition assignments to suggest in --generate. + * + * @param currentAssignment The current partition assignments. + * @param brokerMetadatas The rack information for each broker. + * + * @return A map from partitions to the proposed assignments for each. + */ + private static Map> calculateAssignment(Map> currentAssignment, + List brokerMetadatas) { + Map>>> groupedByTopic = new HashMap<>(); + for (Map.Entry> e : currentAssignment.entrySet()) + groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e); + Map> proposedAssignments = new HashMap<>(); + groupedByTopic.forEach((topic, assignment) -> { + List replicas = assignment.get(0).getValue(); + Map> assignedReplicas = AdminUtils. + assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size()); + assignedReplicas.forEach((partition, replicas0) -> + proposedAssignments.put(new TopicPartition(topic, partition), replicas0)); + }); + return proposedAssignments; + } + + private static Map describeTopics(Admin adminClient, + Set topics) throws ExecutionException, InterruptedException { + Map> futures = adminClient.describeTopics(topics).topicNameValues(); + Map res = new HashMap<>(); + for (Map.Entry> e : futures.entrySet()) { + String topicName = e.getKey(); + KafkaFuture topicDescriptionFuture = e.getValue(); + try { + res.put(topicName, topicDescriptionFuture.get()); + } catch (ExecutionException t) { + if (t.getCause() instanceof UnknownTopicOrPartitionException) + throw new ExecutionException( + new UnknownTopicOrPartitionException("Topic " + topicName + " not found.")); + throw t; + } + } + return res; + } + + /** + * Get the current replica assignments for some topics. + * + * @param adminClient The AdminClient to use. + * @param topics The topics to get information about. + * @return A map from partitions to broker assignments. + * If any topic can't be found, an exception will be thrown. + */ + static Map> getReplicaAssignmentForTopics(Admin adminClient, + List topics + ) throws ExecutionException, InterruptedException { + Map> res = new HashMap<>(); + describeTopics(adminClient, new HashSet<>(topics)).forEach((topicName, topicDescription) -> + topicDescription.partitions().forEach(info -> res.put( + new TopicPartition(topicName, info.partition()), + info.replicas().stream().map(Node::id).collect(Collectors.toList()) + ) + )); + return res; + } + + /** + * Get the current replica assignments for some partitions. + * + * @param adminClient The AdminClient to use. + * @param partitions The partitions to get information about. + * @return A map from partitions to broker assignments. + * If any topic can't be found, an exception will be thrown. + */ + static Map> getReplicaAssignmentForPartitions(Admin adminClient, + Set partitions + ) throws ExecutionException, InterruptedException { + Map> res = new HashMap<>(); + describeTopics(adminClient, partitions.stream().map(TopicPartition::topic).collect(Collectors.toSet())).forEach((topicName, topicDescription) -> + topicDescription.partitions().forEach(info -> { + TopicPartition tp = new TopicPartition(topicName, info.partition()); + if (partitions.contains(tp)) + res.put(tp, info.replicas().stream().map(Node::id).collect(Collectors.toList())); + }) + ); + return res; + } + + /** + * Find the rack information for some brokers. + * + * @param adminClient The AdminClient object. + * @param brokers The brokers to gather metadata about. + * @param enableRackAwareness True if we should return rack information, and throw an + * exception if it is inconsistent. + * + * @return The metadata for each broker that was found. + * Brokers that were not found will be omitted. + */ + static List getBrokerMetadata(Admin adminClient, List brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException { + Set brokerSet = new HashSet<>(brokers); + List results = adminClient.describeCluster().nodes().get().stream() + .filter(node -> brokerSet.contains(node.id())) + .map(node -> (enableRackAwareness && node.rack() != null) + ? new BrokerMetadata(node.id(), Optional.of(node.rack())) + : new BrokerMetadata(node.id(), Optional.empty()) + ).collect(Collectors.toList()); + + long numRackless = results.stream().filter(m -> !m.rack.isPresent()).count(); + if (enableRackAwareness && numRackless != 0 && numRackless != results.size()) { + throw new AdminOperationException("Not all brokers have rack information. Add " + + "--disable-rack-aware in command line to make replica assignment without rack " + + "information."); + } + return results; + } + + /** + * Parse and validate data gathered from the command-line for --generate + * In particular, we parse the JSON and validate that duplicate brokers and + * topics don't appear. + * + * @param reassignmentJson The JSON passed to --generate . + * @param brokerList A list of brokers passed to --generate. + * + * @return A tuple of brokers to reassign, topics to reassign + */ + static Tuple2, List> parseGenerateAssignmentArgs(String reassignmentJson, + String brokerList) throws JsonMappingException { + List brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList()); + Set duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign); + if (!duplicateReassignments.isEmpty()) + throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments)); + List topicsToReassign = parseTopicsData(reassignmentJson); + Set duplicateTopicsToReassign = ToolsUtils.duplicates(topicsToReassign); + if (!duplicateTopicsToReassign.isEmpty()) + throw new AdminCommandFailedException(String.format("List of topics to reassign contains duplicate entries: %s", + duplicateTopicsToReassign)); + return new Tuple2<>(brokerListToReassign, topicsToReassign); + } + + /** + * The entry point for the --execute and --execute-additional commands. + * + * @param adminClient The AdminClient to use. + * @param additional Whether --additional was passed. + * @param reassignmentJson The JSON string to use for the topics to reassign. + * @param interBrokerThrottle The inter-broker throttle to use, or a negative + * number to skip using a throttle. + * @param logDirThrottle The replica log directory throttle to use, or a + * negative number to skip using a throttle. + * @param timeoutMs The maximum time in ms to wait for log directory + * replica assignment to begin. + * @param time The Time object to use. + */ + static void executeAssignment(Admin adminClient, + Boolean additional, + String reassignmentJson, + Long interBrokerThrottle, + Long logDirThrottle, + Long timeoutMs, + Time time + ) throws ExecutionException, InterruptedException, JsonProcessingException, TerseException { + Tuple2>, Map> t0 = parseExecuteAssignmentArgs(reassignmentJson); + + Map> proposedParts = t0.v1; + Map proposedReplicas = t0.v2; + Map currentReassignments = adminClient. + listPartitionReassignments().reassignments().get(); + // If there is an existing assignment, check for --additional before proceeding. + // This helps avoid surprising users. + if (!additional && !currentReassignments.isEmpty()) { + throw new TerseException(CANNOT_EXECUTE_BECAUSE_OF_EXISTING_MESSAGE); + } + Set brokers = new HashSet<>(); + proposedParts.values().forEach(brokers::addAll); + + verifyBrokerIds(adminClient, brokers); + Map> currentParts = getReplicaAssignmentForPartitions(adminClient, proposedParts.keySet()); + System.out.println(currentPartitionReplicaAssignmentToString(proposedParts, currentParts)); + + if (interBrokerThrottle >= 0 || logDirThrottle >= 0) { + System.out.println(YOU_MUST_RUN_VERIFY_PERIODICALLY_MESSAGE); + + if (interBrokerThrottle >= 0) { + Map> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts); + modifyReassignmentThrottle(adminClient, moveMap, interBrokerThrottle); + } + + if (logDirThrottle >= 0) { + Set movingBrokers = calculateMovingBrokers(proposedReplicas.keySet()); + modifyLogDirThrottle(adminClient, movingBrokers, logDirThrottle); + } + } + + // Execute the partition reassignments. + Map errors = alterPartitionReassignments(adminClient, proposedParts); + if (!errors.isEmpty()) { + throw new TerseException( + String.format("Error reassigning partition(s):%n%s", + errors.keySet().stream() + .sorted(ReassignPartitionsCommand::compareTopicPartitions) + .map(part -> part + ": " + errors.get(part).getMessage()) + .collect(Collectors.joining(System.lineSeparator())))); + } + System.out.printf("Successfully started partition reassignment%s for %s%n", + proposedParts.size() == 1 ? "" : "s", + proposedParts.keySet().stream() + .sorted(ReassignPartitionsCommand::compareTopicPartitions) + .map(Objects::toString) + .collect(Collectors.joining(","))); + if (!proposedReplicas.isEmpty()) { + executeMoves(adminClient, proposedReplicas, timeoutMs, time); + } + } + + /** + * Execute some partition log directory movements. + * + * @param adminClient The AdminClient to use. + * @param proposedReplicas A map from TopicPartitionReplicas to the + * directories to move them to. + * @param timeoutMs The maximum time in ms to wait for log directory + * replica assignment to begin. + * @param time The Time object to use. + */ + private static void executeMoves(Admin adminClient, + Map proposedReplicas, + Long timeoutMs, + Time time + ) throws InterruptedException, TerseException { + long startTimeMs = time.milliseconds(); + Map pendingReplicas = new HashMap<>(proposedReplicas); + boolean done = false; + do { + Set completed = alterReplicaLogDirs(adminClient, pendingReplicas); + if (!completed.isEmpty()) { + System.out.printf("Successfully started log directory move%s for: %s%n", + completed.size() == 1 ? "" : "s", + completed.stream() + .sorted(ReassignPartitionsCommand::compareTopicPartitionReplicas) + .map(Object::toString) + .collect(Collectors.joining(","))); + } + completed.forEach(pendingReplicas::remove); + if (pendingReplicas.isEmpty()) { + done = true; + } else if (time.milliseconds() >= startTimeMs + timeoutMs) { + throw new TerseException(String.format( + "Timed out before log directory move%s could be started for: %s", + pendingReplicas.size() == 1 ? "" : "s", + pendingReplicas.keySet().stream() + .sorted(ReassignPartitionsCommand::compareTopicPartitionReplicas) + .map(Object::toString) + .collect(Collectors.joining(",")))); + } else { + // If a replica has been moved to a new host and we also specified a particular + // log directory, we will have to keep retrying the alterReplicaLogDirs + // call. It can't take effect until the replica is moved to that host. + time.sleep(100); + } + } while (!done); + } + + /** + * Entry point for the --list command. + * + * @param adminClient The AdminClient to use. + */ + private static void listReassignments(Admin adminClient) throws ExecutionException, InterruptedException { + System.out.println(curReassignmentsToString(adminClient)); + } + + /** + * Convert the current partition reassignments to text. + * + * @param adminClient The AdminClient to use. + * @return A string describing the current partition reassignments. + */ + static String curReassignmentsToString(Admin adminClient) throws ExecutionException, InterruptedException { + Map currentReassignments = adminClient. + listPartitionReassignments().reassignments().get(); + String text = currentReassignments.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).map(part -> { + PartitionReassignment reassignment = currentReassignments.get(part); + List replicas = reassignment.replicas(); + List addingReplicas = reassignment.addingReplicas(); + List removingReplicas = reassignment.removingReplicas(); + + return String.format("%s: replicas: %s.%s%s", + part, + replicas.stream().map(Object::toString).collect(Collectors.joining(",")), + addingReplicas.isEmpty() ? "" : String.format(" adding: %s.", addingReplicas.stream() + .map(Object::toString) + .collect(Collectors.joining(","))), + removingReplicas.isEmpty() ? "" : String.format(" removing: %s.", removingReplicas.stream() + .map(Object::toString) + .collect(Collectors.joining(","))) + ); + }).collect(Collectors.joining(System.lineSeparator())); + + return text.isEmpty() + ? "No partition reassignments found." + : String.format("Current partition reassignments:%n%s", text); + } + + /** + * Verify that all the brokers in an assignment exist. + * + * @param adminClient The AdminClient to use. + * @param brokers The broker IDs to verify. + */ + private static void verifyBrokerIds(Admin adminClient, Set brokers) throws ExecutionException, InterruptedException { + Set allNodeIds = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet()); + Optional unknown = brokers.stream() + .filter(brokerId -> !allNodeIds.contains(brokerId)) + .findFirst(); + if (unknown.isPresent()) + throw new AdminCommandFailedException("Unknown broker id " + unknown.get()); + } + + /** + * Return the string which we want to print to describe the current partition assignment. + * + * @param proposedParts The proposed partition assignment. + * @param currentParts The current partition assignment. + * + * @return The string to print. We will only print information about + * partitions that appear in the proposed partition assignment. + */ + static String currentPartitionReplicaAssignmentToString(Map> proposedParts, + Map> currentParts) throws JsonProcessingException { + Map> partitionsToBeReassigned = currentParts.entrySet().stream() + .filter(e -> proposedParts.containsKey(e.getKey())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + return String.format("Current partition replica assignment%n%n%s%n%nSave this to use as the %s", + formatAsReassignmentJson(partitionsToBeReassigned, Collections.emptyMap()), + "--reassignment-json-file option during rollback"); + } + + /** + * Execute the given partition reassignments. + * + * @param adminClient The admin client object to use. + * @param reassignments A map from topic names to target replica assignments. + * @return A map from partition objects to error strings. + */ + static Map alterPartitionReassignments(Admin adminClient, + Map> reassignments) throws InterruptedException { + Map> args = new HashMap<>(); + reassignments.forEach((part, replicas) -> args.put(part, Optional.of(new NewPartitionReassignment(replicas)))); + Map> results = adminClient.alterPartitionReassignments(args).values(); + Map errors = new HashMap<>(); + for (Map.Entry> e : results.entrySet()) { + try { + e.getValue().get(); + } catch (ExecutionException t) { + errors.put(e.getKey(), t.getCause()); + } + } + return errors; + } + + /** + * Cancel the given partition reassignments. + * + * @param adminClient The admin client object to use. + * @param reassignments The partition reassignments to cancel. + * @return A map from partition objects to error strings. + */ + static Map cancelPartitionReassignments(Admin adminClient, + Set reassignments) throws InterruptedException { + Map> args = new HashMap<>(); + reassignments.forEach(part -> args.put(part, Optional.empty())); + + Map> results = adminClient.alterPartitionReassignments(args).values(); + Map errors = new HashMap<>(); + for (Map.Entry> e : results.entrySet()) { + try { + e.getValue().get(); + } catch (ExecutionException t) { + errors.put(e.getKey(), t.getCause()); + } + } + return errors; + } + + /** + * Compute the in progress partition move from the current reassignments. + * @param currentReassignments All replicas, adding replicas and removing replicas of target partitions + */ + private static Map> calculateCurrentMoveMap(Map currentReassignments) { + Map> moveMap = new HashMap<>(); + // Add the current reassignments to the move map. + currentReassignments.forEach((part, reassignment) -> { + List allReplicas = reassignment.replicas(); + List addingReplicas = reassignment.addingReplicas(); + + // The addingReplicas is included in the replicas during reassignment + Set sources = new HashSet<>(allReplicas); + addingReplicas.forEach(sources::remove); + + Set destinations = new HashSet<>(addingReplicas); + + Map partMoves = moveMap.computeIfAbsent(part.topic(), k -> new HashMap<>()); + partMoves.put(part.partition(), new PartitionMove(sources, destinations)); + }); + return moveMap; + } + + /** + * Calculate the global map of all partitions that are moving. + * + * @param currentReassignments The currently active reassignments. + * @param proposedParts The proposed location of the partitions (destinations replicas only). + * @param currentParts The current location of the partitions that we are + * proposing to move. + * @return A map from topic name to partition map. + * The partition map is keyed on partition index and contains + * the movements for that partition. + */ + static Map> calculateProposedMoveMap(Map currentReassignments, + Map> proposedParts, + Map> currentParts) { + Map> moveMap = calculateCurrentMoveMap(currentReassignments); + for (Map.Entry> e : proposedParts.entrySet()) { + TopicPartition part = e.getKey(); + List replicas = e.getValue(); + Map partMoves = moveMap.computeIfAbsent(part.topic(), k -> new HashMap<>()); + + // If there is a reassignment in progress, use the sources from moveMap, otherwise + // use the sources from currentParts + Set sources = new HashSet<>(); + + if (partMoves.containsKey(part.partition())) { + PartitionMove move = partMoves.get(part.partition()); + sources.addAll(move.sources); + } else if (currentParts.containsKey(part)) + sources.addAll(currentParts.get(part)); + else + throw new RuntimeException("Trying to reassign a topic partition " + part + " with 0 replicas"); + + Set destinations = new HashSet<>(replicas); + destinations.removeAll(sources); + + partMoves.put(part.partition(), new PartitionMove(sources, destinations)); + } + return moveMap; + } + + /** + * Calculate the leader throttle configurations to use. + * + * @param moveMap The movements. + * @return A map from topic names to leader throttle configurations. + */ + static Map calculateLeaderThrottles(Map> moveMap) { + Map results = new HashMap<>(); + moveMap.forEach((topicName, partMoveMap) -> { + Set components = new TreeSet<>(); + partMoveMap.forEach((partId, move) -> + move.sources.forEach(source -> components.add(String.format("%d:%d", partId, source)))); + results.put(topicName, String.join(",", components)); + }); + return results; + } + + /** + * Calculate the follower throttle configurations to use. + * + * @param moveMap The movements. + * @return A map from topic names to follower throttle configurations. + */ + static Map calculateFollowerThrottles(Map> moveMap) { + Map results = new HashMap<>(); + moveMap.forEach((topicName, partMoveMap) -> { + Set components = new TreeSet<>(); + partMoveMap.forEach((partId, move) -> + move.destinations.forEach(destination -> { + if (!move.sources.contains(destination)) { + components.add(String.format("%d:%d", partId, destination)); + } + }) + ); + results.put(topicName, String.join(",", components)); + }); + + return results; + } + + /** + * Calculate all the brokers which are involved in the given partition reassignments. + * + * @param moveMap The partition movements. + * @return A set of all the brokers involved. + */ + static Set calculateReassigningBrokers(Map> moveMap) { + Set reassigningBrokers = new TreeSet<>(); + moveMap.values().forEach(partMoveMap -> partMoveMap.values().forEach(partMove -> { + reassigningBrokers.addAll(partMove.sources); + reassigningBrokers.addAll(partMove.destinations); + })); + return reassigningBrokers; + } + + /** + * Calculate all the brokers which are involved in the given directory movements. + * + * @param replicaMoves The replica movements. + * @return A set of all the brokers involved. + */ + static Set calculateMovingBrokers(Set replicaMoves) { + return replicaMoves.stream().map(TopicPartitionReplica::brokerId).collect(Collectors.toSet()); + } + + /** + * Modify the topic configurations that control inter-broker throttling. + * + * @param adminClient The adminClient object to use. + * @param leaderThrottles A map from topic names to leader throttle configurations. + * @param followerThrottles A map from topic names to follower throttle configurations. + */ + static void modifyTopicThrottles(Admin adminClient, + Map leaderThrottles, + Map followerThrottles) throws ExecutionException, InterruptedException { + Map> configs = new HashMap<>(); + Set topicNames = new HashSet<>(leaderThrottles.keySet()); + topicNames.addAll(followerThrottles.keySet()); + topicNames.forEach(topicName -> { + List ops = new ArrayList<>(); + if (leaderThrottles.containsKey(topicName)) { + ops.add(new AlterConfigOp(new ConfigEntry(TOPIC_LEVEL_LEADER_THROTTLE, leaderThrottles.get(topicName)), AlterConfigOp.OpType.SET)); + } + if (followerThrottles.containsKey(topicName)) { + ops.add(new AlterConfigOp(new ConfigEntry(TOPIC_LEVEL_FOLLOWER_THROTTLE, followerThrottles.get(topicName)), AlterConfigOp.OpType.SET)); + } + if (!ops.isEmpty()) { + configs.put(new ConfigResource(ConfigResource.Type.TOPIC, topicName), ops); + } + }); + adminClient.incrementalAlterConfigs(configs).all().get(); + } + + private static void modifyReassignmentThrottle( + Admin admin, + Map> moveMap, + Long interBrokerThrottle + ) throws ExecutionException, InterruptedException { + Map leaderThrottles = calculateLeaderThrottles(moveMap); + Map followerThrottles = calculateFollowerThrottles(moveMap); + modifyTopicThrottles(admin, leaderThrottles, followerThrottles); + + Set reassigningBrokers = calculateReassigningBrokers(moveMap); + modifyInterBrokerThrottle(admin, reassigningBrokers, interBrokerThrottle); + } + + /** + * Modify the leader/follower replication throttles for a set of brokers. + * + * @param adminClient The Admin instance to use + * @param reassigningBrokers The set of brokers involved in the reassignment + * @param interBrokerThrottle The new throttle (ignored if less than 0) + */ + static void modifyInterBrokerThrottle(Admin adminClient, + Set reassigningBrokers, + long interBrokerThrottle) throws ExecutionException, InterruptedException { + if (interBrokerThrottle >= 0) { + Map> configs = new HashMap<>(); + reassigningBrokers.forEach(brokerId -> { + List ops = new ArrayList<>(); + ops.add(new AlterConfigOp(new ConfigEntry(BROKER_LEVEL_LEADER_THROTTLE, + Long.toString(interBrokerThrottle)), AlterConfigOp.OpType.SET)); + ops.add(new AlterConfigOp(new ConfigEntry(BROKER_LEVEL_FOLLOWER_THROTTLE, + Long.toString(interBrokerThrottle)), AlterConfigOp.OpType.SET)); + configs.put(new ConfigResource(ConfigResource.Type.BROKER, Long.toString(brokerId)), ops); + }); + adminClient.incrementalAlterConfigs(configs).all().get(); + System.out.println("The inter-broker throttle limit was set to " + interBrokerThrottle + " B/s"); + } + } + + /** + * Modify the log dir reassignment throttle for a set of brokers. + * + * @param admin The Admin instance to use + * @param movingBrokers The set of broker to alter the throttle of + * @param logDirThrottle The new throttle (ignored if less than 0) + */ + static void modifyLogDirThrottle(Admin admin, + Set movingBrokers, + long logDirThrottle) throws ExecutionException, InterruptedException { + if (logDirThrottle >= 0) { + Map> configs = new HashMap<>(); + movingBrokers.forEach(brokerId -> { + List ops = new ArrayList<>(); + ops.add(new AlterConfigOp(new ConfigEntry(BROKER_LEVEL_LOG_DIR_THROTTLE, Long.toString(logDirThrottle)), AlterConfigOp.OpType.SET)); + configs.put(new ConfigResource(ConfigResource.Type.BROKER, Long.toString(brokerId)), ops); + }); + admin.incrementalAlterConfigs(configs).all().get(); + System.out.println("The replica-alter-dir throttle limit was set to " + logDirThrottle + " B/s"); + } + } + + /** + * Parse the reassignment JSON string passed to the --execute command. + * + * @param reassignmentJson The JSON string. + * @return A tuple of the partitions to be reassigned and the replicas + * to be reassigned. + */ + static Tuple2>, Map> parseExecuteAssignmentArgs( + String reassignmentJson + ) throws JsonProcessingException { + Tuple2>>, Map> t0 = parsePartitionReassignmentData(reassignmentJson); + + List>> partitionsToBeReassigned = t0.v1; + Map replicaAssignment = t0.v2; + + if (partitionsToBeReassigned.isEmpty()) + throw new AdminCommandFailedException("Partition reassignment list cannot be empty"); + if (partitionsToBeReassigned.stream().anyMatch(t -> t.v2.isEmpty())) { + throw new AdminCommandFailedException("Partition replica list cannot be empty"); + } + Set duplicateReassignedPartitions = ToolsUtils.duplicates(partitionsToBeReassigned.stream().map(t -> t.v1).collect(Collectors.toList())); + if (!duplicateReassignedPartitions.isEmpty()) { + throw new AdminCommandFailedException(String.format( + "Partition reassignment contains duplicate topic partitions: %s", + duplicateReassignedPartitions.stream().map(Object::toString).collect(Collectors.joining(","))) + ); + } + List>> duplicateEntries = partitionsToBeReassigned.stream() + .map(t -> new Tuple2<>(t.v1, ToolsUtils.duplicates(t.v2))) + .filter(t -> !t.v2.isEmpty()) + .collect(Collectors.toList()); + if (!duplicateEntries.isEmpty()) { + String duplicatesMsg = duplicateEntries.stream().map(t -> + String.format("%s contains multiple entries for %s", + t.v1, + t.v2.stream().map(Object::toString).collect(Collectors.joining(","))) + ).collect(Collectors.joining(". ")); + throw new AdminCommandFailedException(String.format("Partition replica lists may not contain duplicate entries: %s", duplicatesMsg)); + } + return new Tuple2<>(partitionsToBeReassigned.stream().collect(Collectors.toMap(t -> t.v1, t -> t.v2)), replicaAssignment); + } + + /** + * The entry point for the --cancel command. + * + * @param adminClient The AdminClient to use. + * @param jsonString The JSON string to use for the topics and partitions to cancel. + * @param preserveThrottles True if we should avoid changing topic or broker throttles. + * @param timeoutMs The maximum time in ms to wait for log directory + * replica assignment to begin. + * @param time The Time object to use. + * + * @return A tuple of the partition reassignments that were cancelled, + * and the replica movements that were cancelled. + */ + static Tuple2, Set> cancelAssignment(Admin adminClient, + String jsonString, + Boolean preserveThrottles, + Long timeoutMs, + Time time + ) throws ExecutionException, InterruptedException, JsonProcessingException, TerseException { + Tuple2>>, Map> t0 = parsePartitionReassignmentData(jsonString); + + List>> targetParts = t0.v1; + Map targetReplicas = t0.v2; + Set targetPartsSet = targetParts.stream().map(t -> t.v1).collect(Collectors.toSet()); + Set curReassigningParts = new HashSet<>(); + adminClient.listPartitionReassignments(targetPartsSet).reassignments().get().forEach((part, reassignment) -> { + if (reassignment.addingReplicas().isEmpty() || !reassignment.removingReplicas().isEmpty()) + curReassigningParts.add(part); + }); + if (!curReassigningParts.isEmpty()) { + Map errors = cancelPartitionReassignments(adminClient, curReassigningParts); + if (!errors.isEmpty()) { + throw new TerseException(String.format( + "Error cancelling partition reassignment%s for:%n%s", + errors.size() == 1 ? "" : "s", + errors.keySet().stream() + .sorted(ReassignPartitionsCommand::compareTopicPartitions) + .map(part -> part + ": " + errors.get(part).getMessage()).collect(Collectors.joining(System.lineSeparator()))) + ); + } + System.out.printf("Successfully cancelled partition reassignment%s for: %s%n", + curReassigningParts.size() == 1 ? "" : "s", + curReassigningParts.stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).map(Object::toString).collect(Collectors.joining(",")) + ); + } else { + System.out.println("None of the specified partition reassignments are active."); + } + Map curMovingParts = new HashMap<>(); + findLogDirMoveStates(adminClient, targetReplicas).forEach((part, moveState) -> { + if (moveState instanceof ActiveMoveState) + curMovingParts.put(part, ((ActiveMoveState) moveState).currentLogDir); + }); + if (curMovingParts.isEmpty()) { + System.out.print("None of the specified partition moves are active."); + } else { + executeMoves(adminClient, curMovingParts, timeoutMs, time); + } + if (!preserveThrottles) { + clearAllThrottles(adminClient, targetParts); + } + return new Tuple2<>(curReassigningParts, curMovingParts.keySet()); + } + + private static String formatAsReassignmentJson(Map> partitionsToBeReassigned, + Map replicaLogDirAssignment) throws JsonProcessingException { + List> partitions = new ArrayList<>(); + partitionsToBeReassigned.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).forEach(tp -> { + List replicas = partitionsToBeReassigned.get(tp); + Map data = new LinkedHashMap<>(); + + data.put("topic", tp.topic()); + data.put("partition", tp.partition()); + data.put("replicas", replicas); + data.put("log_dirs", replicas.stream() + .map(r -> replicaLogDirAssignment.getOrDefault(new TopicPartitionReplica(tp.topic(), tp.partition(), r), ANY_LOG_DIR)) + .collect(Collectors.toList())); + + partitions.add(data); + }); + + Map results = new LinkedHashMap<>(); + + results.put("version", 1); + results.put("partitions", partitions); + + return Json.encodeAsString(results); + } + + private static List parseTopicsData(String jsonData) throws JsonMappingException { + Optional parsed = Json.parseFull(jsonData); + if (parsed.isPresent()) { + JsonValue js = parsed.get(); + Optional version = js.asJsonObject().get("version"); + return parseTopicsData(version.isPresent() ? version.get().to(INT) : EARLIEST_TOPICS_JSON_VERSION, js); + } else { + throw new AdminOperationException("The input string is not a valid JSON"); + } + } + + private static List parseTopicsData(int version, JsonValue js) throws JsonMappingException { + switch (version) { + case 1: + List results = new ArrayList<>(); + Optional partitionsSeq = js.asJsonObject().get("topics"); + if (partitionsSeq.isPresent()) { + Iterator iter = partitionsSeq.get().asJsonArray().iterator(); + while (iter.hasNext()) { + results.add(iter.next().asJsonObject().apply("topic").to(STRING)); + } + } + return results; + + default: + throw new AdminOperationException("Not supported version field value " + version); + } + } + + private static Tuple2>>, Map> parsePartitionReassignmentData( + String jsonData + ) throws JsonProcessingException { + JsonValue js; + try { + js = Json.tryParseFull(jsonData); + } catch (JsonParseException f) { + throw new AdminOperationException(f); + } + Optional version = js.asJsonObject().get("version"); + return parsePartitionReassignmentData(version.isPresent() ? version.get().to(INT) : EARLIEST_VERSION, js); + } + + // Parses without deduplicating keys so the data can be checked before allowing reassignment to proceed + private static Tuple2>>, Map> parsePartitionReassignmentData( + int version, JsonValue jsonData + ) throws JsonMappingException { + switch (version) { + case 1: + List>> partitionAssignment = new ArrayList<>(); + Map replicaAssignment = new HashMap<>(); + + Optional partitionsSeq = jsonData.asJsonObject().get("partitions"); + if (partitionsSeq.isPresent()) { + Iterator iter = partitionsSeq.get().asJsonArray().iterator(); + while (iter.hasNext()) { + JsonObject partitionFields = iter.next().asJsonObject(); + String topic = partitionFields.apply("topic").to(STRING); + int partition = partitionFields.apply("partition").to(INT); + List newReplicas = partitionFields.apply("replicas").to(INT_LIST); + Optional logDirsOpts = partitionFields.get("log_dirs"); + List newLogDirs; + if (logDirsOpts.isPresent()) + newLogDirs = logDirsOpts.get().to(STRING_LIST); + else + newLogDirs = newReplicas.stream().map(r -> ANY_LOG_DIR).collect(Collectors.toList()); + if (newReplicas.size() != newLogDirs.size()) + throw new AdminCommandFailedException("Size of replicas list " + newReplicas + " is different from " + + "size of log dirs list " + newLogDirs + " for partition " + new TopicPartition(topic, partition)); + partitionAssignment.add(new Tuple2<>(new TopicPartition(topic, partition), newReplicas)); + for (int i = 0; i < newLogDirs.size(); i++) { + Integer replica = newReplicas.get(i); + String logDir = newLogDirs.get(i); + + if (logDir.equals(ANY_LOG_DIR)) + continue; + + replicaAssignment.put(new TopicPartitionReplica(topic, partition, replica), logDir); + } + } + } + + return new Tuple2<>(partitionAssignment, replicaAssignment); + + default: + throw new AdminOperationException("Not supported version field value " + version); + } + } + + static ReassignPartitionsCommandOptions validateAndParseArgs(String[] args) { + ReassignPartitionsCommandOptions opts = new ReassignPartitionsCommandOptions(args); + + CommandLineUtils.maybePrintHelpOrVersion(opts, HELP_TEXT); + + // Determine which action we should perform. + List> validActions = Arrays.asList(opts.generateOpt, opts.executeOpt, opts.verifyOpt, + opts.cancelOpt, opts.listOpt); + + List> allActions = validActions.stream() + .filter(a -> opts.options.has(a)) + .collect(Collectors.toList()); + + if (allActions.size() != 1) { + CommandLineUtils.printUsageAndExit(opts.parser, String.format("Command must include exactly one action: %s", + validActions.stream().map(a -> "--" + a.options().get(0)).collect(Collectors.joining(", ")))); + } + + OptionSpec action = allActions.get(0); + + if (!opts.options.has(opts.bootstrapServerOpt)) + CommandLineUtils.printUsageAndExit(opts.parser, "Please specify --bootstrap-server"); + + // Make sure that we have all the required arguments for our action. + Map, List>> requiredArgs = new HashMap<>(); + + requiredArgs.put(opts.verifyOpt, Collections.singletonList( + opts.reassignmentJsonFileOpt + )); + requiredArgs.put(opts.generateOpt, Arrays.asList( + opts.topicsToMoveJsonFileOpt, + opts.brokerListOpt + )); + requiredArgs.put(opts.executeOpt, Collections.singletonList( + opts.reassignmentJsonFileOpt + )); + requiredArgs.put(opts.cancelOpt, Collections.singletonList( + opts.reassignmentJsonFileOpt + )); + requiredArgs.put(opts.listOpt, Collections.emptyList()); + + CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, requiredArgs.get(action).toArray(new OptionSpec[0])); + + Map, List>> permittedArgs = new HashMap<>(); + + permittedArgs.put(opts.verifyOpt, Arrays.asList( + opts.bootstrapServerOpt, + opts.commandConfigOpt, + opts.preserveThrottlesOpt + )); + permittedArgs.put(opts.generateOpt, Arrays.asList( + opts.bootstrapServerOpt, + opts.brokerListOpt, + opts.commandConfigOpt, + opts.disableRackAware + )); + permittedArgs.put(opts.executeOpt, Arrays.asList( + opts.additionalOpt, + opts.bootstrapServerOpt, + opts.commandConfigOpt, + opts.interBrokerThrottleOpt, + opts.replicaAlterLogDirsThrottleOpt, + opts.timeoutOpt + )); + permittedArgs.put(opts.cancelOpt, Arrays.asList( + opts.bootstrapServerOpt, + opts.commandConfigOpt, + opts.preserveThrottlesOpt, + opts.timeoutOpt + )); + permittedArgs.put(opts.listOpt, Arrays.asList( + opts.bootstrapServerOpt, + opts.commandConfigOpt + )); + + opts.options.specs().forEach(opt -> { + if (!opt.equals(action) && + !requiredArgs.getOrDefault(action, Collections.emptyList()).contains(opt) && + !permittedArgs.getOrDefault(action, Collections.emptyList()).contains(opt)) { + CommandLineUtils.printUsageAndExit(opts.parser, + String.format("Option \"%s\" can't be used with action \"%s\"", opt, action)); + } + }); + + return opts; + } + + static Set alterReplicaLogDirs(Admin adminClient, + Map assignment) throws InterruptedException { + Set results = new HashSet<>(); + Map> values = adminClient.alterReplicaLogDirs(assignment).values(); + + for (Map.Entry> e : values.entrySet()) { + TopicPartitionReplica replica = e.getKey(); + KafkaFuture future = e.getValue(); + try { + future.get(); + results.add(replica); + } catch (ExecutionException t) { + // Ignore ReplicaNotAvailableException. It is OK if the replica is not + // available at this moment. + if (t.getCause() instanceof ReplicaNotAvailableException) + continue; + throw new AdminCommandFailedException("Failed to alter dir for " + replica, t); + } + } + return results; + } +} diff --git a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandOptions.java b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandOptions.java index d0e97c036dd..f35dfe25f73 100644 --- a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandOptions.java +++ b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandOptions.java @@ -109,68 +109,4 @@ public class ReassignPartitionsCommandOptions extends CommandDefaultOptions { options = parser.parse(args); } - - public OptionSpec verifyOpt() { - return verifyOpt; - } - - public OptionSpec generateOpt() { - return generateOpt; - } - - public OptionSpec executeOpt() { - return executeOpt; - } - - public OptionSpec cancelOpt() { - return cancelOpt; - } - - public OptionSpec listOpt() { - return listOpt; - } - - public OptionSpec bootstrapServerOpt() { - return bootstrapServerOpt; - } - - public OptionSpec commandConfigOpt() { - return commandConfigOpt; - } - - public OptionSpec reassignmentJsonFileOpt() { - return reassignmentJsonFileOpt; - } - - public OptionSpec topicsToMoveJsonFileOpt() { - return topicsToMoveJsonFileOpt; - } - - public OptionSpec brokerListOpt() { - return brokerListOpt; - } - - public OptionSpec disableRackAware() { - return disableRackAware; - } - - public OptionSpec interBrokerThrottleOpt() { - return interBrokerThrottleOpt; - } - - public OptionSpec replicaAlterLogDirsThrottleOpt() { - return replicaAlterLogDirsThrottleOpt; - } - - public OptionSpec timeoutOpt() { - return timeoutOpt; - } - - public OptionSpec additionalOpt() { - return additionalOpt; - } - - public OptionSpec preserveThrottlesOpt() { - return preserveThrottlesOpt; - } } diff --git a/tools/src/main/java/org/apache/kafka/tools/reassign/Tuple2.java b/tools/src/main/java/org/apache/kafka/tools/reassign/Tuple2.java new file mode 100644 index 00000000000..a9e84317cf7 --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/tools/reassign/Tuple2.java @@ -0,0 +1,48 @@ +/* + * 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.reassign; + +import java.util.Objects; + +public final class Tuple2 { + public final V1 v1; + + public final V2 v2; + + public Tuple2(V1 v1, V2 v2) { + this.v1 = v1; + this.v2 = v2; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Tuple2 tuple = (Tuple2) o; + return Objects.equals(v1, tuple.v1) && Objects.equals(v2, tuple.v2); + } + + @Override + public int hashCode() { + return Objects.hash(v1, v2); + } + + @Override + public String toString() { + return "Tuple2{v1=" + v1 + ", v2=" + v2 + '}'; + } +} diff --git a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandArgsTest.java b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandArgsTest.java index 7b267646d49..c248796d25f 100644 --- a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandArgsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandArgsTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.tools.reassign; -import kafka.admin.ReassignPartitionsCommand; import org.apache.kafka.common.utils.Exit; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -88,9 +87,9 @@ public class ReassignPartitionsCommandArgsTest { "--bootstrap-server", "localhost:1234", "--execute", "--reassignment-json-file", "myfile.json"}; - ReassignPartitionsCommand.ReassignPartitionsCommandOptions opts = ReassignPartitionsCommand.validateAndParseArgs(args); - assertEquals(10000L, opts.options.valueOf(opts.timeoutOpt())); - assertEquals(-1L, opts.options.valueOf(opts.interBrokerThrottleOpt())); + ReassignPartitionsCommandOptions opts = ReassignPartitionsCommand.validateAndParseArgs(args); + assertEquals(10000L, opts.options.valueOf(opts.timeoutOpt)); + assertEquals(-1L, opts.options.valueOf(opts.interBrokerThrottleOpt)); } @Test @@ -115,7 +114,7 @@ public class ReassignPartitionsCommandArgsTest { @Test public void shouldFailIfNoArgs() { String[] args = new String[0]; - shouldFailWith(ReassignPartitionsCommand.helpText(), args); + shouldFailWith(ReassignPartitionsCommand.HELP_TEXT, args); } @Test @@ -225,7 +224,7 @@ public class ReassignPartitionsCommandArgsTest { public void shouldPrintHelpTextIfHelpArg() { String[] args = new String[] {"--help"}; // note, this is not actually a failed case, it's just we share the same `printUsageAndExit` method when wrong arg received - shouldFailWith(ReassignPartitionsCommand.helpText(), args); + shouldFailWith(ReassignPartitionsCommand.HELP_TEXT, args); } ///// Test --verify diff --git a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java index 896e4f74ffd..b71331a6b23 100644 --- a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.tools.reassign; -import kafka.admin.ReassignPartitionsCommand; +import com.fasterxml.jackson.core.JsonProcessingException; import kafka.cluster.Partition; import kafka.log.UnifiedLog; import kafka.server.HostedPartition; @@ -42,20 +42,18 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.tools.TerseException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import scala.None$; import scala.Option; import scala.Some$; -import scala.Tuple2; import scala.collection.JavaConverters; import scala.collection.Seq; import java.io.Closeable; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -72,16 +70,17 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.stream.IntStream; -import static kafka.admin.ReassignPartitionsCommand.brokerLevelFollowerThrottle; -import static kafka.admin.ReassignPartitionsCommand.brokerLevelLeaderThrottle; -import static kafka.admin.ReassignPartitionsCommand.brokerLevelLogDirThrottle; -import static kafka.admin.ReassignPartitionsCommand.brokerLevelThrottles; -import static kafka.admin.ReassignPartitionsCommand.cancelAssignment; -import static kafka.admin.ReassignPartitionsCommand.executeAssignment; -import static kafka.admin.ReassignPartitionsCommand.verifyAssignment; +import static java.util.Arrays.asList; import static org.apache.kafka.server.common.MetadataVersion.IBP_2_7_IV1; import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS; import static org.apache.kafka.tools.ToolsTestUtils.TEST_WITH_PARAMETERIZED_QUORUM_NAME; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.BROKER_LEVEL_FOLLOWER_THROTTLE; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.BROKER_LEVEL_LEADER_THROTTLE; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.BROKER_LEVEL_LOG_DIR_THROTTLE; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.BROKER_LEVEL_THROTTLES; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.cancelAssignment; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.executeAssignment; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.verifyAssignment; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -102,10 +101,7 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { IntStream.range(0, 4).forEach(brokerId -> { Map brokerConfig = new HashMap<>(); - brokerLevelThrottles().foreach(throttle -> { - brokerConfig.put(throttle, -1L); - return null; - }); + BROKER_LEVEL_THROTTLES.forEach(throttle -> brokerConfig.put(throttle, -1L)); unthrottledBrokerConfigs.put(brokerId, brokerConfig); }); @@ -170,8 +166,8 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { // Check that the assignment has not yet been started yet. Map initialAssignment = new HashMap<>(); - initialAssignment.put(foo0, new PartitionReassignmentState(Arrays.asList(0, 1, 2), Arrays.asList(0, 1, 3), true)); - initialAssignment.put(bar0, new PartitionReassignmentState(Arrays.asList(3, 2, 1), Arrays.asList(3, 2, 0), true)); + initialAssignment.put(foo0, new PartitionReassignmentState(asList(0, 1, 2), asList(0, 1, 3), true)); + initialAssignment.put(bar0, new PartitionReassignmentState(asList(3, 2, 1), asList(3, 2, 0), true)); waitForVerifyAssignment(cluster.adminClient, assignment, false, new VerifyAssignmentResult(initialAssignment)); @@ -180,11 +176,11 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { runExecuteAssignment(cluster.adminClient, false, assignment, -1L, -1L); assertEquals(unthrottledBrokerConfigs, describeBrokerLevelThrottles(unthrottledBrokerConfigs.keySet())); Map finalAssignment = new HashMap<>(); - finalAssignment.put(foo0, new PartitionReassignmentState(Arrays.asList(0, 1, 3), Arrays.asList(0, 1, 3), true)); - finalAssignment.put(bar0, new PartitionReassignmentState(Arrays.asList(3, 2, 0), Arrays.asList(3, 2, 0), true)); + finalAssignment.put(foo0, new PartitionReassignmentState(asList(0, 1, 3), asList(0, 1, 3), true)); + finalAssignment.put(bar0, new PartitionReassignmentState(asList(3, 2, 0), asList(3, 2, 0), true)); - kafka.admin.ReassignPartitionsCommand.VerifyAssignmentResult verifyAssignmentResult = runVerifyAssignment(cluster.adminClient, assignment, false); - assertFalse(verifyAssignmentResult.movesOngoing()); + VerifyAssignmentResult verifyAssignmentResult = runVerifyAssignment(cluster.adminClient, assignment, false); + assertFalse(verifyAssignmentResult.movesOngoing); // Wait for the assignment to complete waitForVerifyAssignment(cluster.adminClient, assignment, false, @@ -209,12 +205,12 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { // Set the high water mark of foo-0 to 123 on its leader. TopicPartition part = new TopicPartition("foo", 0); - cluster.servers.get(0).replicaManager().logManager().truncateFullyAndStartAt(part, 123L, false, None$.empty()); + cluster.servers.get(0).replicaManager().logManager().truncateFullyAndStartAt(part, 123L, false, Option.empty()); // Execute the assignment runExecuteAssignment(cluster.adminClient, false, assignment, -1L, -1L); Map finalAssignment = Collections.singletonMap(part, - new PartitionReassignmentState(Arrays.asList(3, 1, 2), Arrays.asList(3, 1, 2), true)); + new PartitionReassignmentState(asList(3, 1, 2), asList(3, 1, 2), true)); // Wait for the assignment to complete waitForVerifyAssignment(cluster.adminClient, assignment, false, @@ -243,18 +239,18 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { // Execute the assignment with a low throttle long initialThrottle = 1L; runExecuteAssignment(cluster.adminClient, false, assignment, initialThrottle, -1L); - waitForInterBrokerThrottle(Arrays.asList(0, 1, 2, 3), initialThrottle); + waitForInterBrokerThrottle(asList(0, 1, 2, 3), initialThrottle); // Now update the throttle and verify the reassignment completes long updatedThrottle = 300000L; runExecuteAssignment(cluster.adminClient, true, assignment, updatedThrottle, -1L); - waitForInterBrokerThrottle(Arrays.asList(0, 1, 2, 3), updatedThrottle); + waitForInterBrokerThrottle(asList(0, 1, 2, 3), updatedThrottle); Map finalAssignment = new HashMap<>(); finalAssignment.put(new TopicPartition("foo", 0), - new PartitionReassignmentState(Arrays.asList(0, 3, 2), Arrays.asList(0, 3, 2), true)); + new PartitionReassignmentState(asList(0, 3, 2), asList(0, 3, 2), true)); finalAssignment.put(new TopicPartition("baz", 2), - new PartitionReassignmentState(Arrays.asList(3, 2, 1), Arrays.asList(3, 2, 1), true)); + new PartitionReassignmentState(asList(3, 2, 1), asList(3, 2, 1), true)); // Now remove the throttles. waitForVerifyAssignment(cluster.adminClient, assignment, false, @@ -280,47 +276,47 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { // Check that the assignment has not yet been started yet. Map initialAssignment = new HashMap<>(); initialAssignment.put(new TopicPartition("foo", 0), - new PartitionReassignmentState(Arrays.asList(0, 1, 2), Arrays.asList(0, 3, 2), true)); + new PartitionReassignmentState(asList(0, 1, 2), asList(0, 3, 2), true)); initialAssignment.put(new TopicPartition("baz", 2), - new PartitionReassignmentState(Arrays.asList(0, 2, 1), Arrays.asList(3, 2, 1), true)); - assertEquals(asScala(new VerifyAssignmentResult(initialAssignment)), runVerifyAssignment(cluster.adminClient, assignment, false)); + new PartitionReassignmentState(asList(0, 2, 1), asList(3, 2, 1), true)); + assertEquals(new VerifyAssignmentResult(initialAssignment), runVerifyAssignment(cluster.adminClient, assignment, false)); assertEquals(unthrottledBrokerConfigs, describeBrokerLevelThrottles(unthrottledBrokerConfigs.keySet())); // Execute the assignment long interBrokerThrottle = 300000L; runExecuteAssignment(cluster.adminClient, false, assignment, interBrokerThrottle, -1L); - waitForInterBrokerThrottle(Arrays.asList(0, 1, 2, 3), interBrokerThrottle); + waitForInterBrokerThrottle(asList(0, 1, 2, 3), interBrokerThrottle); Map finalAssignment = new HashMap<>(); finalAssignment.put(new TopicPartition("foo", 0), - new PartitionReassignmentState(Arrays.asList(0, 3, 2), Arrays.asList(0, 3, 2), true)); + new PartitionReassignmentState(asList(0, 3, 2), asList(0, 3, 2), true)); finalAssignment.put(new TopicPartition("baz", 2), - new PartitionReassignmentState(Arrays.asList(3, 2, 1), Arrays.asList(3, 2, 1), true)); + new PartitionReassignmentState(asList(3, 2, 1), asList(3, 2, 1), true)); // Wait for the assignment to complete TestUtils.waitUntilTrue( () -> { // Check the reassignment status. - kafka.admin.ReassignPartitionsCommand.VerifyAssignmentResult result = runVerifyAssignment(cluster.adminClient, assignment, true); + VerifyAssignmentResult result = runVerifyAssignment(cluster.adminClient, assignment, true); - if (!result.partsOngoing()) { + if (!result.partsOngoing) { return true; } else { assertFalse( - result.partStates().values().forall(ReassignPartitionsCommand.PartitionReassignmentState::done), + result.partStates.values().stream().allMatch(state -> state.done), "Expected at least one partition reassignment to be ongoing when result = " + result ); - assertEquals(seq(0, 3, 2), result.partStates().get(new TopicPartition("foo", 0)).get().targetReplicas()); - assertEquals(seq(3, 2, 1), result.partStates().get(new TopicPartition("baz", 2)).get().targetReplicas()); + assertEquals(asList(0, 3, 2), result.partStates.get(new TopicPartition("foo", 0)).targetReplicas); + assertEquals(asList(3, 2, 1), result.partStates.get(new TopicPartition("baz", 2)).targetReplicas); System.out.println("Current result: " + result); - waitForInterBrokerThrottle(Arrays.asList(0, 1, 2, 3), interBrokerThrottle); + waitForInterBrokerThrottle(asList(0, 1, 2, 3), interBrokerThrottle); return false; } }, () -> "Expected reassignment to complete.", DEFAULT_MAX_WAIT_MS, 100L); waitForVerifyAssignment(cluster.adminClient, assignment, true, new VerifyAssignmentResult(finalAssignment)); // The throttles should still have been preserved, since we ran with --preserve-throttles - waitForInterBrokerThrottle(Arrays.asList(0, 1, 2, 3), interBrokerThrottle); + waitForInterBrokerThrottle(asList(0, 1, 2, 3), interBrokerThrottle); // Now remove the throttles. waitForVerifyAssignment(cluster.adminClient, assignment, false, new VerifyAssignmentResult(finalAssignment)); @@ -345,8 +341,8 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { false, 500, SecurityProtocol.PLAINTEXT, - None$.empty(), - None$.empty(), + Option.empty(), + Option.empty(), new ByteArrayDeserializer(), new ByteArrayDeserializer() ); @@ -358,9 +354,9 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { } finally { consumer.close(); } - TestUtils.removeReplicationThrottleForPartitions(cluster.adminClient, seq(0, 1, 2, 3), set(part)); + TestUtils.removeReplicationThrottleForPartitions(cluster.adminClient, seq(asList(0, 1, 2, 3)), mutableSet(part).toSet()); Map finalAssignment = Collections.singletonMap(part, - new PartitionReassignmentState(Arrays.asList(3, 2, 1), Arrays.asList(3, 2, 1), true)); + new PartitionReassignmentState(asList(3, 2, 1), asList(3, 2, 1), true)); waitForVerifyAssignment(cluster.adminClient, assignment, false, new VerifyAssignmentResult(finalAssignment)); } @@ -386,27 +382,27 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { describeBrokerLevelThrottles(unthrottledBrokerConfigs.keySet())); long interBrokerThrottle = 1L; runExecuteAssignment(cluster.adminClient, false, assignment, interBrokerThrottle, -1L); - waitForInterBrokerThrottle(Arrays.asList(0, 1, 2, 3), interBrokerThrottle); + waitForInterBrokerThrottle(asList(0, 1, 2, 3), interBrokerThrottle); Map partStates = new HashMap<>(); - partStates.put(foo0, new PartitionReassignmentState(Arrays.asList(0, 1, 3, 2), Arrays.asList(0, 1, 3), false)); - partStates.put(baz1, new PartitionReassignmentState(Arrays.asList(0, 2, 3, 1), Arrays.asList(0, 2, 3), false)); + partStates.put(foo0, new PartitionReassignmentState(asList(0, 1, 3, 2), asList(0, 1, 3), false)); + partStates.put(baz1, new PartitionReassignmentState(asList(0, 2, 3, 1), asList(0, 2, 3), false)); // Verify that the reassignment is running. The very low throttle should keep it // from completing before this runs. waitForVerifyAssignment(cluster.adminClient, assignment, true, new VerifyAssignmentResult(partStates, true, Collections.emptyMap(), false)); // Cancel the reassignment. - assertEquals(new Tuple2<>(set(foo0, baz1), set()), runCancelAssignment(cluster.adminClient, assignment, true)); + assertEquals(new Tuple2<>(new HashSet<>(asList(foo0, baz1)), Collections.emptySet()), runCancelAssignment(cluster.adminClient, assignment, true)); // Broker throttles are still active because we passed --preserve-throttles - waitForInterBrokerThrottle(Arrays.asList(0, 1, 2, 3), interBrokerThrottle); + waitForInterBrokerThrottle(asList(0, 1, 2, 3), interBrokerThrottle); // Cancelling the reassignment again should reveal nothing to cancel. - assertEquals(new Tuple2<>(set(), set()), runCancelAssignment(cluster.adminClient, assignment, false)); + assertEquals(new Tuple2<>(Collections.emptySet(), Collections.emptySet()), runCancelAssignment(cluster.adminClient, assignment, false)); // This time, the broker throttles were removed. waitForBrokerLevelThrottles(unthrottledBrokerConfigs); // Verify that there are no ongoing reassignments. - assertFalse(runVerifyAssignment(cluster.adminClient, assignment, false).partsOngoing()); + assertFalse(runVerifyAssignment(cluster.adminClient, assignment, false).partsOngoing); // Verify that the partition is removed from cancelled replicas verifyReplicaDeleted(foo0, 3); verifyReplicaDeleted(baz1, 3); @@ -429,8 +425,8 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { // We will throttle replica 4 so that only replica 3 joins the ISR TestUtils.setReplicationThrottleForPartitions( cluster.adminClient, - seq(4), - set(foo0), + seq(asList(4)), + mutableSet(foo0).toSet(), 1 ); @@ -443,13 +439,13 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { -1L ); TestUtils.waitUntilTrue( - () -> Objects.equals(TestUtils.currentIsr(cluster.adminClient, foo0), set(0, 1, 2, 3)), + () -> Objects.equals(TestUtils.currentIsr(cluster.adminClient, foo0), mutableSet(0, 1, 2, 3).toSet()), () -> "Timed out while waiting for replica 3 to join the ISR", DEFAULT_MAX_WAIT_MS, 100L ); // Now cancel the assignment and verify that the partition is removed from cancelled replicas - assertEquals(new Tuple2<>(set(foo0), set()), runCancelAssignment(cluster.adminClient, assignment, true)); + assertEquals(new Tuple2<>(Collections.singleton(foo0), Collections.emptySet()), runCancelAssignment(cluster.adminClient, assignment, true)); verifyReplicaDeleted(foo0, 3); verifyReplicaDeleted(foo0, 4); } @@ -473,17 +469,17 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { private void waitForLogDirThrottle(Set throttledBrokers, Long logDirThrottle) { Map throttledConfigMap = new HashMap<>(); - throttledConfigMap.put(brokerLevelLeaderThrottle(), -1L); - throttledConfigMap.put(brokerLevelFollowerThrottle(), -1L); - throttledConfigMap.put(brokerLevelLogDirThrottle(), logDirThrottle); + throttledConfigMap.put(BROKER_LEVEL_LEADER_THROTTLE, -1L); + throttledConfigMap.put(BROKER_LEVEL_FOLLOWER_THROTTLE, -1L); + throttledConfigMap.put(BROKER_LEVEL_LOG_DIR_THROTTLE, logDirThrottle); waitForBrokerThrottles(throttledBrokers, throttledConfigMap); } private void waitForInterBrokerThrottle(List throttledBrokers, Long interBrokerThrottle) { Map throttledConfigMap = new HashMap<>(); - throttledConfigMap.put(brokerLevelLeaderThrottle(), interBrokerThrottle); - throttledConfigMap.put(brokerLevelFollowerThrottle(), interBrokerThrottle); - throttledConfigMap.put(brokerLevelLogDirThrottle(), -1L); + throttledConfigMap.put(BROKER_LEVEL_LEADER_THROTTLE, interBrokerThrottle); + throttledConfigMap.put(BROKER_LEVEL_FOLLOWER_THROTTLE, interBrokerThrottle); + throttledConfigMap.put(BROKER_LEVEL_LOG_DIR_THROTTLE, -1L); waitForBrokerThrottles(throttledBrokers, throttledConfigMap); } @@ -526,10 +522,9 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { .get(); Map throttles = new HashMap<>(); - brokerLevelThrottles().foreach(throttleName -> { + BROKER_LEVEL_THROTTLES.forEach(throttleName -> { String configValue = Optional.ofNullable(brokerConfigs.get(throttleName)).map(ConfigEntry::value).orElse("-1"); throttles.put(throttleName, Long.parseLong(configValue)); - return null; }); results.put(brokerId, throttles); } @@ -549,7 +544,7 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { cluster.produceMessages(topicPartition.topic(), topicPartition.partition(), 700); int targetBrokerId = 0; - List replicas = Arrays.asList(0, 1, 2); + List replicas = asList(0, 1, 2); LogDirReassignment reassignment = buildLogDirReassignment(topicPartition, targetBrokerId, replicas); // Start the replica move, but throttle it to be very slow so that it can't complete @@ -561,7 +556,7 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { // Check the output of --verify waitForVerifyAssignment(cluster.adminClient, reassignment.json, true, new VerifyAssignmentResult(Collections.singletonMap( - topicPartition, new PartitionReassignmentState(Arrays.asList(0, 1, 2), Arrays.asList(0, 1, 2), true) + topicPartition, new PartitionReassignmentState(asList(0, 1, 2), asList(0, 1, 2), true) ), false, Collections.singletonMap( new TopicPartitionReplica(topicPartition.topic(), topicPartition.partition(), 0), new ActiveMoveState(reassignment.currentDir, reassignment.targetDir, reassignment.targetDir) @@ -572,14 +567,14 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { cluster.adminClient.incrementalAlterConfigs(Collections.singletonMap( new ConfigResource(ConfigResource.Type.BROKER, "0"), Collections.singletonList(new AlterConfigOp( - new ConfigEntry(brokerLevelLogDirThrottle(), ""), AlterConfigOp.OpType.DELETE)))) + new ConfigEntry(BROKER_LEVEL_LOG_DIR_THROTTLE, ""), AlterConfigOp.OpType.DELETE)))) .all().get(); waitForBrokerLevelThrottles(unthrottledBrokerConfigs); // Wait for the directory movement to complete. waitForVerifyAssignment(cluster.adminClient, reassignment.json, true, new VerifyAssignmentResult(Collections.singletonMap( - topicPartition, new PartitionReassignmentState(Arrays.asList(0, 1, 2), Arrays.asList(0, 1, 2), true) + topicPartition, new PartitionReassignmentState(asList(0, 1, 2), asList(0, 1, 2), true) ), false, Collections.singletonMap( new TopicPartitionReplica(topicPartition.topic(), topicPartition.partition(), 0), new CompletedMoveState(reassignment.targetDir) @@ -599,7 +594,7 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { cluster.produceMessages(topicPartition.topic(), topicPartition.partition(), 700); int targetBrokerId = 0; - List replicas = Arrays.asList(0, 1, 2); + List replicas = asList(0, 1, 2); LogDirReassignment reassignment = buildLogDirReassignment(topicPartition, targetBrokerId, replicas); // Start the replica move with a low throttle so it does not complete @@ -616,7 +611,7 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { waitForVerifyAssignment(cluster.adminClient, reassignment.json, true, new VerifyAssignmentResult(Collections.singletonMap( - topicPartition, new PartitionReassignmentState(Arrays.asList(0, 1, 2), Arrays.asList(0, 1, 2), true) + topicPartition, new PartitionReassignmentState(asList(0, 1, 2), asList(0, 1, 2), true) ), false, Collections.singletonMap( new TopicPartitionReplica(topicPartition.topic(), topicPartition.partition(), targetBrokerId), new CompletedMoveState(reassignment.targetDir) @@ -672,22 +667,25 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { - private kafka.admin.ReassignPartitionsCommand.VerifyAssignmentResult runVerifyAssignment(Admin adminClient, String jsonString, + private VerifyAssignmentResult runVerifyAssignment(Admin adminClient, String jsonString, Boolean preserveThrottles) { System.out.println("==> verifyAssignment(adminClient, jsonString=" + jsonString); - return verifyAssignment(adminClient, jsonString, preserveThrottles); + try { + return verifyAssignment(adminClient, jsonString, preserveThrottles); + } catch (ExecutionException | InterruptedException | JsonProcessingException e) { + throw new RuntimeException(e); + } } private void waitForVerifyAssignment(Admin adminClient, String jsonString, Boolean preserveThrottles, VerifyAssignmentResult expectedResult) { - final kafka.admin.ReassignPartitionsCommand.VerifyAssignmentResult expectedResult0 = asScala(expectedResult); - final kafka.admin.ReassignPartitionsCommand.VerifyAssignmentResult[] latestResult = {null}; + final VerifyAssignmentResult[] latestResult = {null}; TestUtils.waitUntilTrue( () -> { latestResult[0] = runVerifyAssignment(adminClient, jsonString, preserveThrottles); - return expectedResult0.equals(latestResult[0]); + return expectedResult.equals(latestResult[0]); }, () -> "Timed out waiting for verifyAssignment result " + expectedResult + ". " + "The latest result was " + latestResult[0], DEFAULT_MAX_WAIT_MS, 10L); } @@ -696,22 +694,30 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { Boolean additional, String reassignmentJson, Long interBrokerThrottle, - Long replicaAlterLogDirsThrottle) { + Long replicaAlterLogDirsThrottle) throws RuntimeException { System.out.println("==> executeAssignment(adminClient, additional=" + additional + ", " + "reassignmentJson=" + reassignmentJson + ", " + "interBrokerThrottle=" + interBrokerThrottle + ", " + "replicaAlterLogDirsThrottle=" + replicaAlterLogDirsThrottle + "))"); - executeAssignment(adminClient, additional, reassignmentJson, - interBrokerThrottle, replicaAlterLogDirsThrottle, 10000L, Time.SYSTEM); + try { + executeAssignment(adminClient, additional, reassignmentJson, + interBrokerThrottle, replicaAlterLogDirsThrottle, 10000L, Time.SYSTEM); + } catch (ExecutionException | InterruptedException | JsonProcessingException | TerseException e) { + throw new RuntimeException(e); + } } - private Tuple2, scala.collection.immutable.Set> runCancelAssignment( + private Tuple2, Set> runCancelAssignment( Admin adminClient, String jsonString, Boolean preserveThrottles ) { System.out.println("==> cancelAssignment(adminClient, jsonString=" + jsonString); - return cancelAssignment(adminClient, jsonString, preserveThrottles, 10000L, Time.SYSTEM); + try { + return cancelAssignment(adminClient, jsonString, preserveThrottles, 10000L, Time.SYSTEM); + } catch (ExecutionException | InterruptedException | JsonProcessingException | TerseException e) { + throw new RuntimeException(e); + } } static class BrokerDirs { @@ -751,9 +757,9 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { } private final Map>> topics = new HashMap<>(); { - topics.put("foo", Arrays.asList(Arrays.asList(0, 1, 2), Arrays.asList(1, 2, 3))); - topics.put("bar", Arrays.asList(Arrays.asList(3, 2, 1))); - topics.put("baz", Arrays.asList(Arrays.asList(1, 0, 2), Arrays.asList(2, 0, 1), Arrays.asList(0, 2, 1))); + topics.put("foo", asList(asList(0, 1, 2), asList(1, 2, 3))); + topics.put("bar", asList(asList(3, 2, 1))); + topics.put("baz", asList(asList(1, 0, 2), asList(2, 0, 1), asList(0, 2, 1))); } private final List servers = new ArrayList<>(); @@ -770,9 +776,9 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { false, // shorten test time true, TestUtils.RandomPort(), - scala.None$.empty(), - scala.None$.empty(), - scala.None$.empty(), + Option.empty(), + Option.empty(), + Option.empty(), true, false, TestUtils.RandomPort(), @@ -805,7 +811,7 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { public void createServers() { brokers.keySet().forEach(brokerId -> - servers.add(createBroker(brokerConfigs.get(brokerId), Time.SYSTEM, true, scala.None$.empty())) + servers.add(createBroker(brokerConfigs.get(brokerId), Time.SYSTEM, true, Option.empty())) ); } @@ -857,68 +863,13 @@ public class ReassignPartitionsIntegrationTest extends QuorumTestHarness { } } - private ReassignPartitionsCommand.VerifyAssignmentResult asScala(VerifyAssignmentResult res) { - Map partStates = new HashMap<>(); - res.partStates.forEach((tp, state) -> partStates.put(tp, asScala(state))); - - Map moveStates = new HashMap<>(); - res.moveStates.forEach((tpr, state) -> moveStates.put(tpr, asScala(state))); - - return new ReassignPartitionsCommand.VerifyAssignmentResult(asScala(partStates), res.partsOngoing, asScala(moveStates), res.movesOngoing); - } - - @SuppressWarnings({"unchecked"}) - private ReassignPartitionsCommand.PartitionReassignmentState asScala(PartitionReassignmentState state) { - return new ReassignPartitionsCommand.PartitionReassignmentState( - seq((List) state.currentReplicas), - seq((List) state.targetReplicas), - state.done - ); - } - - private ReassignPartitionsCommand.LogDirMoveState asScala(LogDirMoveState state) { - if (state instanceof ActiveMoveState) { - ActiveMoveState s = (ActiveMoveState) state; - return new ReassignPartitionsCommand.ActiveMoveState(s.currentLogDir, s.targetLogDir, s.futureLogDir); - } else if (state instanceof CancelledMoveState) { - CancelledMoveState s = (CancelledMoveState) state; - return new ReassignPartitionsCommand.CancelledMoveState(s.currentLogDir, s.targetLogDir); - } else if (state instanceof CompletedMoveState) { - CompletedMoveState s = (CompletedMoveState) state; - return new ReassignPartitionsCommand.CompletedMoveState(s.targetLogDir); - } else if (state instanceof MissingLogDirMoveState) { - MissingLogDirMoveState s = (MissingLogDirMoveState) state; - return new ReassignPartitionsCommand.MissingLogDirMoveState(s.targetLogDir); - } else if (state instanceof MissingReplicaMoveState) { - MissingReplicaMoveState s = (MissingReplicaMoveState) state; - return new ReassignPartitionsCommand.MissingReplicaMoveState(s.targetLogDir); - } - - throw new IllegalArgumentException("Unknown state " + state); - } - - @SuppressWarnings("unchecked") - static scala.collection.immutable.Set set(final T... set) { - return mutableSet(set).toSet(); - } - @SuppressWarnings({"deprecation", "unchecked"}) private static scala.collection.mutable.Set mutableSet(final T...set) { - return JavaConverters.asScalaSet(new HashSet<>(Arrays.asList(set))); - } - - @SuppressWarnings({"unchecked"}) - private static Seq seq(T... seq) { - return seq(Arrays.asList(seq)); + return JavaConverters.asScalaSet(new HashSet<>(asList(set))); } @SuppressWarnings({"deprecation"}) private static Seq seq(Collection seq) { return JavaConverters.asScalaIteratorConverter(seq.iterator()).asScala().toSeq(); } - - @SuppressWarnings("deprecation") - private static scala.collection.Map asScala(Map jmap) { - return JavaConverters.mapAsScalaMap(jmap); - } } diff --git a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java index 4639744226d..d0e1cb4baaf 100644 --- a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.tools.reassign; -import kafka.admin.ReassignPartitionsCommand; import org.apache.kafka.admin.BrokerMetadata; import org.apache.kafka.clients.admin.Config; import org.apache.kafka.clients.admin.MockAdminClient; @@ -36,12 +35,8 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import scala.Tuple2; -import scala.collection.JavaConverters; -import scala.collection.Seq; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -52,37 +47,38 @@ import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import java.util.stream.Stream; -import static kafka.admin.ReassignPartitionsCommand.alterPartitionReassignments; -import static kafka.admin.ReassignPartitionsCommand.alterReplicaLogDirs; -import static kafka.admin.ReassignPartitionsCommand.brokerLevelFollowerThrottle; -import static kafka.admin.ReassignPartitionsCommand.brokerLevelLeaderThrottle; -import static kafka.admin.ReassignPartitionsCommand.brokerLevelLogDirThrottle; -import static kafka.admin.ReassignPartitionsCommand.calculateFollowerThrottles; -import static kafka.admin.ReassignPartitionsCommand.calculateLeaderThrottles; -import static kafka.admin.ReassignPartitionsCommand.calculateMovingBrokers; -import static kafka.admin.ReassignPartitionsCommand.calculateProposedMoveMap; -import static kafka.admin.ReassignPartitionsCommand.calculateReassigningBrokers; -import static kafka.admin.ReassignPartitionsCommand.cancelPartitionReassignments; -import static kafka.admin.ReassignPartitionsCommand.compareTopicPartitionReplicas; -import static kafka.admin.ReassignPartitionsCommand.compareTopicPartitions; -import static kafka.admin.ReassignPartitionsCommand.curReassignmentsToString; -import static kafka.admin.ReassignPartitionsCommand.currentPartitionReplicaAssignmentToString; -import static kafka.admin.ReassignPartitionsCommand.executeAssignment; -import static kafka.admin.ReassignPartitionsCommand.findLogDirMoveStates; -import static kafka.admin.ReassignPartitionsCommand.findPartitionReassignmentStates; -import static kafka.admin.ReassignPartitionsCommand.generateAssignment; -import static kafka.admin.ReassignPartitionsCommand.getBrokerMetadata; -import static kafka.admin.ReassignPartitionsCommand.getReplicaAssignmentForPartitions; -import static kafka.admin.ReassignPartitionsCommand.getReplicaAssignmentForTopics; -import static kafka.admin.ReassignPartitionsCommand.modifyInterBrokerThrottle; -import static kafka.admin.ReassignPartitionsCommand.modifyLogDirThrottle; -import static kafka.admin.ReassignPartitionsCommand.modifyTopicThrottles; -import static kafka.admin.ReassignPartitionsCommand.parseExecuteAssignmentArgs; -import static kafka.admin.ReassignPartitionsCommand.parseGenerateAssignmentArgs; -import static kafka.admin.ReassignPartitionsCommand.partitionReassignmentStatesToString; -import static kafka.admin.ReassignPartitionsCommand.replicaMoveStatesToString; -import static kafka.admin.ReassignPartitionsCommand.topicLevelFollowerThrottle; -import static kafka.admin.ReassignPartitionsCommand.topicLevelLeaderThrottle; +import static java.util.Arrays.asList; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.BROKER_LEVEL_FOLLOWER_THROTTLE; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.BROKER_LEVEL_LEADER_THROTTLE; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.BROKER_LEVEL_LOG_DIR_THROTTLE; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.TOPIC_LEVEL_FOLLOWER_THROTTLE; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.TOPIC_LEVEL_LEADER_THROTTLE; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.alterPartitionReassignments; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.alterReplicaLogDirs; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.calculateFollowerThrottles; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.calculateLeaderThrottles; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.calculateMovingBrokers; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.calculateProposedMoveMap; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.calculateReassigningBrokers; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.cancelPartitionReassignments; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.compareTopicPartitionReplicas; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.compareTopicPartitions; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.curReassignmentsToString; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.currentPartitionReplicaAssignmentToString; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.executeAssignment; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.findLogDirMoveStates; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.findPartitionReassignmentStates; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.generateAssignment; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.getBrokerMetadata; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.getReplicaAssignmentForPartitions; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.getReplicaAssignmentForTopics; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.modifyInterBrokerThrottle; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.modifyLogDirThrottle; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.modifyTopicThrottles; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.parseExecuteAssignmentArgs; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.parseGenerateAssignmentArgs; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.partitionReassignmentStatesToString; +import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.replicaMoveStatesToString; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -106,109 +102,107 @@ public class ReassignPartitionsUnitTest { @Test public void testCompareTopicPartitions() { assertTrue(compareTopicPartitions(new TopicPartition("abc", 0), - new TopicPartition("abc", 1))); + new TopicPartition("abc", 1)) < 0); assertFalse(compareTopicPartitions(new TopicPartition("def", 0), - new TopicPartition("abc", 1))); + new TopicPartition("abc", 1)) < 0); } @Test public void testCompareTopicPartitionReplicas() { assertTrue(compareTopicPartitionReplicas(new TopicPartitionReplica("def", 0, 0), - new TopicPartitionReplica("abc", 0, 1))); + new TopicPartitionReplica("abc", 0, 1)) < 0); assertFalse(compareTopicPartitionReplicas(new TopicPartitionReplica("def", 0, 0), - new TopicPartitionReplica("cde", 0, 0))); + new TopicPartitionReplica("cde", 0, 0)) < 0); } @Test public void testPartitionReassignStatesToString() { - Map states = new HashMap<>(); + Map states = new HashMap<>(); states.put(new TopicPartition("foo", 0), - new ReassignPartitionsCommand.PartitionReassignmentState(seq(1, 2, 3), seq(1, 2, 3), true)); + new PartitionReassignmentState(asList(1, 2, 3), asList(1, 2, 3), true)); states.put(new TopicPartition("foo", 1), - new ReassignPartitionsCommand.PartitionReassignmentState(seq(1, 2, 3), seq(1, 2, 4), false)); + new PartitionReassignmentState(asList(1, 2, 3), asList(1, 2, 4), false)); states.put(new TopicPartition("bar", 0), - new ReassignPartitionsCommand.PartitionReassignmentState(seq(1, 2, 3), seq(1, 2, 4), false)); + new PartitionReassignmentState(asList(1, 2, 3), asList(1, 2, 4), false)); assertEquals(String.join(System.lineSeparator(), "Status of partition reassignment:", "Reassignment of partition bar-0 is still in progress.", "Reassignment of partition foo-0 is completed.", "Reassignment of partition foo-1 is still in progress."), - partitionReassignmentStatesToString(asScala(states))); + partitionReassignmentStatesToString(states)); } private void addTopics(MockAdminClient adminClient) { List b = adminClient.brokers(); - adminClient.addTopic(false, "foo", Arrays.asList( + adminClient.addTopic(false, "foo", asList( new TopicPartitionInfo(0, b.get(0), - Arrays.asList(b.get(0), b.get(1), b.get(2)), - Arrays.asList(b.get(0), b.get(1))), + asList(b.get(0), b.get(1), b.get(2)), + asList(b.get(0), b.get(1))), new TopicPartitionInfo(1, b.get(1), - Arrays.asList(b.get(1), b.get(2), b.get(3)), - Arrays.asList(b.get(1), b.get(2), b.get(3))) + asList(b.get(1), b.get(2), b.get(3)), + asList(b.get(1), b.get(2), b.get(3))) ), Collections.emptyMap()); - adminClient.addTopic(false, "bar", Arrays.asList( + adminClient.addTopic(false, "bar", asList( new TopicPartitionInfo(0, b.get(2), - Arrays.asList(b.get(2), b.get(3), b.get(0)), - Arrays.asList(b.get(2), b.get(3), b.get(0))) + asList(b.get(2), b.get(3), b.get(0)), + asList(b.get(2), b.get(3), b.get(0))) ), Collections.emptyMap()); } - @SuppressWarnings("unchecked") @Test - public void testFindPartitionReassignmentStates() { + public void testFindPartitionReassignmentStates() throws Exception { try (MockAdminClient adminClient = new MockAdminClient.Builder().numBrokers(4).build()) { addTopics(adminClient); // Create a reassignment and test findPartitionReassignmentStates. - Map> reassignments = new HashMap<>(); + Map> reassignments = new HashMap<>(); - reassignments.put(new TopicPartition("foo", 0), seq(0, 1, 3)); - reassignments.put(new TopicPartition("quux", 0), seq(1, 2, 3)); + reassignments.put(new TopicPartition("foo", 0), asList(0, 1, 3)); + reassignments.put(new TopicPartition("quux", 0), asList(1, 2, 3)); - scala.collection.Map reassignmentResult = alterPartitionReassignments(adminClient, - asScala(reassignments)); + Map reassignmentResult = alterPartitionReassignments(adminClient, reassignments); assertEquals(1, reassignmentResult.size()); - assertEquals(UnknownTopicOrPartitionException.class, reassignmentResult.get(new TopicPartition("quux", 0)).get().getClass()); + assertEquals(UnknownTopicOrPartitionException.class, reassignmentResult.get(new TopicPartition("quux", 0)).getClass()); - Map expStates = new HashMap<>(); + Map expStates = new HashMap<>(); expStates.put(new TopicPartition("foo", 0), - new ReassignPartitionsCommand.PartitionReassignmentState(seq(0, 1, 2), seq(0, 1, 3), false)); + new PartitionReassignmentState(asList(0, 1, 2), asList(0, 1, 3), false)); expStates.put(new TopicPartition("foo", 1), - new ReassignPartitionsCommand.PartitionReassignmentState(seq(1, 2, 3), seq(1, 2, 3), true)); + new PartitionReassignmentState(asList(1, 2, 3), asList(1, 2, 3), true)); - Tuple2, Object> actual = - findPartitionReassignmentStates(adminClient, seq( - new Tuple2<>(new TopicPartition("foo", 0), seq(0, 1, 3)), - new Tuple2<>(new TopicPartition("foo", 1), seq(1, 2, 3)) + Tuple2, Boolean> actual = + findPartitionReassignmentStates(adminClient, asList( + new Tuple2<>(new TopicPartition("foo", 0), asList(0, 1, 3)), + new Tuple2<>(new TopicPartition("foo", 1), asList(1, 2, 3)) )); - assertEquals(asScala(expStates), actual._1); - assertTrue((Boolean) actual._2); + assertEquals(expStates, actual.v1); + assertTrue(actual.v2); // Cancel the reassignment and test findPartitionReassignmentStates again. - scala.collection.Map cancelResult = cancelPartitionReassignments(adminClient, - set(new TopicPartition("foo", 0), new TopicPartition("quux", 2))); + Map cancelResult = cancelPartitionReassignments(adminClient, + new HashSet<>(asList(new TopicPartition("foo", 0), new TopicPartition("quux", 2)))); assertEquals(1, cancelResult.size()); - assertEquals(UnknownTopicOrPartitionException.class, cancelResult.get(new TopicPartition("quux", 2)).get().getClass()); + assertEquals(UnknownTopicOrPartitionException.class, cancelResult.get(new TopicPartition("quux", 2)).getClass()); expStates.clear(); expStates.put(new TopicPartition("foo", 0), - new ReassignPartitionsCommand.PartitionReassignmentState(seq(0, 1, 2), seq(0, 1, 3), true)); + new PartitionReassignmentState(asList(0, 1, 2), asList(0, 1, 3), true)); expStates.put(new TopicPartition("foo", 1), - new ReassignPartitionsCommand.PartitionReassignmentState(seq(1, 2, 3), seq(1, 2, 3), true)); + new PartitionReassignmentState(asList(1, 2, 3), asList(1, 2, 3), true)); - actual = findPartitionReassignmentStates(adminClient, seq( - new Tuple2<>(new TopicPartition("foo", 0), seq(0, 1, 3)), - new Tuple2<>(new TopicPartition("foo", 1), seq(1, 2, 3)) + actual = findPartitionReassignmentStates(adminClient, asList( + new Tuple2<>(new TopicPartition("foo", 0), asList(0, 1, 3)), + new Tuple2<>(new TopicPartition("foo", 1), asList(1, 2, 3)) )); - assertEquals(asScala(expStates), actual._1); - assertFalse((Boolean) actual._2); + assertEquals(expStates, actual.v1); + assertFalse(actual.v2); } } @@ -216,19 +210,19 @@ public class ReassignPartitionsUnitTest { public void testFindLogDirMoveStates() throws Exception { try (MockAdminClient adminClient = new MockAdminClient.Builder(). numBrokers(4). - brokerLogDirs(Arrays.asList( - Arrays.asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"), - Arrays.asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"), - Arrays.asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"), - Arrays.asList("/tmp/kafka-logs0", null))) + brokerLogDirs(asList( + asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"), + asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"), + asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"), + asList("/tmp/kafka-logs0", null))) .build()) { addTopics(adminClient); List b = adminClient.brokers(); - adminClient.addTopic(false, "quux", Arrays.asList( + adminClient.addTopic(false, "quux", asList( new TopicPartitionInfo(0, b.get(2), - Arrays.asList(b.get(1), b.get(2), b.get(3)), - Arrays.asList(b.get(1), b.get(2), b.get(3)))), + asList(b.get(1), b.get(2), b.get(3)), + asList(b.get(1), b.get(2), b.get(3)))), Collections.emptyMap()); Map replicaAssignment = new HashMap<>(); @@ -238,15 +232,15 @@ public class ReassignPartitionsUnitTest { adminClient.alterReplicaLogDirs(replicaAssignment).all().get(); - Map states = new HashMap<>(); + Map states = new HashMap<>(); - states.put(new TopicPartitionReplica("bar", 0, 0), new ReassignPartitionsCommand.CompletedMoveState("/tmp/kafka-logs0")); - states.put(new TopicPartitionReplica("foo", 0, 0), new ReassignPartitionsCommand.ActiveMoveState("/tmp/kafka-logs0", + states.put(new TopicPartitionReplica("bar", 0, 0), new CompletedMoveState("/tmp/kafka-logs0")); + states.put(new TopicPartitionReplica("foo", 0, 0), new ActiveMoveState("/tmp/kafka-logs0", "/tmp/kafka-logs1", "/tmp/kafka-logs1")); - states.put(new TopicPartitionReplica("foo", 1, 0), new ReassignPartitionsCommand.CancelledMoveState("/tmp/kafka-logs0", + states.put(new TopicPartitionReplica("foo", 1, 0), new CancelledMoveState("/tmp/kafka-logs0", "/tmp/kafka-logs1")); - states.put(new TopicPartitionReplica("quux", 1, 0), new ReassignPartitionsCommand.MissingLogDirMoveState("/tmp/kafka-logs1")); - states.put(new TopicPartitionReplica("quuz", 0, 0), new ReassignPartitionsCommand.MissingReplicaMoveState("/tmp/kafka-logs0")); + states.put(new TopicPartitionReplica("quux", 1, 0), new MissingLogDirMoveState("/tmp/kafka-logs1")); + states.put(new TopicPartitionReplica("quuz", 0, 0), new MissingReplicaMoveState("/tmp/kafka-logs0")); Map targetMoves = new HashMap<>(); @@ -256,23 +250,23 @@ public class ReassignPartitionsUnitTest { targetMoves.put(new TopicPartitionReplica("quux", 1, 0), "/tmp/kafka-logs1"); targetMoves.put(new TopicPartitionReplica("quuz", 0, 0), "/tmp/kafka-logs0"); - assertEquals(asScala(states), findLogDirMoveStates(adminClient, asScala(targetMoves))); + assertEquals(states, findLogDirMoveStates(adminClient, targetMoves)); } } @Test public void testReplicaMoveStatesToString() { - Map states = new HashMap<>(); + Map states = new HashMap<>(); - states.put(new TopicPartitionReplica("bar", 0, 0), new ReassignPartitionsCommand.CompletedMoveState("/tmp/kafka-logs0")); - states.put(new TopicPartitionReplica("foo", 0, 0), new ReassignPartitionsCommand.ActiveMoveState("/tmp/kafka-logs0", + states.put(new TopicPartitionReplica("bar", 0, 0), new CompletedMoveState("/tmp/kafka-logs0")); + states.put(new TopicPartitionReplica("foo", 0, 0), new ActiveMoveState("/tmp/kafka-logs0", "/tmp/kafka-logs1", "/tmp/kafka-logs1")); - states.put(new TopicPartitionReplica("foo", 1, 0), new ReassignPartitionsCommand.CancelledMoveState("/tmp/kafka-logs0", + states.put(new TopicPartitionReplica("foo", 1, 0), new CancelledMoveState("/tmp/kafka-logs0", "/tmp/kafka-logs1")); - states.put(new TopicPartitionReplica("quux", 0, 0), new ReassignPartitionsCommand.MissingReplicaMoveState("/tmp/kafka-logs1")); - states.put(new TopicPartitionReplica("quux", 1, 1), new ReassignPartitionsCommand.ActiveMoveState("/tmp/kafka-logs0", + states.put(new TopicPartitionReplica("quux", 0, 0), new MissingReplicaMoveState("/tmp/kafka-logs1")); + states.put(new TopicPartitionReplica("quux", 1, 1), new ActiveMoveState("/tmp/kafka-logs0", "/tmp/kafka-logs1", "/tmp/kafka-logs2")); - states.put(new TopicPartitionReplica("quux", 2, 1), new ReassignPartitionsCommand.MissingLogDirMoveState("/tmp/kafka-logs1")); + states.put(new TopicPartitionReplica("quux", 2, 1), new MissingLogDirMoveState("/tmp/kafka-logs1")); assertEquals(String.join(System.lineSeparator(), "Reassignment of replica bar-0-0 completed successfully.", @@ -282,59 +276,59 @@ public class ReassignPartitionsUnitTest { "Partition quux-1 on broker 1 is being moved to log dir /tmp/kafka-logs2 instead of /tmp/kafka-logs1.", "Partition quux-2 is not found in any live log dir on broker 1. " + "There is likely an offline log directory on the broker."), - replicaMoveStatesToString(asScala(states))); + replicaMoveStatesToString(states)); } @Test - public void testGetReplicaAssignments() { + public void testGetReplicaAssignments() throws Exception { try (MockAdminClient adminClient = new MockAdminClient.Builder().numBrokers(4).build()) { addTopics(adminClient); - Map> assignments = new HashMap<>(); + Map> assignments = new HashMap<>(); - assignments.put(new TopicPartition("foo", 0), seq(0, 1, 2)); - assignments.put(new TopicPartition("foo", 1), seq(1, 2, 3)); + assignments.put(new TopicPartition("foo", 0), asList(0, 1, 2)); + assignments.put(new TopicPartition("foo", 1), asList(1, 2, 3)); - assertEquals(asScala(assignments), getReplicaAssignmentForTopics(adminClient, seq("foo"))); + assertEquals(assignments, getReplicaAssignmentForTopics(adminClient, asList("foo"))); assignments.clear(); - assignments.put(new TopicPartition("foo", 0), seq(0, 1, 2)); - assignments.put(new TopicPartition("bar", 0), seq(2, 3, 0)); + assignments.put(new TopicPartition("foo", 0), asList(0, 1, 2)); + assignments.put(new TopicPartition("bar", 0), asList(2, 3, 0)); - assertEquals(asScala(assignments), - getReplicaAssignmentForPartitions(adminClient, set(new TopicPartition("foo", 0), new TopicPartition("bar", 0)))); + assertEquals(assignments, + getReplicaAssignmentForPartitions(adminClient, new HashSet<>(asList(new TopicPartition("foo", 0), new TopicPartition("bar", 0))))); } } @Test - public void testGetBrokerRackInformation() { + public void testGetBrokerRackInformation() throws Exception { try (MockAdminClient adminClient = new MockAdminClient.Builder(). - brokers(Arrays.asList(new Node(0, "localhost", 9092, "rack0"), + brokers(asList(new Node(0, "localhost", 9092, "rack0"), new Node(1, "localhost", 9093, "rack1"), new Node(2, "localhost", 9094, null))). build()) { - assertEquals(seq( + assertEquals(asList( new BrokerMetadata(0, Optional.of("rack0")), new BrokerMetadata(1, Optional.of("rack1")) - ), getBrokerMetadata(adminClient, seq(0, 1), true)); - assertEquals(seq( + ), getBrokerMetadata(adminClient, asList(0, 1), true)); + assertEquals(asList( new BrokerMetadata(0, Optional.empty()), new BrokerMetadata(1, Optional.empty()) - ), getBrokerMetadata(adminClient, seq(0, 1), false)); + ), getBrokerMetadata(adminClient, asList(0, 1), false)); assertStartsWith("Not all brokers have rack information", assertThrows(AdminOperationException.class, - () -> getBrokerMetadata(adminClient, seq(1, 2), true)).getMessage()); - assertEquals(seq( + () -> getBrokerMetadata(adminClient, asList(1, 2), true)).getMessage()); + assertEquals(asList( new BrokerMetadata(1, Optional.empty()), new BrokerMetadata(2, Optional.empty()) - ), getBrokerMetadata(adminClient, seq(1, 2), false)); + ), getBrokerMetadata(adminClient, asList(1, 2), false)); } } @Test - public void testParseGenerateAssignmentArgs() { + public void testParseGenerateAssignmentArgs() throws Exception { assertStartsWith("Broker list contains duplicate entries", assertThrows(AdminCommandFailedException.class, () -> parseGenerateAssignmentArgs( "{\"topics\": [{\"topic\": \"foo\"}], \"version\":1}", "1,1,2"), @@ -343,13 +337,13 @@ public class ReassignPartitionsUnitTest { assertThrows(AdminCommandFailedException.class, () -> parseGenerateAssignmentArgs( "{\"topics\": [{\"topic\": \"foo\"}], \"version\":1}", "5,2,3,4,5"), "Expected to detect duplicate broker list entries").getMessage()); - assertEquals(new Tuple2<>(seq(5, 2, 3, 4), seq("foo")), + assertEquals(new Tuple2<>(asList(5, 2, 3, 4), asList("foo")), parseGenerateAssignmentArgs("{\"topics\": [{\"topic\": \"foo\"}], \"version\":1}", "5,2,3,4")); assertStartsWith("List of topics to reassign contains duplicate entries", assertThrows(AdminCommandFailedException.class, () -> parseGenerateAssignmentArgs( "{\"topics\": [{\"topic\": \"foo\"},{\"topic\": \"foo\"}], \"version\":1}", "5,2,3,4"), "Expected to detect duplicate topic entries").getMessage()); - assertEquals(new Tuple2<>(seq(5, 3, 4), seq("foo", "bar")), + assertEquals(new Tuple2<>(asList(5, 3, 4), asList("foo", "bar")), parseGenerateAssignmentArgs( "{\"topics\": [{\"topic\": \"foo\"},{\"topic\": \"bar\"}], \"version\":1}", "5,3,4")); } @@ -377,9 +371,9 @@ public class ReassignPartitionsUnitTest { } @Test - public void testGenerateAssignmentWithInconsistentRacks() { + public void testGenerateAssignmentWithInconsistentRacks() throws Exception { try (MockAdminClient adminClient = new MockAdminClient.Builder(). - brokers(Arrays.asList( + brokers(asList( new Node(0, "localhost", 9092, "rack0"), new Node(1, "localhost", 9093, "rack0"), new Node(2, "localhost", 9094, null), @@ -394,59 +388,58 @@ public class ReassignPartitionsUnitTest { () -> generateAssignment(adminClient, "{\"topics\":[{\"topic\":\"foo\"}]}", "0,1,2,3", true), "Expected generateAssignment to fail").getMessage()); // It should succeed when --disable-rack-aware is used. - Tuple2>, scala.collection.Map>> + Tuple2>, Map>> proposedCurrent = generateAssignment(adminClient, "{\"topics\":[{\"topic\":\"foo\"}]}", "0,1,2,3", false); - Map> expCurrent = new HashMap<>(); + Map> expCurrent = new HashMap<>(); - expCurrent.put(new TopicPartition("foo", 0), seq(0, 1, 2)); - expCurrent.put(new TopicPartition("foo", 1), seq(1, 2, 3)); + expCurrent.put(new TopicPartition("foo", 0), asList(0, 1, 2)); + expCurrent.put(new TopicPartition("foo", 1), asList(1, 2, 3)); - assertEquals(asScala(expCurrent), proposedCurrent._2()); + assertEquals(expCurrent, proposedCurrent.v2); } } @Test - public void testGenerateAssignmentWithFewerBrokers() { + public void testGenerateAssignmentWithFewerBrokers() throws Exception { try (MockAdminClient adminClient = new MockAdminClient.Builder().numBrokers(4).build()) { addTopics(adminClient); - List goalBrokers = Arrays.asList(0, 1, 3); + List goalBrokers = asList(0, 1, 3); - Tuple2>, scala.collection.Map>> + Tuple2>, Map>> proposedCurrent = generateAssignment(adminClient, "{\"topics\":[{\"topic\":\"foo\"},{\"topic\":\"bar\"}]}", goalBrokers.stream().map(Object::toString).collect(Collectors.joining(",")), false); - Map> expCurrent = new HashMap<>(); + Map> expCurrent = new HashMap<>(); - expCurrent.put(new TopicPartition("foo", 0), seq(0, 1, 2)); - expCurrent.put(new TopicPartition("foo", 1), seq(1, 2, 3)); - expCurrent.put(new TopicPartition("bar", 0), seq(2, 3, 0)); + expCurrent.put(new TopicPartition("foo", 0), asList(0, 1, 2)); + expCurrent.put(new TopicPartition("foo", 1), asList(1, 2, 3)); + expCurrent.put(new TopicPartition("bar", 0), asList(2, 3, 0)); - assertEquals(asScala(expCurrent), proposedCurrent._2()); + assertEquals(expCurrent, proposedCurrent.v2); // The proposed assignment should only span the provided brokers - proposedCurrent._1().values().foreach(replicas -> { - assertTrue(replicas.forall(replica -> goalBrokers.contains((Integer) replica)), - "Proposed assignment " + proposedCurrent._1() + " puts replicas on brokers other than " + goalBrokers); - return null; - }); + proposedCurrent.v1.values().forEach(replicas -> + assertTrue(goalBrokers.containsAll(replicas), + "Proposed assignment " + proposedCurrent.v1 + " puts replicas on brokers other than " + goalBrokers) + ); } } @Test - public void testCurrentPartitionReplicaAssignmentToString() { - Map> proposedParts = new HashMap<>(); + public void testCurrentPartitionReplicaAssignmentToString() throws Exception { + Map> proposedParts = new HashMap<>(); - proposedParts.put(new TopicPartition("foo", 1), seq(1, 2, 3)); - proposedParts.put(new TopicPartition("bar", 0), seq(7, 8, 9)); + proposedParts.put(new TopicPartition("foo", 1), asList(1, 2, 3)); + proposedParts.put(new TopicPartition("bar", 0), asList(7, 8, 9)); - Map> currentParts = new HashMap<>(); + Map> currentParts = new HashMap<>(); - currentParts.put(new TopicPartition("foo", 0), seq(1, 2, 3)); - currentParts.put(new TopicPartition("foo", 1), seq(4, 5, 6)); - currentParts.put(new TopicPartition("bar", 0), seq(7, 8)); - currentParts.put(new TopicPartition("baz", 0), seq(10, 11, 12)); + currentParts.put(new TopicPartition("foo", 0), asList(1, 2, 3)); + currentParts.put(new TopicPartition("foo", 1), asList(4, 5, 6)); + currentParts.put(new TopicPartition("bar", 0), asList(7, 8)); + currentParts.put(new TopicPartition("baz", 0), asList(10, 11, 12)); assertEquals(String.join(System.lineSeparator(), "Current partition replica assignment", @@ -457,7 +450,7 @@ public class ReassignPartitionsUnitTest { "}", "", "Save this to use as the --reassignment-json-file option during rollback"), - currentPartitionReplicaAssignmentToString(asScala(proposedParts), asScala(currentParts)) + currentPartitionReplicaAssignmentToString(proposedParts, currentParts) ); } @@ -473,79 +466,78 @@ public class ReassignPartitionsUnitTest { Map currentReassignments = new HashMap<>(); currentReassignments.put(new TopicPartition("foo", 0), new PartitionReassignment( - Arrays.asList(1, 2, 3, 4), Arrays.asList(4), Arrays.asList(3))); + asList(1, 2, 3, 4), asList(4), asList(3))); currentReassignments.put(new TopicPartition("foo", 1), new PartitionReassignment( - Arrays.asList(4, 5, 6, 7, 8), Arrays.asList(7, 8), Arrays.asList(4, 5))); + asList(4, 5, 6, 7, 8), asList(7, 8), asList(4, 5))); currentReassignments.put(new TopicPartition("foo", 2), new PartitionReassignment( - Arrays.asList(1, 2, 3, 4), Arrays.asList(3, 4), Arrays.asList(1, 2))); + asList(1, 2, 3, 4), asList(3, 4), asList(1, 2))); currentReassignments.put(new TopicPartition("foo", 3), new PartitionReassignment( - Arrays.asList(1, 2, 3, 4), Arrays.asList(3, 4), Arrays.asList(1, 2))); + asList(1, 2, 3, 4), asList(3, 4), asList(1, 2))); currentReassignments.put(new TopicPartition("foo", 4), new PartitionReassignment( - Arrays.asList(1, 2, 3, 4), Arrays.asList(3, 4), Arrays.asList(1, 2))); + asList(1, 2, 3, 4), asList(3, 4), asList(1, 2))); currentReassignments.put(new TopicPartition("foo", 5), new PartitionReassignment( - Arrays.asList(1, 2, 3, 4), Arrays.asList(3, 4), Arrays.asList(1, 2))); + asList(1, 2, 3, 4), asList(3, 4), asList(1, 2))); - Map> proposedParts = new HashMap<>(); + Map> proposedParts = new HashMap<>(); - proposedParts.put(new TopicPartition("foo", 0), seq(1, 2, 5)); - proposedParts.put(new TopicPartition("foo", 2), seq(3, 4)); - proposedParts.put(new TopicPartition("foo", 3), seq(5, 6)); - proposedParts.put(new TopicPartition("foo", 4), seq(3)); - proposedParts.put(new TopicPartition("foo", 5), seq(3, 4, 5, 6)); - proposedParts.put(new TopicPartition("bar", 0), seq(1, 2, 3)); + proposedParts.put(new TopicPartition("foo", 0), asList(1, 2, 5)); + proposedParts.put(new TopicPartition("foo", 2), asList(3, 4)); + proposedParts.put(new TopicPartition("foo", 3), asList(5, 6)); + proposedParts.put(new TopicPartition("foo", 4), asList(3)); + proposedParts.put(new TopicPartition("foo", 5), asList(3, 4, 5, 6)); + proposedParts.put(new TopicPartition("bar", 0), asList(1, 2, 3)); - Map> currentParts = new HashMap<>(); + Map> currentParts = new HashMap<>(); - currentParts.put(new TopicPartition("foo", 0), seq(1, 2, 3, 4)); - currentParts.put(new TopicPartition("foo", 1), seq(4, 5, 6, 7, 8)); - currentParts.put(new TopicPartition("foo", 2), seq(1, 2, 3, 4)); - currentParts.put(new TopicPartition("foo", 3), seq(1, 2, 3, 4)); - currentParts.put(new TopicPartition("foo", 4), seq(1, 2, 3, 4)); - currentParts.put(new TopicPartition("foo", 5), seq(1, 2, 3, 4)); - currentParts.put(new TopicPartition("bar", 0), seq(2, 3, 4)); - currentParts.put(new TopicPartition("baz", 0), seq(1, 2, 3)); + currentParts.put(new TopicPartition("foo", 0), asList(1, 2, 3, 4)); + currentParts.put(new TopicPartition("foo", 1), asList(4, 5, 6, 7, 8)); + currentParts.put(new TopicPartition("foo", 2), asList(1, 2, 3, 4)); + currentParts.put(new TopicPartition("foo", 3), asList(1, 2, 3, 4)); + currentParts.put(new TopicPartition("foo", 4), asList(1, 2, 3, 4)); + currentParts.put(new TopicPartition("foo", 5), asList(1, 2, 3, 4)); + currentParts.put(new TopicPartition("bar", 0), asList(2, 3, 4)); + currentParts.put(new TopicPartition("baz", 0), asList(1, 2, 3)); - scala.collection.mutable.Map> - moveMap = calculateProposedMoveMap(asScala(currentReassignments), asScala(proposedParts), asScala(currentParts)); + Map> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts); - Map fooMoves = new HashMap<>(); + Map fooMoves = new HashMap<>(); - fooMoves.put(0, new ReassignPartitionsCommand.PartitionMove(mutableSet(1, 2, 3), mutableSet(5))); - fooMoves.put(1, new ReassignPartitionsCommand.PartitionMove(mutableSet(4, 5, 6), mutableSet(7, 8))); - fooMoves.put(2, new ReassignPartitionsCommand.PartitionMove(mutableSet(1, 2), mutableSet(3, 4))); - fooMoves.put(3, new ReassignPartitionsCommand.PartitionMove(mutableSet(1, 2), mutableSet(5, 6))); - fooMoves.put(4, new ReassignPartitionsCommand.PartitionMove(mutableSet(1, 2), mutableSet(3))); - fooMoves.put(5, new ReassignPartitionsCommand.PartitionMove(mutableSet(1, 2), mutableSet(3, 4, 5, 6))); + fooMoves.put(0, new PartitionMove(new HashSet<>(asList(1, 2, 3)), new HashSet<>(asList(5)))); + fooMoves.put(1, new PartitionMove(new HashSet<>(asList(4, 5, 6)), new HashSet<>(asList(7, 8)))); + fooMoves.put(2, new PartitionMove(new HashSet<>(asList(1, 2)), new HashSet<>(asList(3, 4)))); + fooMoves.put(3, new PartitionMove(new HashSet<>(asList(1, 2)), new HashSet<>(asList(5, 6)))); + fooMoves.put(4, new PartitionMove(new HashSet<>(asList(1, 2)), new HashSet<>(asList(3)))); + fooMoves.put(5, new PartitionMove(new HashSet<>(asList(1, 2)), new HashSet<>(asList(3, 4, 5, 6)))); - Map barMoves = new HashMap<>(); + Map barMoves = new HashMap<>(); - barMoves.put(0, new ReassignPartitionsCommand.PartitionMove(mutableSet(2, 3, 4), mutableSet(1))); + barMoves.put(0, new PartitionMove(new HashSet<>(asList(2, 3, 4)), new HashSet<>(asList(1)))); - assertEquals(asScala(fooMoves), moveMap.get("foo").get()); - assertEquals(asScala(barMoves), moveMap.get("bar").get()); + assertEquals(fooMoves, moveMap.get("foo")); + assertEquals(barMoves, moveMap.get("bar")); Map expLeaderThrottle = new HashMap<>(); expLeaderThrottle.put("foo", "0:1,0:2,0:3,1:4,1:5,1:6,2:1,2:2,3:1,3:2,4:1,4:2,5:1,5:2"); expLeaderThrottle.put("bar", "0:2,0:3,0:4"); - assertEquals(asScala(expLeaderThrottle), calculateLeaderThrottles(moveMap)); + assertEquals(expLeaderThrottle, calculateLeaderThrottles(moveMap)); Map expFollowerThrottle = new HashMap<>(); expFollowerThrottle.put("foo", "0:5,1:7,1:8,2:3,2:4,3:5,3:6,4:3,5:3,5:4,5:5,5:6"); expFollowerThrottle.put("bar", "0:1"); - assertEquals(asScala(expFollowerThrottle), calculateFollowerThrottles(moveMap)); + assertEquals(expFollowerThrottle, calculateFollowerThrottles(moveMap)); - assertEquals(set(1, 2, 3, 4, 5, 6, 7, 8), calculateReassigningBrokers(moveMap)); - assertEquals(set(0, 2), calculateMovingBrokers(set( + assertEquals(new HashSet<>(asList(1, 2, 3, 4, 5, 6, 7, 8)), calculateReassigningBrokers(moveMap)); + assertEquals(new HashSet<>(asList(0, 2)), calculateMovingBrokers(new HashSet<>(asList( new TopicPartitionReplica("quux", 0, 0), - new TopicPartitionReplica("quux", 1, 2)))); + new TopicPartitionReplica("quux", 1, 2))))); } @Test - public void testParseExecuteAssignmentArgs() { + public void testParseExecuteAssignmentArgs() throws Exception { assertStartsWith("Partition reassignment list cannot be empty", assertThrows(AdminCommandFailedException.class, () -> parseExecuteAssignmentArgs("{\"version\":1,\"partitions\":[]}"), @@ -569,19 +561,19 @@ public class ReassignPartitionsUnitTest { "{\"topic\":\"foo\",\"partition\":1,\"replicas\":[2,3],\"log_dirs\":[\"/abc\",\"/def\"]}" + "]}"), "Expected to detect a partition replica list with duplicate entries").getMessage()); - Map> partitionsToBeReassigned = new HashMap<>(); + Map> partitionsToBeReassigned = new HashMap<>(); - partitionsToBeReassigned.put(new TopicPartition("foo", 0), seq(1, 2, 3)); - partitionsToBeReassigned.put(new TopicPartition("foo", 1), seq(3, 4, 5)); + partitionsToBeReassigned.put(new TopicPartition("foo", 0), asList(1, 2, 3)); + partitionsToBeReassigned.put(new TopicPartition("foo", 1), asList(3, 4, 5)); - Tuple2>, scala.collection.Map> actual = parseExecuteAssignmentArgs( + Tuple2>, Map> actual = parseExecuteAssignmentArgs( "{\"version\":1,\"partitions\":" + "[{\"topic\":\"foo\",\"partition\":0,\"replicas\":[1,2,3],\"log_dirs\":[\"any\",\"any\",\"any\"]}," + "{\"topic\":\"foo\",\"partition\":1,\"replicas\":[3,4,5],\"log_dirs\":[\"any\",\"any\",\"any\"]}" + "]}"); - assertEquals(asScala(partitionsToBeReassigned), actual._1); - assertTrue(actual._2.isEmpty()); + assertEquals(partitionsToBeReassigned, actual.v1); + assertTrue(actual.v2.isEmpty()); Map replicaAssignment = new HashMap<>(); @@ -594,8 +586,8 @@ public class ReassignPartitionsUnitTest { "[{\"topic\":\"foo\",\"partition\":0,\"replicas\":[1,2,3],\"log_dirs\":[\"/tmp/a\",\"/tmp/b\",\"/tmp/c\"]}" + "]}"); - assertEquals(asScala(Collections.singletonMap(new TopicPartition("foo", 0), seq(1, 2, 3))), actual._1); - assertEquals(asScala(replicaAssignment), actual._2); + assertEquals(Collections.singletonMap(new TopicPartition("foo", 0), asList(1, 2, 3)), actual.v1); + assertEquals(replicaAssignment, actual.v2); } @Test @@ -627,8 +619,8 @@ public class ReassignPartitionsUnitTest { @Test public void testModifyBrokerInterBrokerThrottle() throws Exception { try (MockAdminClient adminClient = new MockAdminClient.Builder().numBrokers(4).build()) { - modifyInterBrokerThrottle(adminClient, set(0, 1, 2), 1000); - modifyInterBrokerThrottle(adminClient, set(0, 3), 100); + modifyInterBrokerThrottle(adminClient, new HashSet<>(asList(0, 1, 2)), 1000); + modifyInterBrokerThrottle(adminClient, new HashSet<>(asList(0, 3)), 100); List brokers = new ArrayList<>(); for (int i = 0; i < 4; i++) brokers.add(new ConfigResource(ConfigResource.Type.BROKER, Integer.toString(i))); @@ -643,8 +635,8 @@ public class ReassignPartitionsUnitTest { @Test public void testModifyLogDirThrottle() throws Exception { try (MockAdminClient adminClient = new MockAdminClient.Builder().numBrokers(4).build()) { - modifyLogDirThrottle(adminClient, set(0, 1, 2), 2000); - modifyLogDirThrottle(adminClient, set(0, 3), -1); + modifyLogDirThrottle(adminClient, new HashSet<>(asList(0, 1, 2)), 2000); + modifyLogDirThrottle(adminClient, new HashSet<>(asList(0, 3)), -1); List brokers = new ArrayList<>(); for (int i = 0; i < 4; i++) @@ -660,19 +652,18 @@ public class ReassignPartitionsUnitTest { } @Test - public void testCurReassignmentsToString() { + public void testCurReassignmentsToString() throws Exception { try (MockAdminClient adminClient = new MockAdminClient.Builder().numBrokers(4).build()) { addTopics(adminClient); assertEquals("No partition reassignments found.", curReassignmentsToString(adminClient)); - Map> reassignments = new HashMap<>(); + Map> reassignments = new HashMap<>(); - reassignments.put(new TopicPartition("foo", 1), seq(4, 5, 3)); - reassignments.put(new TopicPartition("foo", 0), seq(0, 1, 4, 2)); - reassignments.put(new TopicPartition("bar", 0), seq(2, 3)); + reassignments.put(new TopicPartition("foo", 1), asList(4, 5, 3)); + reassignments.put(new TopicPartition("foo", 0), asList(0, 1, 4, 2)); + reassignments.put(new TopicPartition("bar", 0), asList(2, 3)); - scala.collection.Map reassignmentResult = - alterPartitionReassignments(adminClient, asScala(reassignments)); + Map reassignmentResult = alterPartitionReassignments(adminClient, reassignments); assertTrue(reassignmentResult.isEmpty()); assertEquals(String.join(System.lineSeparator(), @@ -691,13 +682,13 @@ public class ReassignPartitionsUnitTest { config.entries().forEach(entry -> configs.put(entry.name(), entry.value())); if (expectedInterBrokerThrottle >= 0) { assertEquals(Long.toString(expectedInterBrokerThrottle), - configs.getOrDefault(brokerLevelLeaderThrottle(), "")); + configs.getOrDefault(BROKER_LEVEL_LEADER_THROTTLE, "")); assertEquals(Long.toString(expectedInterBrokerThrottle), - configs.getOrDefault(brokerLevelFollowerThrottle(), "")); + configs.getOrDefault(BROKER_LEVEL_FOLLOWER_THROTTLE, "")); } if (expectedReplicaAlterLogDirsThrottle >= 0) { assertEquals(Long.toString(expectedReplicaAlterLogDirsThrottle), - configs.getOrDefault(brokerLevelLogDirThrottle(), "")); + configs.getOrDefault(BROKER_LEVEL_LOG_DIR_THROTTLE, "")); } } @@ -712,8 +703,8 @@ public class ReassignPartitionsUnitTest { leaderThrottles.put("bar", "leaderBar"); modifyTopicThrottles(adminClient, - asScala(leaderThrottles), - asScala(Collections.singletonMap("bar", "followerBar"))); + leaderThrottles, + Collections.singletonMap("bar", "followerBar")); List topics = Stream.of("bar", "foo").map( id -> new ConfigResource(ConfigResource.Type.TOPIC, id)).collect(Collectors.toList()); Map results = adminClient.describeConfigs(topics).all().get(); @@ -728,17 +719,17 @@ public class ReassignPartitionsUnitTest { Map configs = new HashMap<>(); config.entries().forEach(entry -> configs.put(entry.name(), entry.value())); assertEquals(expectedLeaderThrottle, - configs.getOrDefault(topicLevelLeaderThrottle(), "")); + configs.getOrDefault(TOPIC_LEVEL_LEADER_THROTTLE, "")); assertEquals(expectedFollowerThrottle, - configs.getOrDefault(topicLevelFollowerThrottle(), "")); + configs.getOrDefault(TOPIC_LEVEL_FOLLOWER_THROTTLE, "")); } @Test - public void testAlterReplicaLogDirs() { + public void testAlterReplicaLogDirs() throws Exception { try (MockAdminClient adminClient = new MockAdminClient.Builder(). numBrokers(4). brokerLogDirs(Collections.nCopies(4, - Arrays.asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"))). + asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"))). build()) { addTopics(adminClient); @@ -749,8 +740,8 @@ public class ReassignPartitionsUnitTest { assignment.put(new TopicPartitionReplica("quux", 1, 0), "/tmp/kafka-logs1"); assertEquals( - set(new TopicPartitionReplica("foo", 0, 0)), - alterReplicaLogDirs(adminClient, asScala(assignment)) + new HashSet<>(asList(new TopicPartitionReplica("foo", 0, 0))), + alterReplicaLogDirs(adminClient, assignment) ); } } @@ -767,24 +758,4 @@ public class ReassignPartitionsUnitTest { assertThrows(AdminOperationException.class, () -> executeAssignment(adminClient, false, "{invalid_json", -1L, -1L, 10000L, Time.SYSTEM)).getMessage()); } } - - @SuppressWarnings("unchecked") - private static scala.collection.immutable.Set set(final T... set) { - return mutableSet(set).toSet(); - } - - @SuppressWarnings({"deprecation", "unchecked"}) - private static scala.collection.mutable.Set mutableSet(final T...set) { - return JavaConverters.asScalaSet(new HashSet<>(Arrays.asList(set))); - } - - @SuppressWarnings({"deprecation", "unchecked"}) - private static Seq seq(T... seq) { - return JavaConverters.asScalaIteratorConverter(Arrays.asList(seq).iterator()).asScala().toSeq(); - } - - @SuppressWarnings("deprecation") - private static scala.collection.Map asScala(Map jmap) { - return JavaConverters.mapAsScalaMap(jmap); - } }