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

[GitHub] [kafka] nizhikov opened a new pull request, #13247: [WIP] KAFKA-14595 Move PartitionReassignmentState to java

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

   This PR contains changes required to move `PartitionReassignmentState` class to java code.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on a diff in pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandOptions.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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 joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+
+public class ReassignPartitionsCommandOptions extends CommandDefaultOptions {
+    // Actions
+    private final OptionSpecBuilder verifyOpt;

Review Comment:
   Thanks. fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] mimaison commented on a diff in pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/VerifyAssignmentResult.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionReplica;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * A result returned from verifyAssignment.
+ */
+public final class VerifyAssignmentResult {
+    private final Map<TopicPartition, PartitionReassignmentState> partStates;
+    private final boolean partsOngoing;
+    private final Map<TopicPartitionReplica, LogDirMoveState> moveStates;
+    private final boolean movesOngoing;
+
+    public VerifyAssignmentResult(Map<TopicPartition, PartitionReassignmentState> partStates) {
+        this(partStates, false, Collections.emptyMap(), false);
+    }
+
+    /**
+     * @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.
+     */
+    public VerifyAssignmentResult(
+        Map<TopicPartition, PartitionReassignmentState> partStates,
+        boolean partsOngoing,
+        Map<org.apache.kafka.common.TopicPartitionReplica, LogDirMoveState> moveStates,
+        boolean movesOngoing
+    ) {
+        this.partStates = partStates;
+        this.partsOngoing = partsOngoing;
+        this.moveStates = moveStates;
+        this.movesOngoing = movesOngoing;
+    }
+
+    public Map<TopicPartition, PartitionReassignmentState> partStates() {
+        return partStates;
+    }
+
+    public boolean partsOngoing() {
+        return partsOngoing;
+    }
+
+    public Map<TopicPartitionReplica, LogDirMoveState> moveStates() {
+        return moveStates;
+    }
+
+    public boolean movesOngoing() {
+        return movesOngoing;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        VerifyAssignmentResult that = (VerifyAssignmentResult) o;
+        return partsOngoing == that.partsOngoing && movesOngoing == that.movesOngoing && Objects.equals(partStates, that.partStates) && Objects.equals(moveStates, that.moveStates);
+    }
+
+    @Override
+    public int hashCode() {

Review Comment:
   Do we need `hashCode()` and `toString()` here? It looks like the original `VerifyAssignmentResult` class is only used by test (afaict the tool creates an instance but it's not used)



##########
tools/src/main/java/org/apache/kafka/tools/reassign/CancelledMoveState.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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;
+
+/**
+ * A replica log directory move state where there is no move in progress, but we did not
+ * reach the target log directory.
+ */
+public final class CancelledMoveState implements LogDirMoveState {
+    private final String currentLogDir;
+
+    private final String targetLogDir;
+
+    /**
+     * @param currentLogDir       The current log directory.
+     * @param targetLogDir        The log directory that we wanted the replica to move to.
+     */
+    public CancelledMoveState(String currentLogDir, String targetLogDir) {
+        this.currentLogDir = currentLogDir;
+        this.targetLogDir = targetLogDir;
+    }
+
+    public String currentLogDir() {
+        return currentLogDir;
+    }
+
+    public String targetLogDir() {
+        return targetLogDir;
+    }
+
+    @Override
+    public boolean done() {
+        return true;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        CancelledMoveState that = (CancelledMoveState) o;
+        return Objects.equals(currentLogDir, that.currentLogDir) && Objects.equals(targetLogDir, that.targetLogDir);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(currentLogDir, targetLogDir);
+    }
+
+    @Override
+    public String toString() {
+        return "CancelledMoveState{" +
+            "currentLogDir='" + currentLogDir + '\'' +
+            ", targetLogDir='" + targetLogDir + '\'' +
+            '}';
+    }
+}

Review Comment:
   We're missing a new line here



##########
tools/src/main/java/org/apache/kafka/tools/reassign/ActiveMoveState.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+/**
+ * A replica log directory move state where the move is in progress.
+ */
+public final class ActiveMoveState implements LogDirMoveState {

Review Comment:
   These classes are extremely verbose. I wonder if we could make the `final` fields public and get rid of all the setters. 



##########
tools/src/main/java/org/apache/kafka/tools/reassign/TerseReassignmentFailureException.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * 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 org.apache.kafka.common.KafkaException;
+
+/**
+ * An exception thrown to indicate that the command has failed, but we don't want to
+ * print a stack trace.
+ */
+public class TerseReassignmentFailureException extends KafkaException {

Review Comment:
   Yes we try to keep the logic as is as but in this case, I think we can do this simplification. There's no point of creating this new class where it really looks like it will not be needed. Worst case scenario, we'll create it when we move the tool if we can't do without.



##########
tools/src/main/java/org/apache/kafka/tools/reassign/ActiveMoveState.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+/**
+ * A replica log directory move state where the move is in progress.
+ */
+public final class ActiveMoveState implements LogDirMoveState {
+    private final String currentLogDir;
+
+    private final String targetLogDir;
+
+    private final String futureLogDir;
+
+    /**
+     * @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.
+     */
+    public ActiveMoveState(String currentLogDir, String targetLogDir, String futureLogDir) {
+        this.currentLogDir = currentLogDir;
+        this.targetLogDir = targetLogDir;
+        this.futureLogDir = futureLogDir;
+    }
+
+    public String currentLogDir() {
+        return currentLogDir;
+    }
+
+    public String targetLogDir() {
+        return targetLogDir;
+    }
+
+    public String futureLogDir() {
+        return futureLogDir;
+    }
+
+    @Override
+    public boolean done() {
+        return false;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        ActiveMoveState that = (ActiveMoveState) o;
+        return Objects.equals(currentLogDir, that.currentLogDir) && Objects.equals(targetLogDir, that.targetLogDir) && Objects.equals(futureLogDir, that.futureLogDir);
+    }
+
+    @Override
+    public int hashCode() {

Review Comment:
   Do we need to override `hashCode()` and `toString()` for these classes?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   Hello @showuon 
   Can you, join the review, please?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java:
##########
@@ -769,22 +762,12 @@ public void testPropagateInvalidJsonError() {
     }
 
     @SuppressWarnings("unchecked")
-    private static <T> scala.collection.immutable.Set<T> set(final T... set) {
-        return mutableSet(set).toSet();
-    }
-
-    @SuppressWarnings({"deprecation", "unchecked"})
-    private static <T> scala.collection.mutable.Set<T> mutableSet(final T...set) {
-        return JavaConverters.asScalaSet(new HashSet<>(Arrays.asList(set)));
+    private static <T> Set<T> set(final T... set) {
+        return new HashSet<>(Arrays.asList(set));
     }
 
-    @SuppressWarnings({"deprecation", "unchecked"})
-    private static <T> Seq<T> seq(T... seq) {
-        return JavaConverters.asScalaIteratorConverter(Arrays.asList(seq).iterator()).asScala().toSeq();
-    }
-
-    @SuppressWarnings("deprecation")
-    private static <K, V> scala.collection.Map<K, V> asScala(Map<K, V> jmap) {
-        return JavaConverters.mapAsScalaMap(jmap);
+    @SuppressWarnings("unchecked")

Review Comment:
   Yes. Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   @tledkov 
   
   > `ReplicationQuotasTestRig` no longer needed?
   
   This class not used through current codebase.
   But it has dependency on `ReassignPartitionCommand`.
   
   So, for now, I just removed it.
   If there are use-cases for it, we must write them down and cover class with tests, at least :)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   Thanks all for the help, review and merge! 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 [WIP] Move ReassignPartitionsCommand to java

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

   @mimaison There are few classes that used elsewhere in code and in `ReassignPartitionsCommand`. Specifically:
   
   * `BrokerMetadata` simple case class.
   * `AdminUtils` partition distribution algorithm implementation.
   
   Let's rewrite those two classes and move them into `server-common` in separate ticket? 
   Prior to `ReassignPartitionsCommand` command. 
   Because, current changes for command are pretty huge, already.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
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<V1, V2> {
+    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 "Tuple{v1=" + v1 + ", v2=" + v2 + '}';

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1501 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.println(partitionReassignmentStatesToString(partStates));
+        return new Tuple2<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });

Review Comment:
   Greate suggestion. Thanks.



##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1501 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.println(partitionReassignmentStatesToString(partStates));
+        return new Tuple2<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });

Review Comment:
   Great suggestion. Thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1501 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.println(partitionReassignmentStatesToString(partStates));
+        return new Tuple2<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });

Review Comment:
   What about using simple stream API here:
   ```suggestion
           Set<String> topicNamesToLookUp = notFoundReassignments.stream()
               .map(e -> e.v1)
               .filter(part -> !currentReassignments.containsKey(part))
               .map(TopicPartition::topic)
               .collect(Collectors.toSet());
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java:
##########
@@ -769,22 +762,12 @@ public void testPropagateInvalidJsonError() {
     }
 
     @SuppressWarnings("unchecked")
-    private static <T> scala.collection.immutable.Set<T> set(final T... set) {
-        return mutableSet(set).toSet();
-    }
-
-    @SuppressWarnings({"deprecation", "unchecked"})
-    private static <T> scala.collection.mutable.Set<T> mutableSet(final T...set) {
-        return JavaConverters.asScalaSet(new HashSet<>(Arrays.asList(set)));
+    private static <T> Set<T> set(final T... set) {

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on a diff in pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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;
+
+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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                              List<Tuple<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.print(partitionReassignmentStatesToString(partStates));
+        return new Tuple<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                           List<Tuple<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+
+            if (currentReassignments.containsKey(part)) {
+                PartitionReassignment reassignment = currentReassignments.get(part);
+                notFoundResults.add(new Tuple<>(part, new PartitionReassignmentState(
+                    reassignment.replicas(),
+                    targetReplicas,
+                    false)));
+            } else {
+                notFoundResults.add(new Tuple<>(part, topicDescriptionFutureToState(part.partition(),
+                    topicDescriptions.get(part.topic()), targetReplicas)));
+            }
+        }
+
+        Map<TopicPartition, PartitionReassignmentState> allResults = new HashMap<>();
+        foundResults.forEach(e -> allResults.put(e.v1, e.v2));
+        notFoundResults.forEach(e -> allResults.put(e.v1, e.v2));
+
+        return new Tuple<>(allResults, currentReassignments.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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 Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                  Map<TopicPartitionReplica, String> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> moveStates = findLogDirMoveStates(adminClient, targetReassignments);
+        System.out.println(replicaMoveStatesToString(moveStates));
+        return new Tuple<>(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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient.

Review Comment:
   Moved. Thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on a diff in pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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;
+
+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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                              List<Tuple<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.print(partitionReassignmentStatesToString(partStates));

Review Comment:
   Thanks, fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> 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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                             String reassignmentJson,
+                                                                                                             String brokerListString,
+                                                                                                             Boolean enableRackAwareness
+    ) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple2<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        Map<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                                                Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                Set<TopicPartition> partitions
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                           String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> 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<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> t0 = parseExecuteAssignmentArgs(reassignmentJson);
+
+        Map<TopicPartition, List<Integer>> proposedParts = t0.v1;
+        Map<TopicPartitionReplica, String> proposedReplicas = t0.v2;
+        Map<TopicPartition, PartitionReassignment> 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<Integer> brokers = new HashSet<>();
+        proposedParts.values().forEach(brokers::addAll);
+
+        verifyBrokerIds(adminClient, brokers);
+        Map<TopicPartition, List<Integer>> 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<String, Map<Integer, PartitionMove>> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts);
+                modifyReassignmentThrottle(adminClient, moveMap, interBrokerThrottle);
+            }
+
+            if (logDirThrottle >= 0) {
+                Set<Integer> movingBrokers = calculateMovingBrokers(proposedReplicas.keySet());
+                modifyLogDirThrottle(adminClient, movingBrokers, logDirThrottle);
+            }
+        }
+
+        // Execute the partition reassignments.
+        Map<TopicPartition, Throwable> 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<TopicPartitionReplica, String> proposedReplicas,
+                                     Long timeoutMs,
+                                     Time time
+    ) throws InterruptedException, TerseException {
+        long startTimeMs = time.milliseconds();
+        Map<TopicPartitionReplica, String> pendingReplicas = new HashMap<>(proposedReplicas);
+        boolean done = false;
+        do {
+            Set<TopicPartitionReplica> 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<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+        String text = currentReassignments.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).map(part -> {
+            PartitionReassignment reassignment = currentReassignments.get(part);
+            List<Integer> replicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+            List<Integer> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Set<Integer> allNodeIds = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Optional<Integer> 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<TopicPartition, List<Integer>> proposedParts,
+                                                            Map<TopicPartition, List<Integer>> currentParts) throws JsonProcessingException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, Throwable> alterPartitionReassignments(Admin adminClient,
+                                                                      Map<TopicPartition, List<Integer>> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.forEach((part, replicas) -> args.put(part, Optional.of(new NewPartitionReassignment(replicas))));
+        Map<TopicPartition, KafkaFuture<Void>> results = adminClient.alterPartitionReassignments(args).values();
+        Map<TopicPartition, Throwable> errors = new HashMap<>();
+        for (Map.Entry<TopicPartition, KafkaFuture<Void>> 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<TopicPartition, Throwable> cancelPartitionReassignments(Admin adminClient,
+                                                                       Set<TopicPartition> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.forEach(part -> args.put(part, Optional.empty()));
+
+        Map<TopicPartition, KafkaFuture<Void>> results = adminClient.alterPartitionReassignments(args).values();
+        Map<TopicPartition, Throwable> errors = new HashMap<>();
+        for (Map.Entry<TopicPartition, KafkaFuture<Void>> 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<String, Map<Integer, PartitionMove>> calculateCurrentMoveMap(Map<TopicPartition, PartitionReassignment> currentReassignments) {
+        Map<String, Map<Integer, PartitionMove>> moveMap = new HashMap<>();
+        // Add the current reassignments to the move map.
+        currentReassignments.forEach((part, reassignment) -> {
+
+            List<Integer> allReplicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+
+            // The addingReplicas is included in the replicas during reassignment
+            Set<Integer> sources = new HashSet<>(allReplicas);
+            addingReplicas.forEach(sources::remove);
+
+            Set<Integer> destinations = new HashSet<>(addingReplicas);
+
+            Map<Integer, PartitionMove> 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<String, Map<Integer, PartitionMove>> calculateProposedMoveMap(Map<TopicPartition, PartitionReassignment> currentReassignments,
+                                                                             Map<TopicPartition, List<Integer>> proposedParts,
+                                                                             Map<TopicPartition, List<Integer>> currentParts) {
+        Map<String, Map<Integer, PartitionMove>> moveMap = calculateCurrentMoveMap(currentReassignments);
+        for (Map.Entry<TopicPartition, List<Integer>> e : proposedParts.entrySet()) {
+            TopicPartition part = e.getKey();
+            List<Integer> replicas = e.getValue();
+            Map<Integer, PartitionMove> 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<Integer> 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<Integer> 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<String, String> calculateLeaderThrottles(Map<String, Map<Integer, PartitionMove>> moveMap) {
+        Map<String, String> results = new HashMap<>();
+        moveMap.forEach((topicName, partMoveMap) -> {
+            Set<String> 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<String, String> calculateFollowerThrottles(Map<String, Map<Integer, PartitionMove>> moveMap) {
+        Map<String, String> results = new HashMap<>();
+        moveMap.forEach((topicName, partMoveMap) -> {
+            Set<String> 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<Integer> calculateReassigningBrokers(Map<String, Map<Integer, PartitionMove>> moveMap) {
+        Set<Integer> 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<Integer> calculateMovingBrokers(Set<TopicPartitionReplica> 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<String, String> leaderThrottles,
+                                     Map<String, String> followerThrottles) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configs = new HashMap<>();
+        Set<String> topicNames = new HashSet<>(leaderThrottles.keySet());
+        topicNames.addAll(followerThrottles.keySet());
+        topicNames.forEach(topicName -> {
+            List<AlterConfigOp> 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<String, Map<Integer, PartitionMove>> moveMap,
+        Long interBrokerThrottle
+    ) throws ExecutionException, InterruptedException {
+        Map<String, String> leaderThrottles = calculateLeaderThrottles(moveMap);
+        Map<String, String> followerThrottles = calculateFollowerThrottles(moveMap);
+        modifyTopicThrottles(admin, leaderThrottles, followerThrottles);
+
+        Set<Integer> 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<Integer> reassigningBrokers,
+                                          long interBrokerThrottle) throws ExecutionException, InterruptedException {
+        if (interBrokerThrottle >= 0) {
+            Map<ConfigResource, Collection<AlterConfigOp>> configs = new HashMap<>();
+            reassigningBrokers.forEach(brokerId -> {
+                List<AlterConfigOp> 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<Integer> movingBrokers,
+                                     long logDirThrottle) throws ExecutionException, InterruptedException {
+        if (logDirThrottle >= 0) {
+            Map<ConfigResource, Collection<AlterConfigOp>> configs = new HashMap<>();
+            movingBrokers.forEach(brokerId -> {
+                List<AlterConfigOp> 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<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> parseExecuteAssignmentArgs(
+        String reassignmentJson
+    ) throws JsonProcessingException {
+        Tuple2<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(reassignmentJson);
+
+        List<Tuple2<TopicPartition, List<Integer>>> partitionsToBeReassigned = t0.v1;
+        Map<TopicPartitionReplica, String> 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<TopicPartition> 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<Tuple2<TopicPartition, Set<Integer>>> 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<TopicPartition>, Set<TopicPartitionReplica>> cancelAssignment(Admin adminClient,
+                                                                                    String jsonString,
+                                                                                    Boolean preserveThrottles,
+                                                                                    Long timeoutMs,
+                                                                                    Time time
+    ) throws ExecutionException, InterruptedException, JsonProcessingException, TerseException {
+        Tuple2<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetReplicas = t0.v2;
+        Set<TopicPartition> targetPartsSet = targetParts.stream().map(t -> t.v1).collect(Collectors.toSet());
+        Set<TopicPartition> 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<TopicPartition, Throwable> 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<TopicPartitionReplica, String> 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<TopicPartition, List<Integer>> partitionsToBeReassigned,
+                                                   Map<TopicPartitionReplica, String> replicaLogDirAssignment) throws JsonProcessingException {
+        List<Map<String, Object>> partitions = new ArrayList<>();
+        partitionsToBeReassigned.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).forEach(tp -> {
+            List<Integer> replicas = partitionsToBeReassigned.get(tp);
+            Map<String, Object> data = new LinkedHashMap<>();

Review Comment:
   > Does scala Map with .asJAva guarantee keys order?
   
   Yes. Please, take a look to `ReassignPartitionsUnitTest#testCurrentPartitionReplicaAssignmentToString`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   @tledkov Great catch. Unused code removed.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] ijuma commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

   If you want to do it in small steps, one way is that you introduce the new classes, but you do not update the command to use them. That way you can put them in the right destination from the start. In any case, I'll leave it to @mimaison to say how he'd prefer it.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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

   Hello @mimaison . Is there any change you review #14355 ?
   It's contains `ReassignPartitionsUnitTest` in java to simplify this PR even more.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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

   Tests for #14355 looks OK. Please, review.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] kamalcph commented on a diff in pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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


##########
checkstyle/checkstyle.xml:
##########
@@ -119,7 +119,7 @@
     </module>
     <module name="ClassDataAbstractionCoupling">
       <!-- default is 7 -->
-      <property name="max" value="25"/>
+      <property name="max" value="30"/>

Review Comment:
   Instead of changing the default value for all the files, can we suppress the checkstyle only on the specified file?
   
   ```
   @SuppressWarnings("ClassDataAbstractionCoupling")
   ```



##########
checkstyle/import-control.xml:
##########
@@ -292,7 +294,11 @@
     <allow pkg="scala.collection" />
 
     <subpackage name="reassign">
-      <allow pkg="kafka.admin" />
+        <allow pkg="kafka.admin" />

Review Comment:
   nit: L297 alignment



##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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;
+
+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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                              List<Tuple<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.print(partitionReassignmentStatesToString(partStates));

Review Comment:
   In scala, it's `println` -> `System.out.println`



##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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;
+
+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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                              List<Tuple<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.print(partitionReassignmentStatesToString(partStates));
+        return new Tuple<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                           List<Tuple<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+
+            if (currentReassignments.containsKey(part)) {
+                PartitionReassignment reassignment = currentReassignments.get(part);
+                notFoundResults.add(new Tuple<>(part, new PartitionReassignmentState(
+                    reassignment.replicas(),
+                    targetReplicas,
+                    false)));
+            } else {
+                notFoundResults.add(new Tuple<>(part, topicDescriptionFutureToState(part.partition(),
+                    topicDescriptions.get(part.topic()), targetReplicas)));
+            }
+        }
+
+        Map<TopicPartition, PartitionReassignmentState> allResults = new HashMap<>();
+        foundResults.forEach(e -> allResults.put(e.v1, e.v2));
+        notFoundResults.forEach(e -> allResults.put(e.v1, e.v2));
+
+        return new Tuple<>(allResults, currentReassignments.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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 Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                  Map<TopicPartitionReplica, String> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> moveStates = findLogDirMoveStates(adminClient, targetReassignments);
+        System.out.println(replicaMoveStatesToString(moveStates));
+        return new Tuple<>(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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient.
+            describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple<TopicPartition, List<Integer>>> targetParts) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        targetParts.forEach(t -> brokers.addAll(t.v2));
+
+        System.out.printf("Clearing broker-level throttles on broker%s %s%n",
+            brokers.size() == 1 ? "" : "s", brokers.stream().map(Object::toString).collect(Collectors.joining(",")));

Review Comment:
   nit:
   
   We can use Utils#join here and similar places:
   ```
   Utils.join(brokers, ",");
   ```



##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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;
+
+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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                              List<Tuple<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.print(partitionReassignmentStatesToString(partStates));
+        return new Tuple<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                           List<Tuple<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+
+            if (currentReassignments.containsKey(part)) {
+                PartitionReassignment reassignment = currentReassignments.get(part);
+                notFoundResults.add(new Tuple<>(part, new PartitionReassignmentState(
+                    reassignment.replicas(),
+                    targetReplicas,
+                    false)));
+            } else {
+                notFoundResults.add(new Tuple<>(part, topicDescriptionFutureToState(part.partition(),
+                    topicDescriptions.get(part.topic()), targetReplicas)));
+            }
+        }
+
+        Map<TopicPartition, PartitionReassignmentState> allResults = new HashMap<>();
+        foundResults.forEach(e -> allResults.put(e.v1, e.v2));
+        notFoundResults.forEach(e -> allResults.put(e.v1, e.v2));
+
+        return new Tuple<>(allResults, currentReassignments.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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 Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                  Map<TopicPartitionReplica, String> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> moveStates = findLogDirMoveStates(adminClient, targetReassignments);
+        System.out.println(replicaMoveStatesToString(moveStates));
+        return new Tuple<>(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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient.

Review Comment:
   subjective:
   
   Can we use the dot (.) in the next line similar to other files?
   
   ```java
   adminClient
       .describeReplicaLogDirs(..)
   ```
   



##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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;
+
+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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                              List<Tuple<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.print(partitionReassignmentStatesToString(partStates));
+        return new Tuple<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                           List<Tuple<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+
+            if (currentReassignments.containsKey(part)) {
+                PartitionReassignment reassignment = currentReassignments.get(part);
+                notFoundResults.add(new Tuple<>(part, new PartitionReassignmentState(
+                    reassignment.replicas(),
+                    targetReplicas,
+                    false)));
+            } else {
+                notFoundResults.add(new Tuple<>(part, topicDescriptionFutureToState(part.partition(),
+                    topicDescriptions.get(part.topic()), targetReplicas)));
+            }
+        }
+
+        Map<TopicPartition, PartitionReassignmentState> allResults = new HashMap<>();
+        foundResults.forEach(e -> allResults.put(e.v1, e.v2));
+        notFoundResults.forEach(e -> allResults.put(e.v1, e.v2));
+
+        return new Tuple<>(allResults, currentReassignments.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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 Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                  Map<TopicPartitionReplica, String> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> moveStates = findLogDirMoveStates(adminClient, targetReassignments);
+        System.out.println(replicaMoveStatesToString(moveStates));
+        return new Tuple<>(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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient.
+            describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple<TopicPartition, List<Integer>>> targetParts) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        targetParts.forEach(t -> brokers.addAll(t.v2));
+
+        System.out.printf("Clearing broker-level throttles on broker%s %s%n",
+            brokers.size() == 1 ? "" : "s", brokers.stream().map(Object::toString).collect(Collectors.joining(",")));
+        clearBrokerLevelThrottles(adminClient, brokers);
+
+        Set<String> 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", topics.stream().map(Object::toString).collect(Collectors.joining(",")));
+        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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, 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 Tuple<Map<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                            String reassignmentJson,
+                                                                                                            String brokerListString,
+                                                                                                            Boolean enableRackAwareness) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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",
+            formatAsReassignmentJson(proposedAssignments, Collections.emptyMap()));
+        return new Tuple<>(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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        HashMap<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                       Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics) throws ExecutionException, InterruptedException {
+
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                        Set<TopicPartition> partitions) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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 Tuple<List<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                                  String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> duplicateTopicsToReassign = ToolsUtils.duplicates(topicsToReassign);
+        if (!duplicateTopicsToReassign.isEmpty())
+            throw new AdminCommandFailedException(String.format("List of topics to reassign contains duplicate entries: %s",
+                duplicateTopicsToReassign));
+        return new Tuple<>(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,

Review Comment:
   nit: Parameter alignment



##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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;
+
+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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                              List<Tuple<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.print(partitionReassignmentStatesToString(partStates));
+        return new Tuple<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                           List<Tuple<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+
+            if (currentReassignments.containsKey(part)) {
+                PartitionReassignment reassignment = currentReassignments.get(part);
+                notFoundResults.add(new Tuple<>(part, new PartitionReassignmentState(
+                    reassignment.replicas(),
+                    targetReplicas,
+                    false)));
+            } else {
+                notFoundResults.add(new Tuple<>(part, topicDescriptionFutureToState(part.partition(),
+                    topicDescriptions.get(part.topic()), targetReplicas)));
+            }
+        }
+
+        Map<TopicPartition, PartitionReassignmentState> allResults = new HashMap<>();
+        foundResults.forEach(e -> allResults.put(e.v1, e.v2));
+        notFoundResults.forEach(e -> allResults.put(e.v1, e.v2));
+
+        return new Tuple<>(allResults, currentReassignments.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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 Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                  Map<TopicPartitionReplica, String> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> moveStates = findLogDirMoveStates(adminClient, targetReassignments);
+        System.out.println(replicaMoveStatesToString(moveStates));
+        return new Tuple<>(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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient.
+            describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple<TopicPartition, List<Integer>>> targetParts) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        targetParts.forEach(t -> brokers.addAll(t.v2));
+
+        System.out.printf("Clearing broker-level throttles on broker%s %s%n",
+            brokers.size() == 1 ? "" : "s", brokers.stream().map(Object::toString).collect(Collectors.joining(",")));
+        clearBrokerLevelThrottles(adminClient, brokers);
+
+        Set<String> 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", topics.stream().map(Object::toString).collect(Collectors.joining(",")));
+        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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, 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 Tuple<Map<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                            String reassignmentJson,
+                                                                                                            String brokerListString,
+                                                                                                            Boolean enableRackAwareness) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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",
+            formatAsReassignmentJson(proposedAssignments, Collections.emptyMap()));
+        return new Tuple<>(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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        HashMap<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                       Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics) throws ExecutionException, InterruptedException {
+
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                        Set<TopicPartition> partitions) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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 Tuple<List<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                                  String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> duplicateTopicsToReassign = ToolsUtils.duplicates(topicsToReassign);
+        if (!duplicateTopicsToReassign.isEmpty())
+            throw new AdminCommandFailedException(String.format("List of topics to reassign contains duplicate entries: %s",
+                duplicateTopicsToReassign));
+        return new Tuple<>(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 {
+        Tuple<Map<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> t0 = parseExecuteAssignmentArgs(reassignmentJson);
+
+        Map<TopicPartition, List<Integer>> proposedParts = t0.v1;
+        Map<TopicPartitionReplica, String> proposedReplicas = t0.v2;
+        Map<TopicPartition, PartitionReassignment> 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<Integer> brokers = new HashSet<>();
+        proposedParts.values().forEach(brokers::addAll);
+
+        verifyBrokerIds(adminClient, brokers);
+        Map<TopicPartition, List<Integer>> 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<String, Map<Integer, PartitionMove>> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts);
+                modifyReassignmentThrottle(adminClient, moveMap, interBrokerThrottle);
+            }
+
+            if (logDirThrottle >= 0) {
+                Set<Integer> movingBrokers = calculateMovingBrokers(proposedReplicas.keySet());
+                modifyLogDirThrottle(adminClient, movingBrokers, logDirThrottle);
+            }
+        }
+
+        // Execute the partition reassignments.
+        Map<TopicPartition, Throwable> 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<TopicPartitionReplica, String> proposedReplicas,
+                                     Long timeoutMs,
+                                     Time time) throws InterruptedException, TerseException {
+        long startTimeMs = time.milliseconds();
+        Map<TopicPartitionReplica, String> pendingReplicas = new HashMap<>();
+        pendingReplicas.putAll(proposedReplicas);
+        boolean done = false;
+        do {
+            Set<TopicPartitionReplica> completed = alterReplicaLogDirs(adminClient, pendingReplicas);
+            if (!completed.isEmpty()) {
+                System.out.printf("Successfully started log directory move%s for: %s",
+                    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<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+        String text = currentReassignments.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).map(part -> {
+            PartitionReassignment reassignment = currentReassignments.get(part);
+            List<Integer> replicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+            List<Integer> 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.length() == 0
+            ? "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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Set<Integer> allNodeIds = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Optional<Integer> unknown = brokers.stream().
+            filter(i -> !allNodeIds.contains(i)).
+            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<TopicPartition, List<Integer>> proposedParts,
+                                                            Map<TopicPartition, List<Integer>> currentParts) throws JsonProcessingException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, Throwable> alterPartitionReassignments(Admin adminClient,
+                                                                      Map<TopicPartition, List<Integer>> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.entrySet().forEach(e -> {
+            TopicPartition part = e.getKey();
+            List<Integer> replicas = e.getValue();
+
+            args.put(part, Optional.of(new NewPartitionReassignment(replicas)));
+        });
+        Map<TopicPartition, KafkaFuture<Void>> results = adminClient.alterPartitionReassignments(args).values();
+        Map<TopicPartition, Throwable> errors = new HashMap<>();
+        for (Map.Entry<TopicPartition, KafkaFuture<Void>> 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<TopicPartition, Throwable> cancelPartitionReassignments(Admin adminClient,
+                                                                               Set<TopicPartition> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.forEach(part -> args.put(part, Optional.empty()));
+
+        Map<TopicPartition, KafkaFuture<Void>> results = adminClient.alterPartitionReassignments(args).values();
+        Map<TopicPartition, Throwable> errors = new HashMap<>();
+        for (Map.Entry<TopicPartition, KafkaFuture<Void>> 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<String, Map<Integer, PartitionMove>> calculateCurrentMoveMap(Map<TopicPartition, PartitionReassignment> currentReassignments) {
+        Map<String, Map<Integer, PartitionMove>> moveMap = new HashMap<>();
+        // Add the current reassignments to the move map.
+        currentReassignments.entrySet().forEach(e -> {
+            TopicPartition part = e.getKey();
+            PartitionReassignment reassignment = e.getValue();
+
+            List<Integer> allReplicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+
+            // The addingReplicas is included in the replicas during reassignment
+            Set<Integer> sources = new HashSet<>(allReplicas);
+            sources.removeAll(addingReplicas);
+
+            Set<Integer> destinations = new HashSet<>(addingReplicas);
+
+            Map<Integer, PartitionMove> 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<String, Map<Integer, PartitionMove>> calculateProposedMoveMap(Map<TopicPartition, PartitionReassignment> currentReassignments,
+                                                                             Map<TopicPartition, List<Integer>> proposedParts,
+                                                                             Map<TopicPartition, List<Integer>> currentParts) {
+        Map<String, Map<Integer, PartitionMove>> moveMap = calculateCurrentMoveMap(currentReassignments);
+        for (Map.Entry<TopicPartition, List<Integer>> e : proposedParts.entrySet()) {
+            TopicPartition part = e.getKey();
+            List<Integer> replicas = e.getValue();
+            Map<Integer, PartitionMove> 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<Integer> 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<Integer> 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<String, String> calculateLeaderThrottles(Map<String, Map<Integer, PartitionMove>> moveMap) {
+        Map<String, String> results = new HashMap<>();
+        moveMap.forEach((topicName, partMoveMap) -> {
+            Set<String> 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<String, String> calculateFollowerThrottles(Map<String, Map<Integer, PartitionMove>> moveMap) {
+        Map<String, String> results = new HashMap<>();
+        moveMap.forEach((topicName, partMoveMap) -> {
+            Set<String> 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<Integer> calculateReassigningBrokers(Map<String, Map<Integer, PartitionMove>> moveMap) {
+        Set<Integer> 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<Integer> calculateMovingBrokers(Set<TopicPartitionReplica> 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<String, String> leaderThrottles,
+                                     Map<String, String> followerThrottles) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configs = new HashMap<>();
+        Set<String> topicNames = new HashSet<>(leaderThrottles.keySet());
+        topicNames.addAll(followerThrottles.keySet());
+        topicNames.forEach(topicName -> {
+            List<AlterConfigOp> 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<String, Map<Integer, PartitionMove>> moveMap, Long interBrokerThrottle) throws ExecutionException, InterruptedException {
+        Map<String, String> leaderThrottles = calculateLeaderThrottles(moveMap);
+        Map<String, String> followerThrottles = calculateFollowerThrottles(moveMap);
+        modifyTopicThrottles(admin, leaderThrottles, followerThrottles);
+
+        Set<Integer> 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<Integer> reassigningBrokers,
+                                          long interBrokerThrottle) throws ExecutionException, InterruptedException {
+        if (interBrokerThrottle >= 0) {
+            Map<ConfigResource, Collection<AlterConfigOp>> configs = new HashMap<>();
+            reassigningBrokers.forEach(brokerId -> {
+                List<AlterConfigOp> 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<Integer> movingBrokers,
+                                     long logDirThrottle) throws ExecutionException, InterruptedException {
+        if (logDirThrottle >= 0) {
+            Map<ConfigResource, Collection<AlterConfigOp>> configs = new HashMap<>();
+            movingBrokers.forEach(brokerId -> {
+                List<AlterConfigOp> 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 Tuple<Map<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> parseExecuteAssignmentArgs(String reassignmentJson) throws JsonProcessingException {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(reassignmentJson);
+
+        List<Tuple<TopicPartition, List<Integer>>> partitionsToBeReassigned = t0.v1;
+        Map<TopicPartitionReplica, String> 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<TopicPartition> 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<Tuple<TopicPartition, Set<Integer>>> duplicateEntries = partitionsToBeReassigned.stream()
+            .map(t -> new Tuple<>(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 Tuple<>(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 Tuple<Set<TopicPartition>, Set<TopicPartitionReplica>> cancelAssignment(Admin adminClient,
+                                                                                   String jsonString,
+                                                                                   Boolean preserveThrottles,
+                                                                                   Long timeoutMs,
+                                                                                   Time time) throws ExecutionException, InterruptedException, JsonProcessingException, TerseException {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetReplicas = t0.v2;
+        Set<TopicPartition> targetPartsSet = targetParts.stream().map(t -> t.v1).collect(Collectors.toSet());
+        Set<TopicPartition> 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<TopicPartition, Throwable> 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",

Review Comment:
   new line at the end (%n)



##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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;
+
+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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                              List<Tuple<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.print(partitionReassignmentStatesToString(partStates));
+        return new Tuple<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                           List<Tuple<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+
+            if (currentReassignments.containsKey(part)) {
+                PartitionReassignment reassignment = currentReassignments.get(part);
+                notFoundResults.add(new Tuple<>(part, new PartitionReassignmentState(
+                    reassignment.replicas(),
+                    targetReplicas,
+                    false)));
+            } else {
+                notFoundResults.add(new Tuple<>(part, topicDescriptionFutureToState(part.partition(),
+                    topicDescriptions.get(part.topic()), targetReplicas)));
+            }
+        }
+
+        Map<TopicPartition, PartitionReassignmentState> allResults = new HashMap<>();
+        foundResults.forEach(e -> allResults.put(e.v1, e.v2));
+        notFoundResults.forEach(e -> allResults.put(e.v1, e.v2));
+
+        return new Tuple<>(allResults, currentReassignments.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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 Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                  Map<TopicPartitionReplica, String> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> moveStates = findLogDirMoveStates(adminClient, targetReassignments);
+        System.out.println(replicaMoveStatesToString(moveStates));
+        return new Tuple<>(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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient.
+            describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);

Review Comment:
   Left a minor [comment](https://github.com/apache/kafka/pull/14172/files#r1328541806) in #14172 



##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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;
+
+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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                              List<Tuple<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.print(partitionReassignmentStatesToString(partStates));
+        return new Tuple<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                           List<Tuple<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+
+            if (currentReassignments.containsKey(part)) {
+                PartitionReassignment reassignment = currentReassignments.get(part);
+                notFoundResults.add(new Tuple<>(part, new PartitionReassignmentState(
+                    reassignment.replicas(),
+                    targetReplicas,
+                    false)));
+            } else {
+                notFoundResults.add(new Tuple<>(part, topicDescriptionFutureToState(part.partition(),
+                    topicDescriptions.get(part.topic()), targetReplicas)));
+            }
+        }
+
+        Map<TopicPartition, PartitionReassignmentState> allResults = new HashMap<>();
+        foundResults.forEach(e -> allResults.put(e.v1, e.v2));
+        notFoundResults.forEach(e -> allResults.put(e.v1, e.v2));
+
+        return new Tuple<>(allResults, currentReassignments.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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 Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                  Map<TopicPartitionReplica, String> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> moveStates = findLogDirMoveStates(adminClient, targetReassignments);
+        System.out.println(replicaMoveStatesToString(moveStates));
+        return new Tuple<>(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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient.
+            describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple<TopicPartition, List<Integer>>> targetParts) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        targetParts.forEach(t -> brokers.addAll(t.v2));
+
+        System.out.printf("Clearing broker-level throttles on broker%s %s%n",
+            brokers.size() == 1 ? "" : "s", brokers.stream().map(Object::toString).collect(Collectors.joining(",")));
+        clearBrokerLevelThrottles(adminClient, brokers);
+
+        Set<String> 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", topics.stream().map(Object::toString).collect(Collectors.joining(",")));
+        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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, 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 Tuple<Map<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                            String reassignmentJson,
+                                                                                                            String brokerListString,
+                                                                                                            Boolean enableRackAwareness) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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",
+            formatAsReassignmentJson(proposedAssignments, Collections.emptyMap()));
+        return new Tuple<>(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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        HashMap<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();

Review Comment:
   LHS: 
   
   HashMap -> Map



##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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;
+
+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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                              List<Tuple<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.print(partitionReassignmentStatesToString(partStates));
+        return new Tuple<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                           List<Tuple<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+
+            if (currentReassignments.containsKey(part)) {
+                PartitionReassignment reassignment = currentReassignments.get(part);
+                notFoundResults.add(new Tuple<>(part, new PartitionReassignmentState(
+                    reassignment.replicas(),
+                    targetReplicas,
+                    false)));
+            } else {
+                notFoundResults.add(new Tuple<>(part, topicDescriptionFutureToState(part.partition(),
+                    topicDescriptions.get(part.topic()), targetReplicas)));
+            }
+        }
+
+        Map<TopicPartition, PartitionReassignmentState> allResults = new HashMap<>();
+        foundResults.forEach(e -> allResults.put(e.v1, e.v2));
+        notFoundResults.forEach(e -> allResults.put(e.v1, e.v2));
+
+        return new Tuple<>(allResults, currentReassignments.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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 Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                  Map<TopicPartitionReplica, String> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> moveStates = findLogDirMoveStates(adminClient, targetReassignments);
+        System.out.println(replicaMoveStatesToString(moveStates));
+        return new Tuple<>(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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient.
+            describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple<TopicPartition, List<Integer>>> targetParts) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        targetParts.forEach(t -> brokers.addAll(t.v2));
+
+        System.out.printf("Clearing broker-level throttles on broker%s %s%n",
+            brokers.size() == 1 ? "" : "s", brokers.stream().map(Object::toString).collect(Collectors.joining(",")));
+        clearBrokerLevelThrottles(adminClient, brokers);
+
+        Set<String> 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", topics.stream().map(Object::toString).collect(Collectors.joining(",")));
+        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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, 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 Tuple<Map<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                            String reassignmentJson,
+                                                                                                            String brokerListString,
+                                                                                                            Boolean enableRackAwareness) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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",
+            formatAsReassignmentJson(proposedAssignments, Collections.emptyMap()));
+        return new Tuple<>(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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        HashMap<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                       Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics) throws ExecutionException, InterruptedException {
+
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                        Set<TopicPartition> partitions) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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 Tuple<List<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                                  String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> duplicateTopicsToReassign = ToolsUtils.duplicates(topicsToReassign);
+        if (!duplicateTopicsToReassign.isEmpty())
+            throw new AdminCommandFailedException(String.format("List of topics to reassign contains duplicate entries: %s",
+                duplicateTopicsToReassign));
+        return new Tuple<>(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 {
+        Tuple<Map<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> t0 = parseExecuteAssignmentArgs(reassignmentJson);
+
+        Map<TopicPartition, List<Integer>> proposedParts = t0.v1;
+        Map<TopicPartitionReplica, String> proposedReplicas = t0.v2;
+        Map<TopicPartition, PartitionReassignment> 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<Integer> brokers = new HashSet<>();
+        proposedParts.values().forEach(brokers::addAll);
+
+        verifyBrokerIds(adminClient, brokers);
+        Map<TopicPartition, List<Integer>> 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<String, Map<Integer, PartitionMove>> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts);
+                modifyReassignmentThrottle(adminClient, moveMap, interBrokerThrottle);
+            }
+
+            if (logDirThrottle >= 0) {
+                Set<Integer> movingBrokers = calculateMovingBrokers(proposedReplicas.keySet());
+                modifyLogDirThrottle(adminClient, movingBrokers, logDirThrottle);
+            }
+        }
+
+        // Execute the partition reassignments.
+        Map<TopicPartition, Throwable> 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<TopicPartitionReplica, String> proposedReplicas,
+                                     Long timeoutMs,
+                                     Time time) throws InterruptedException, TerseException {
+        long startTimeMs = time.milliseconds();
+        Map<TopicPartitionReplica, String> pendingReplicas = new HashMap<>();
+        pendingReplicas.putAll(proposedReplicas);
+        boolean done = false;
+        do {
+            Set<TopicPartitionReplica> completed = alterReplicaLogDirs(adminClient, pendingReplicas);
+            if (!completed.isEmpty()) {
+                System.out.printf("Successfully started log directory move%s for: %s",
+                    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<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+        String text = currentReassignments.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).map(part -> {
+            PartitionReassignment reassignment = currentReassignments.get(part);
+            List<Integer> replicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+            List<Integer> 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.length() == 0
+            ? "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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Set<Integer> allNodeIds = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Optional<Integer> unknown = brokers.stream().
+            filter(i -> !allNodeIds.contains(i)).

Review Comment:
   rename i -> brokerId



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on a diff in pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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;
+
+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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                              List<Tuple<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.print(partitionReassignmentStatesToString(partStates));
+        return new Tuple<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                           List<Tuple<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+
+            if (currentReassignments.containsKey(part)) {
+                PartitionReassignment reassignment = currentReassignments.get(part);
+                notFoundResults.add(new Tuple<>(part, new PartitionReassignmentState(
+                    reassignment.replicas(),
+                    targetReplicas,
+                    false)));
+            } else {
+                notFoundResults.add(new Tuple<>(part, topicDescriptionFutureToState(part.partition(),
+                    topicDescriptions.get(part.topic()), targetReplicas)));
+            }
+        }
+
+        Map<TopicPartition, PartitionReassignmentState> allResults = new HashMap<>();
+        foundResults.forEach(e -> allResults.put(e.v1, e.v2));
+        notFoundResults.forEach(e -> allResults.put(e.v1, e.v2));
+
+        return new Tuple<>(allResults, currentReassignments.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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 Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                  Map<TopicPartitionReplica, String> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> moveStates = findLogDirMoveStates(adminClient, targetReassignments);
+        System.out.println(replicaMoveStatesToString(moveStates));
+        return new Tuple<>(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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient.
+            describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple<TopicPartition, List<Integer>>> targetParts) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        targetParts.forEach(t -> brokers.addAll(t.v2));
+
+        System.out.printf("Clearing broker-level throttles on broker%s %s%n",
+            brokers.size() == 1 ? "" : "s", brokers.stream().map(Object::toString).collect(Collectors.joining(",")));
+        clearBrokerLevelThrottles(adminClient, brokers);
+
+        Set<String> 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", topics.stream().map(Object::toString).collect(Collectors.joining(",")));
+        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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, 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 Tuple<Map<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                            String reassignmentJson,
+                                                                                                            String brokerListString,
+                                                                                                            Boolean enableRackAwareness) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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",
+            formatAsReassignmentJson(proposedAssignments, Collections.emptyMap()));
+        return new Tuple<>(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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        HashMap<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                       Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics) throws ExecutionException, InterruptedException {
+
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                        Set<TopicPartition> partitions) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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 Tuple<List<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                                  String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> duplicateTopicsToReassign = ToolsUtils.duplicates(topicsToReassign);
+        if (!duplicateTopicsToReassign.isEmpty())
+            throw new AdminCommandFailedException(String.format("List of topics to reassign contains duplicate entries: %s",
+                duplicateTopicsToReassign));
+        return new Tuple<>(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,

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1523 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient
+            .describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, 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<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                             String reassignmentJson,
+                                                                                                             String brokerListString,
+                                                                                                             Boolean enableRackAwareness
+    ) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple2<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        Map<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                                                Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics
+    ) throws ExecutionException, InterruptedException {
+

Review Comment:
   Empty line



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java:
##########
@@ -857,68 +864,18 @@ public void close() {
         }
     }
 
-    private ReassignPartitionsCommand.VerifyAssignmentResult asScala(VerifyAssignmentResult res) {
-        Map<TopicPartition, ReassignPartitionsCommand.PartitionReassignmentState> partStates = new HashMap<>();
-        res.partStates.forEach((tp, state) -> partStates.put(tp, asScala(state)));
-
-        Map<TopicPartitionReplica, ReassignPartitionsCommand.LogDirMoveState> 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 <T> scala.collection.immutable.Set<T> set(final T... set) {
-        return mutableSet(set).toSet();
+    static <T> Set<T> set(final T... set) {
+        return new HashSet<>(Arrays.asList(set));

Review Comment:
   Were we not able to remove the deprecated method below?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   Hello @gharris1727 
   Do you have time to take a look at final patch that transfer `ReassignPartitionCommand` from scala to java?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala:
##########
@@ -1,340 +0,0 @@
-/**

Review Comment:
   Don't find any usages, test or else signs that this class required somehow.
   As far as I can see this is test class that is not used anymore.
   
   Do we really need it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java:
##########
@@ -769,22 +762,12 @@ public void testPropagateInvalidJsonError() {
     }
 
     @SuppressWarnings("unchecked")
-    private static <T> scala.collection.immutable.Set<T> set(final T... set) {
-        return mutableSet(set).toSet();
-    }
-
-    @SuppressWarnings({"deprecation", "unchecked"})
-    private static <T> scala.collection.mutable.Set<T> mutableSet(final T...set) {
-        return JavaConverters.asScalaSet(new HashSet<>(Arrays.asList(set)));
+    private static <T> Set<T> set(final T... set) {

Review Comment:
   Can we remove this as well?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   Thanks @nizhikov for the changes. I'm going let the build run. I will also wait a day or so to see if @mimaison has any more comments.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] ijuma commented on pull request #13247: [WIP] KAFKA-14595 Move value object of ReassignPartitionsCommand to java

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

   Is `server-common` the right module? What requires access to these classes aside from `tools`?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

   @mimaison New classes moved to tools.
   
   Please, share your opinion - should I continue work on this task and move task logic to java?
   Or we will merge current changes and then move logic in separate PR?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on a diff in pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ActiveMoveState.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+/**
+ * A replica log directory move state where the move is in progress.
+ */
+public final class ActiveMoveState implements LogDirMoveState {

Review Comment:
   Done



##########
tools/src/main/java/org/apache/kafka/tools/reassign/ActiveMoveState.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+/**
+ * A replica log directory move state where the move is in progress.
+ */
+public final class ActiveMoveState implements LogDirMoveState {
+    private final String currentLogDir;
+
+    private final String targetLogDir;
+
+    private final String futureLogDir;
+
+    /**
+     * @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.
+     */
+    public ActiveMoveState(String currentLogDir, String targetLogDir, String futureLogDir) {
+        this.currentLogDir = currentLogDir;
+        this.targetLogDir = targetLogDir;
+        this.futureLogDir = futureLogDir;
+    }
+
+    public String currentLogDir() {
+        return currentLogDir;
+    }
+
+    public String targetLogDir() {
+        return targetLogDir;
+    }
+
+    public String futureLogDir() {
+        return futureLogDir;
+    }
+
+    @Override
+    public boolean done() {
+        return false;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        ActiveMoveState that = (ActiveMoveState) o;
+        return Objects.equals(currentLogDir, that.currentLogDir) && Objects.equals(targetLogDir, that.targetLogDir) && Objects.equals(futureLogDir, that.futureLogDir);
+    }
+
+    @Override
+    public int hashCode() {

Review Comment:
   scala case classes generated `equals`, `hashCode`, `toString` automatically.
   So these methods exists in scala version.
   
   Anyway, I removed them. We can implement them when we want.



##########
tools/src/main/java/org/apache/kafka/tools/reassign/CancelledMoveState.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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;
+
+/**
+ * A replica log directory move state where there is no move in progress, but we did not
+ * reach the target log directory.
+ */
+public final class CancelledMoveState implements LogDirMoveState {
+    private final String currentLogDir;
+
+    private final String targetLogDir;
+
+    /**
+     * @param currentLogDir       The current log directory.
+     * @param targetLogDir        The log directory that we wanted the replica to move to.
+     */
+    public CancelledMoveState(String currentLogDir, String targetLogDir) {
+        this.currentLogDir = currentLogDir;
+        this.targetLogDir = targetLogDir;
+    }
+
+    public String currentLogDir() {
+        return currentLogDir;
+    }
+
+    public String targetLogDir() {
+        return targetLogDir;
+    }
+
+    @Override
+    public boolean done() {
+        return true;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        CancelledMoveState that = (CancelledMoveState) o;
+        return Objects.equals(currentLogDir, that.currentLogDir) && Objects.equals(targetLogDir, that.targetLogDir);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(currentLogDir, targetLogDir);
+    }
+
+    @Override
+    public String toString() {
+        return "CancelledMoveState{" +
+            "currentLogDir='" + currentLogDir + '\'' +
+            ", targetLogDir='" + targetLogDir + '\'' +
+            '}';
+    }
+}

Review Comment:
   Fixed



##########
tools/src/main/java/org/apache/kafka/tools/reassign/VerifyAssignmentResult.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartitionReplica;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * A result returned from verifyAssignment.
+ */
+public final class VerifyAssignmentResult {
+    private final Map<TopicPartition, PartitionReassignmentState> partStates;
+    private final boolean partsOngoing;
+    private final Map<TopicPartitionReplica, LogDirMoveState> moveStates;
+    private final boolean movesOngoing;
+
+    public VerifyAssignmentResult(Map<TopicPartition, PartitionReassignmentState> partStates) {
+        this(partStates, false, Collections.emptyMap(), false);
+    }
+
+    /**
+     * @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.
+     */
+    public VerifyAssignmentResult(
+        Map<TopicPartition, PartitionReassignmentState> partStates,
+        boolean partsOngoing,
+        Map<org.apache.kafka.common.TopicPartitionReplica, LogDirMoveState> moveStates,
+        boolean movesOngoing
+    ) {
+        this.partStates = partStates;
+        this.partsOngoing = partsOngoing;
+        this.moveStates = moveStates;
+        this.movesOngoing = movesOngoing;
+    }
+
+    public Map<TopicPartition, PartitionReassignmentState> partStates() {
+        return partStates;
+    }
+
+    public boolean partsOngoing() {
+        return partsOngoing;
+    }
+
+    public Map<TopicPartitionReplica, LogDirMoveState> moveStates() {
+        return moveStates;
+    }
+
+    public boolean movesOngoing() {
+        return movesOngoing;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        VerifyAssignmentResult that = (VerifyAssignmentResult) o;
+        return partsOngoing == that.partsOngoing && movesOngoing == that.movesOngoing && Objects.equals(partStates, that.partStates) && Objects.equals(moveStates, that.moveStates);
+    }
+
+    @Override
+    public int hashCode() {

Review Comment:
   Removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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

   @mimaison Is there any change that PR will be reviewed in current shape? Or I should think on how split it?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java:
##########
@@ -16,7 +16,7 @@
  */

Review Comment:
   I prepared  #14588  that rewrites `ReplicationQuotasTestRig` in java.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   @mimaison 
   @jolshan 
   
   Is there anything a have to do to finish this PR?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java:
##########
@@ -16,7 +16,7 @@
  */

Review Comment:
   No. 
   
   Can you, please, show me an example of similar ticket? So I can create one.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1513 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                             String reassignmentJson,
+                                                                                                             String brokerListString,
+                                                                                                             Boolean enableRackAwareness
+    ) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple2<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        Map<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                                                Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                Set<TopicPartition> partitions
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                           String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> 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<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> t0 = parseExecuteAssignmentArgs(reassignmentJson);
+
+        Map<TopicPartition, List<Integer>> proposedParts = t0.v1;
+        Map<TopicPartitionReplica, String> proposedReplicas = t0.v2;
+        Map<TopicPartition, PartitionReassignment> 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<Integer> brokers = new HashSet<>();
+        proposedParts.values().forEach(brokers::addAll);
+
+        verifyBrokerIds(adminClient, brokers);
+        Map<TopicPartition, List<Integer>> 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<String, Map<Integer, PartitionMove>> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts);
+                modifyReassignmentThrottle(adminClient, moveMap, interBrokerThrottle);
+            }
+
+            if (logDirThrottle >= 0) {
+                Set<Integer> movingBrokers = calculateMovingBrokers(proposedReplicas.keySet());
+                modifyLogDirThrottle(adminClient, movingBrokers, logDirThrottle);
+            }
+        }
+
+        // Execute the partition reassignments.
+        Map<TopicPartition, Throwable> 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<TopicPartitionReplica, String> proposedReplicas,
+                                     Long timeoutMs,
+                                     Time time
+    ) throws InterruptedException, TerseException {
+        long startTimeMs = time.milliseconds();
+        Map<TopicPartitionReplica, String> pendingReplicas = new HashMap<>();
+        pendingReplicas.putAll(proposedReplicas);
+        boolean done = false;
+        do {
+            Set<TopicPartitionReplica> 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<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+        String text = currentReassignments.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).map(part -> {
+            PartitionReassignment reassignment = currentReassignments.get(part);
+            List<Integer> replicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+            List<Integer> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Set<Integer> allNodeIds = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Optional<Integer> 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<TopicPartition, List<Integer>> proposedParts,
+                                                            Map<TopicPartition, List<Integer>> currentParts) throws JsonProcessingException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, Throwable> alterPartitionReassignments(Admin adminClient,
+                                                                      Map<TopicPartition, List<Integer>> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.entrySet().forEach(e -> {
+            TopicPartition part = e.getKey();
+            List<Integer> replicas = e.getValue();
+
+            args.put(part, Optional.of(new NewPartitionReassignment(replicas)));
+        });

Review Comment:
   Fixed. Thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1501 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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";

Review Comment:
   Is it a good idea to duplicate parameter's names constants?
   What about moving `DynamicConfig` and `LogConfig` to `server-commons` project?
   Of course not in the scope of this ticket. But duplicate parameter name constants are potentially dangerous.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1523 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient
+            .describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, 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<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                             String reassignmentJson,
+                                                                                                             String brokerListString,
+                                                                                                             Boolean enableRackAwareness
+    ) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple2<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        Map<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                                                Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics
+    ) throws ExecutionException, InterruptedException {
+
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                Set<TopicPartition> partitions
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                           String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> 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,

Review Comment:
   Original `ReassignPartitionsCommand#executeAssignment` has parameters with default values. Does we have to add set of `executeAssignment` overloaded methods? Does is make sense?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] tledkov commented on a diff in pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/Tuple.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 Tuple<V1, V2> {

Review Comment:
   Minor naming issue. What about  `Tuple2` / `BiTuple` etc? It's just that the name "tuple" suggests a tuple with an arbitrary number of fields.
   Also, can we afford to move this class to `org.apache.kafka.common` or `org.apache.kafka.lang`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

   > We can create the new Java classes in tools and not touch the existing Scala code
   
   Thanks for feedback.
   
   OK. I will move new classes to tools and revert scala code changes made in this PR.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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

   @mimaison Thanks for review and merge of #14172 
   
   The PR still pretty big but touches only 15 classes.
   Are you comfortable with reviewing it as is? 
   
   I can try to move tests of scala version of `ReassignPartitionCommand` to java in separate PR.
   It will require some tricks with scala to java collection but changes will become smaller. 
   
   What do you think?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] tledkov commented on a diff in pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/Tuple.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 Tuple<V1, V2> {

Review Comment:
   Minor naming issue. What about  `Tuple2` / `BiTuple` etc? It's just that the name "tuple" suggests a tuple with an arbitrary number of fields.
   Also, can we afford to move this class to `org.apache.kafka.common` / `org.apache.kafka.utils` or `org.apache.kafka.lang`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> 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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                             String reassignmentJson,
+                                                                                                             String brokerListString,
+                                                                                                             Boolean enableRackAwareness
+    ) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple2<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        Map<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                                                Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                Set<TopicPartition> partitions
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                           String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> 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<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> t0 = parseExecuteAssignmentArgs(reassignmentJson);
+
+        Map<TopicPartition, List<Integer>> proposedParts = t0.v1;
+        Map<TopicPartitionReplica, String> proposedReplicas = t0.v2;
+        Map<TopicPartition, PartitionReassignment> 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<Integer> brokers = new HashSet<>();
+        proposedParts.values().forEach(brokers::addAll);
+
+        verifyBrokerIds(adminClient, brokers);
+        Map<TopicPartition, List<Integer>> 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<String, Map<Integer, PartitionMove>> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts);
+                modifyReassignmentThrottle(adminClient, moveMap, interBrokerThrottle);
+            }
+
+            if (logDirThrottle >= 0) {
+                Set<Integer> movingBrokers = calculateMovingBrokers(proposedReplicas.keySet());
+                modifyLogDirThrottle(adminClient, movingBrokers, logDirThrottle);
+            }
+        }
+
+        // Execute the partition reassignments.
+        Map<TopicPartition, Throwable> 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<TopicPartitionReplica, String> proposedReplicas,
+                                     Long timeoutMs,
+                                     Time time
+    ) throws InterruptedException, TerseException {
+        long startTimeMs = time.milliseconds();
+        Map<TopicPartitionReplica, String> pendingReplicas = new HashMap<>(proposedReplicas);
+        boolean done = false;
+        do {
+            Set<TopicPartitionReplica> 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<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+        String text = currentReassignments.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).map(part -> {
+            PartitionReassignment reassignment = currentReassignments.get(part);
+            List<Integer> replicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+            List<Integer> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Set<Integer> allNodeIds = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Optional<Integer> 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<TopicPartition, List<Integer>> proposedParts,
+                                                            Map<TopicPartition, List<Integer>> currentParts) throws JsonProcessingException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, Throwable> alterPartitionReassignments(Admin adminClient,
+                                                                      Map<TopicPartition, List<Integer>> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.forEach((part, replicas) -> args.put(part, Optional.of(new NewPartitionReassignment(replicas))));
+        Map<TopicPartition, KafkaFuture<Void>> results = adminClient.alterPartitionReassignments(args).values();
+        Map<TopicPartition, Throwable> errors = new HashMap<>();
+        for (Map.Entry<TopicPartition, KafkaFuture<Void>> 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<TopicPartition, Throwable> cancelPartitionReassignments(Admin adminClient,
+                                                                       Set<TopicPartition> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.forEach(part -> args.put(part, Optional.empty()));
+
+        Map<TopicPartition, KafkaFuture<Void>> results = adminClient.alterPartitionReassignments(args).values();
+        Map<TopicPartition, Throwable> errors = new HashMap<>();
+        for (Map.Entry<TopicPartition, KafkaFuture<Void>> 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<String, Map<Integer, PartitionMove>> calculateCurrentMoveMap(Map<TopicPartition, PartitionReassignment> currentReassignments) {
+        Map<String, Map<Integer, PartitionMove>> moveMap = new HashMap<>();
+        // Add the current reassignments to the move map.
+        currentReassignments.forEach((part, reassignment) -> {
+
+            List<Integer> allReplicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+
+            // The addingReplicas is included in the replicas during reassignment
+            Set<Integer> sources = new HashSet<>(allReplicas);
+            addingReplicas.forEach(sources::remove);
+
+            Set<Integer> destinations = new HashSet<>(addingReplicas);
+
+            Map<Integer, PartitionMove> 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<String, Map<Integer, PartitionMove>> calculateProposedMoveMap(Map<TopicPartition, PartitionReassignment> currentReassignments,
+                                                                             Map<TopicPartition, List<Integer>> proposedParts,
+                                                                             Map<TopicPartition, List<Integer>> currentParts) {
+        Map<String, Map<Integer, PartitionMove>> moveMap = calculateCurrentMoveMap(currentReassignments);
+        for (Map.Entry<TopicPartition, List<Integer>> e : proposedParts.entrySet()) {
+            TopicPartition part = e.getKey();
+            List<Integer> replicas = e.getValue();
+            Map<Integer, PartitionMove> 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<Integer> 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<Integer> 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<String, String> calculateLeaderThrottles(Map<String, Map<Integer, PartitionMove>> moveMap) {
+        Map<String, String> results = new HashMap<>();
+        moveMap.forEach((topicName, partMoveMap) -> {
+            Set<String> 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<String, String> calculateFollowerThrottles(Map<String, Map<Integer, PartitionMove>> moveMap) {
+        Map<String, String> results = new HashMap<>();
+        moveMap.forEach((topicName, partMoveMap) -> {
+            Set<String> 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<Integer> calculateReassigningBrokers(Map<String, Map<Integer, PartitionMove>> moveMap) {
+        Set<Integer> 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<Integer> calculateMovingBrokers(Set<TopicPartitionReplica> 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<String, String> leaderThrottles,
+                                     Map<String, String> followerThrottles) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configs = new HashMap<>();
+        Set<String> topicNames = new HashSet<>(leaderThrottles.keySet());
+        topicNames.addAll(followerThrottles.keySet());
+        topicNames.forEach(topicName -> {
+            List<AlterConfigOp> 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<String, Map<Integer, PartitionMove>> moveMap,
+        Long interBrokerThrottle
+    ) throws ExecutionException, InterruptedException {
+        Map<String, String> leaderThrottles = calculateLeaderThrottles(moveMap);
+        Map<String, String> followerThrottles = calculateFollowerThrottles(moveMap);
+        modifyTopicThrottles(admin, leaderThrottles, followerThrottles);
+
+        Set<Integer> 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<Integer> reassigningBrokers,
+                                          long interBrokerThrottle) throws ExecutionException, InterruptedException {
+        if (interBrokerThrottle >= 0) {
+            Map<ConfigResource, Collection<AlterConfigOp>> configs = new HashMap<>();
+            reassigningBrokers.forEach(brokerId -> {
+                List<AlterConfigOp> 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<Integer> movingBrokers,
+                                     long logDirThrottle) throws ExecutionException, InterruptedException {
+        if (logDirThrottle >= 0) {
+            Map<ConfigResource, Collection<AlterConfigOp>> configs = new HashMap<>();
+            movingBrokers.forEach(brokerId -> {
+                List<AlterConfigOp> 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<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> parseExecuteAssignmentArgs(
+        String reassignmentJson
+    ) throws JsonProcessingException {
+        Tuple2<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(reassignmentJson);
+
+        List<Tuple2<TopicPartition, List<Integer>>> partitionsToBeReassigned = t0.v1;
+        Map<TopicPartitionReplica, String> 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<TopicPartition> 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<Tuple2<TopicPartition, Set<Integer>>> 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<TopicPartition>, Set<TopicPartitionReplica>> cancelAssignment(Admin adminClient,
+                                                                                    String jsonString,
+                                                                                    Boolean preserveThrottles,
+                                                                                    Long timeoutMs,
+                                                                                    Time time
+    ) throws ExecutionException, InterruptedException, JsonProcessingException, TerseException {
+        Tuple2<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetReplicas = t0.v2;
+        Set<TopicPartition> targetPartsSet = targetParts.stream().map(t -> t.v1).collect(Collectors.toSet());
+        Set<TopicPartition> 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<TopicPartition, Throwable> 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<TopicPartitionReplica, String> 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<TopicPartition, List<Integer>> partitionsToBeReassigned,
+                                                   Map<TopicPartitionReplica, String> replicaLogDirAssignment) throws JsonProcessingException {
+        List<Map<String, Object>> partitions = new ArrayList<>();
+        partitionsToBeReassigned.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).forEach(tp -> {
+            List<Integer> replicas = partitionsToBeReassigned.get(tp);
+            Map<String, Object> data = new LinkedHashMap<>();

Review Comment:
   Why `LinkedHashMap`? Is there any contract on the order of the fields? Does scala `Map` with `.asJAva` guarantee  keys order?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1501 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1523 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient
+            .describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;

Review Comment:
   What about stream API + Tuple2:
   ```suggestion
           return targetMoves.entrySet().stream()
               .map(e -> {
                   TopicPartitionReplica replica = e.getKey();
                   String targetLogDir = e.getValue();
   
                   if (!replicaLogDirInfos.containsKey(replica)) {
                       return new Tuple2<>(e.getKey(), new MissingReplicaMoveState(targetLogDir));
                   } else {
                       DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
                       if (info.getCurrentReplicaLogDir() == null) {
                           return new Tuple2<>(e.getKey(), new MissingLogDirMoveState(targetLogDir));
                       } else if (info.getFutureReplicaLogDir() == null) {
                           if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
                               return new Tuple2<>(e.getKey(), new CompletedMoveState(targetLogDir));
                           } else {
                               return new Tuple2<>(e.getKey(), new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir));
                           }
                       } else {
                           return new Tuple2<>(e.getKey(), new ActiveMoveState(info.getCurrentReplicaLogDir(),
                               targetLogDir,
                               info.getFutureReplicaLogDir()));
                       }
                   }
               })
               .collect(Collectors.toMap(t -> t.v1, t -> t.v2));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   Sorry @nizhikov for the delay, this is next in my review queue. Can you rebase on trunk to resolve the conflicts? Thanks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java:
##########
@@ -16,7 +16,7 @@
  */

Review Comment:
   Do we have a Jira to remind us to update the few test that are disabled on KRaft once JBOD is available?



##########
core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala:
##########
@@ -1,340 +0,0 @@
-/**

Review Comment:
   What happens to this file? Do we plan to migrate it too or are we deleting it?



##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java:
##########
@@ -769,22 +762,12 @@ public void testPropagateInvalidJsonError() {
     }
 
     @SuppressWarnings("unchecked")
-    private static <T> scala.collection.immutable.Set<T> set(final T... set) {
-        return mutableSet(set).toSet();
-    }
-
-    @SuppressWarnings({"deprecation", "unchecked"})
-    private static <T> scala.collection.mutable.Set<T> mutableSet(final T...set) {
-        return JavaConverters.asScalaSet(new HashSet<>(Arrays.asList(set)));
+    private static <T> Set<T> set(final T... set) {
+        return new HashSet<>(Arrays.asList(set));
     }
 
-    @SuppressWarnings({"deprecation", "unchecked"})
-    private static <T> Seq<T> seq(T... seq) {
-        return JavaConverters.asScalaIteratorConverter(Arrays.asList(seq).iterator()).asScala().toSeq();
-    }
-
-    @SuppressWarnings("deprecation")
-    private static <K, V> scala.collection.Map<K, V> asScala(Map<K, V> jmap) {
-        return JavaConverters.mapAsScalaMap(jmap);
+    @SuppressWarnings("unchecked")

Review Comment:
   Do we really need this? If you're really after saving characters, we could import static Arrays.asList



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java:
##########
@@ -769,22 +762,12 @@ public void testPropagateInvalidJsonError() {
     }
 
     @SuppressWarnings("unchecked")
-    private static <T> scala.collection.immutable.Set<T> set(final T... set) {
-        return mutableSet(set).toSet();
-    }
-
-    @SuppressWarnings({"deprecation", "unchecked"})
-    private static <T> scala.collection.mutable.Set<T> mutableSet(final T...set) {
-        return JavaConverters.asScalaSet(new HashSet<>(Arrays.asList(set)));
+    private static <T> Set<T> set(final T... set) {

Review Comment:
   Were we also not unable to remove these warnings?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   @jolshan Before merging you probably want to run the ReassignPartitionsTest and ThrottlingTest system tests to verify the command still works as expected.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java:
##########
@@ -857,68 +864,13 @@ public void close() {
         }
     }
 
-    private ReassignPartitionsCommand.VerifyAssignmentResult asScala(VerifyAssignmentResult res) {
-        Map<TopicPartition, ReassignPartitionsCommand.PartitionReassignmentState> partStates = new HashMap<>();
-        res.partStates.forEach((tp, state) -> partStates.put(tp, asScala(state)));
-
-        Map<TopicPartitionReplica, ReassignPartitionsCommand.LogDirMoveState> 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 <T> scala.collection.immutable.Set<T> set(final T... set) {
-        return mutableSet(set).toSet();
-    }
-
     @SuppressWarnings({"deprecation", "unchecked"})
     private static <T> scala.collection.mutable.Set<T> mutableSet(final T...set) {
-        return JavaConverters.asScalaSet(new HashSet<>(Arrays.asList(set)));
-    }
-
-    @SuppressWarnings({"unchecked"})
-    private static <T> Seq<T> seq(T... seq) {
-        return seq(Arrays.asList(seq));
+        return JavaConverters.asScalaSet(new HashSet<>(asList(set)));

Review Comment:
   Got it -- so we just have these two methods with deprecated methods? And we need to do more to get these removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   CI looks OK


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala:
##########
@@ -1,340 +0,0 @@
-/**

Review Comment:
   [KAFKA-15645](https://issues.apache.org/jira/browse/KAFKA-15645) created



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   @mimaison Thanks for the hint. I will run system test, also.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on a diff in pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandOptions.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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 joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+
+public class ReassignPartitionsCommandOptions extends CommandDefaultOptions {
+    // Actions
+    private final OptionSpecBuilder verifyOpt;
+    private final OptionSpecBuilder generateOpt;
+    private final OptionSpecBuilder executeOpt;
+    private final OptionSpecBuilder cancelOpt;
+    private final OptionSpecBuilder listOpt;
+
+    // Arguments
+    private final ArgumentAcceptingOptionSpec<String> bootstrapServerOpt;

Review Comment:
   Thanks. fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1523 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient
+            .describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+

Review Comment:
   ```suggestion
           targetMoves.forEach((replica, targetLogDir) -> {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java:
##########
@@ -16,7 +16,7 @@
  */

Review Comment:
   [KAFKA-15646](https://issues.apache.org/jira/browse/KAFKA-15646) created.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala:
##########
@@ -1,340 +0,0 @@
-/**

Review Comment:
   ok -- if you want to make a quick jira, that would be helpful to track. :) 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java:
##########
@@ -769,22 +762,12 @@ public void testPropagateInvalidJsonError() {
     }
 
     @SuppressWarnings("unchecked")
-    private static <T> scala.collection.immutable.Set<T> set(final T... set) {
-        return mutableSet(set).toSet();
-    }
-
-    @SuppressWarnings({"deprecation", "unchecked"})
-    private static <T> scala.collection.mutable.Set<T> mutableSet(final T...set) {
-        return JavaConverters.asScalaSet(new HashSet<>(Arrays.asList(set)));
+    private static <T> Set<T> set(final T... set) {
+        return new HashSet<>(Arrays.asList(set));
     }
 
-    @SuppressWarnings({"deprecation", "unchecked"})
-    private static <T> Seq<T> seq(T... seq) {
-        return JavaConverters.asScalaIteratorConverter(Arrays.asList(seq).iterator()).asScala().toSeq();
-    }
-
-    @SuppressWarnings("deprecation")
-    private static <K, V> scala.collection.Map<K, V> asScala(Map<K, V> jmap) {
-        return JavaConverters.mapAsScalaMap(jmap);
+    @SuppressWarnings("unchecked")

Review Comment:
   Can we get rid of this method now? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

   @mimaison Thanks for the review. I will address your comments in the nearest time.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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

   @mimaison Thanks for review and merge of #14096. 
   
   I create PR that contains java version of ReassignPartitionsCommand POJOs.
   Please, take a look - #14172
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   Note, this PR and #14588 are mutual exclusive.
   After merging one, other must be merged with trunk


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   `throttling_test.py` OK too:
   
   ```
   [INFO:2023-10-20 10:03:55,974]: starting test run with session id 2023-10-20--002...
   [INFO:2023-10-20 10:03:55,974]: running 2 tests...
   [INFO:2023-10-20 10:03:55,974]: Triggering test 1 of 2...
   [INFO:2023-10-20 10:03:55,988]: RunnerClient: Loading test {'directory': '/opt/kafka-dev/tests/kafkatest/tests/core', 'file_name': 'throttling_test.py', 'cls_name': 'ThrottlingTest', 'method_name': 'test_throttled_reassignment', 'injected_args': {'bounce_brokers': False}}
   [INFO:2023-10-20 10:03:55,991]: RunnerClient: kafkatest.tests.core.throttling_test.ThrottlingTest.test_throttled_reassignment.bounce_brokers=False: on run 1/1
   [INFO:2023-10-20 10:03:55,993]: RunnerClient: kafkatest.tests.core.throttling_test.ThrottlingTest.test_throttled_reassignment.bounce_brokers=False: Setting up...
   [INFO:2023-10-20 10:03:59,653]: RunnerClient: kafkatest.tests.core.throttling_test.ThrottlingTest.test_throttled_reassignment.bounce_brokers=False: Running...
   [INFO:2023-10-20 10:09:48,461]: RunnerClient: kafkatest.tests.core.throttling_test.ThrottlingTest.test_throttled_reassignment.bounce_brokers=False: Tearing down...
   [INFO:2023-10-20 10:11:00,903]: RunnerClient: kafkatest.tests.core.throttling_test.ThrottlingTest.test_throttled_reassignment.bounce_brokers=False: PASS
   [INFO:2023-10-20 10:11:00,904]: RunnerClient: kafkatest.tests.core.throttling_test.ThrottlingTest.test_throttled_reassignment.bounce_brokers=False: Data: None
   [INFO:2023-10-20 10:11:00,932]: ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
   [INFO:2023-10-20 10:11:00,933]: Triggering test 2 of 2...
   [INFO:2023-10-20 10:11:00,949]: RunnerClient: Loading test {'directory': '/opt/kafka-dev/tests/kafkatest/tests/core', 'file_name': 'throttling_test.py', 'cls_name': 'ThrottlingTest', 'method_name': 'test_throttled_reassignment', 'injected_args': {'bounce_brokers': True}}
   [INFO:2023-10-20 10:11:00,952]: RunnerClient: kafkatest.tests.core.throttling_test.ThrottlingTest.test_throttled_reassignment.bounce_brokers=True: on run 1/1
   [INFO:2023-10-20 10:11:00,954]: RunnerClient: kafkatest.tests.core.throttling_test.ThrottlingTest.test_throttled_reassignment.bounce_brokers=True: Setting up...
   [INFO:2023-10-20 10:11:04,805]: RunnerClient: kafkatest.tests.core.throttling_test.ThrottlingTest.test_throttled_reassignment.bounce_brokers=True: Running...
   [INFO:2023-10-20 10:18:04,927]: RunnerClient: kafkatest.tests.core.throttling_test.ThrottlingTest.test_throttled_reassignment.bounce_brokers=True: Tearing down...
   [INFO:2023-10-20 10:19:12,329]: RunnerClient: kafkatest.tests.core.throttling_test.ThrottlingTest.test_throttled_reassignment.bounce_brokers=True: PASS
   [INFO:2023-10-20 10:19:12,330]: RunnerClient: kafkatest.tests.core.throttling_test.ThrottlingTest.test_throttled_reassignment.bounce_brokers=True: Data: None
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1501 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {

Review Comment:
   Too long line. Could we format the method like this:
   ```suggestion
       private static Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(
           Admin adminClient,
           List<Tuple2<TopicPartition, List<Integer>>> targets
       ) throws ExecutionException, InterruptedException {
   ```
   or this style is disallow by Kafka code conventions?
   In case the style is allowed, please fix methods below (e.g. `findPartitionReassignmentStates`, `verifyReplicaMoves`, etc) 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1523 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient
+            .describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;

Review Comment:
   Thanks. Great suggestion. I improved it a bit to omit intermidiate tuples:
   ```
           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());
           }));
   
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1513 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                             String reassignmentJson,
+                                                                                                             String brokerListString,
+                                                                                                             Boolean enableRackAwareness
+    ) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple2<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        Map<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                                                Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                Set<TopicPartition> partitions
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                           String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> 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<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> t0 = parseExecuteAssignmentArgs(reassignmentJson);
+
+        Map<TopicPartition, List<Integer>> proposedParts = t0.v1;
+        Map<TopicPartitionReplica, String> proposedReplicas = t0.v2;
+        Map<TopicPartition, PartitionReassignment> 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<Integer> brokers = new HashSet<>();
+        proposedParts.values().forEach(brokers::addAll);
+
+        verifyBrokerIds(adminClient, brokers);
+        Map<TopicPartition, List<Integer>> 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<String, Map<Integer, PartitionMove>> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts);
+                modifyReassignmentThrottle(adminClient, moveMap, interBrokerThrottle);
+            }
+
+            if (logDirThrottle >= 0) {
+                Set<Integer> movingBrokers = calculateMovingBrokers(proposedReplicas.keySet());
+                modifyLogDirThrottle(adminClient, movingBrokers, logDirThrottle);
+            }
+        }
+
+        // Execute the partition reassignments.
+        Map<TopicPartition, Throwable> 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<TopicPartitionReplica, String> proposedReplicas,
+                                     Long timeoutMs,
+                                     Time time
+    ) throws InterruptedException, TerseException {
+        long startTimeMs = time.milliseconds();
+        Map<TopicPartitionReplica, String> pendingReplicas = new HashMap<>();
+        pendingReplicas.putAll(proposedReplicas);
+        boolean done = false;
+        do {
+            Set<TopicPartitionReplica> 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);
+            }
+
+

Review Comment:
   Two empty lines



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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

   It seems tests are OK in #14217 
   Please, review.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on a diff in pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/TerseReassignmentFailureException.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * 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 org.apache.kafka.common.KafkaException;
+
+/**
+ * An exception thrown to indicate that the command has failed, but we don't want to
+ * print a stack trace.
+ */
+public class TerseReassignmentFailureException extends KafkaException {

Review Comment:
   I think the scope of ticket is move scala code to java without changes.
   `TerseReassignmentFailureException` used in `ReassignPartitionsCommand` so it seems class should stay as is.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

   Tests failures seems unrelated.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on a diff in pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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;
+
+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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                              List<Tuple<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.print(partitionReassignmentStatesToString(partStates));
+        return new Tuple<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                           List<Tuple<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+
+            if (currentReassignments.containsKey(part)) {
+                PartitionReassignment reassignment = currentReassignments.get(part);
+                notFoundResults.add(new Tuple<>(part, new PartitionReassignmentState(
+                    reassignment.replicas(),
+                    targetReplicas,
+                    false)));
+            } else {
+                notFoundResults.add(new Tuple<>(part, topicDescriptionFutureToState(part.partition(),
+                    topicDescriptions.get(part.topic()), targetReplicas)));
+            }
+        }
+
+        Map<TopicPartition, PartitionReassignmentState> allResults = new HashMap<>();
+        foundResults.forEach(e -> allResults.put(e.v1, e.v2));
+        notFoundResults.forEach(e -> allResults.put(e.v1, e.v2));
+
+        return new Tuple<>(allResults, currentReassignments.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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 Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                  Map<TopicPartitionReplica, String> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> moveStates = findLogDirMoveStates(adminClient, targetReassignments);
+        System.out.println(replicaMoveStatesToString(moveStates));
+        return new Tuple<>(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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient.
+            describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple<TopicPartition, List<Integer>>> targetParts) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        targetParts.forEach(t -> brokers.addAll(t.v2));
+
+        System.out.printf("Clearing broker-level throttles on broker%s %s%n",
+            brokers.size() == 1 ? "" : "s", brokers.stream().map(Object::toString).collect(Collectors.joining(",")));
+        clearBrokerLevelThrottles(adminClient, brokers);
+
+        Set<String> 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", topics.stream().map(Object::toString).collect(Collectors.joining(",")));
+        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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, 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 Tuple<Map<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                            String reassignmentJson,
+                                                                                                            String brokerListString,
+                                                                                                            Boolean enableRackAwareness) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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",
+            formatAsReassignmentJson(proposedAssignments, Collections.emptyMap()));
+        return new Tuple<>(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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        HashMap<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                       Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics) throws ExecutionException, InterruptedException {
+
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                        Set<TopicPartition> partitions) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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 Tuple<List<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                                  String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> duplicateTopicsToReassign = ToolsUtils.duplicates(topicsToReassign);
+        if (!duplicateTopicsToReassign.isEmpty())
+            throw new AdminCommandFailedException(String.format("List of topics to reassign contains duplicate entries: %s",
+                duplicateTopicsToReassign));
+        return new Tuple<>(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 {
+        Tuple<Map<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> t0 = parseExecuteAssignmentArgs(reassignmentJson);
+
+        Map<TopicPartition, List<Integer>> proposedParts = t0.v1;
+        Map<TopicPartitionReplica, String> proposedReplicas = t0.v2;
+        Map<TopicPartition, PartitionReassignment> 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<Integer> brokers = new HashSet<>();
+        proposedParts.values().forEach(brokers::addAll);
+
+        verifyBrokerIds(adminClient, brokers);
+        Map<TopicPartition, List<Integer>> 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<String, Map<Integer, PartitionMove>> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts);
+                modifyReassignmentThrottle(adminClient, moveMap, interBrokerThrottle);
+            }
+
+            if (logDirThrottle >= 0) {
+                Set<Integer> movingBrokers = calculateMovingBrokers(proposedReplicas.keySet());
+                modifyLogDirThrottle(adminClient, movingBrokers, logDirThrottle);
+            }
+        }
+
+        // Execute the partition reassignments.
+        Map<TopicPartition, Throwable> 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<TopicPartitionReplica, String> proposedReplicas,
+                                     Long timeoutMs,
+                                     Time time) throws InterruptedException, TerseException {
+        long startTimeMs = time.milliseconds();
+        Map<TopicPartitionReplica, String> pendingReplicas = new HashMap<>();
+        pendingReplicas.putAll(proposedReplicas);
+        boolean done = false;
+        do {
+            Set<TopicPartitionReplica> completed = alterReplicaLogDirs(adminClient, pendingReplicas);
+            if (!completed.isEmpty()) {
+                System.out.printf("Successfully started log directory move%s for: %s",
+                    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<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+        String text = currentReassignments.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).map(part -> {
+            PartitionReassignment reassignment = currentReassignments.get(part);
+            List<Integer> replicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+            List<Integer> 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.length() == 0
+            ? "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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Set<Integer> allNodeIds = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Optional<Integer> unknown = brokers.stream().
+            filter(i -> !allNodeIds.contains(i)).
+            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<TopicPartition, List<Integer>> proposedParts,
+                                                            Map<TopicPartition, List<Integer>> currentParts) throws JsonProcessingException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, Throwable> alterPartitionReassignments(Admin adminClient,
+                                                                      Map<TopicPartition, List<Integer>> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.entrySet().forEach(e -> {
+            TopicPartition part = e.getKey();
+            List<Integer> replicas = e.getValue();
+
+            args.put(part, Optional.of(new NewPartitionReassignment(replicas)));
+        });
+        Map<TopicPartition, KafkaFuture<Void>> results = adminClient.alterPartitionReassignments(args).values();
+        Map<TopicPartition, Throwable> errors = new HashMap<>();
+        for (Map.Entry<TopicPartition, KafkaFuture<Void>> 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<TopicPartition, Throwable> cancelPartitionReassignments(Admin adminClient,
+                                                                               Set<TopicPartition> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.forEach(part -> args.put(part, Optional.empty()));
+
+        Map<TopicPartition, KafkaFuture<Void>> results = adminClient.alterPartitionReassignments(args).values();
+        Map<TopicPartition, Throwable> errors = new HashMap<>();
+        for (Map.Entry<TopicPartition, KafkaFuture<Void>> 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<String, Map<Integer, PartitionMove>> calculateCurrentMoveMap(Map<TopicPartition, PartitionReassignment> currentReassignments) {
+        Map<String, Map<Integer, PartitionMove>> moveMap = new HashMap<>();
+        // Add the current reassignments to the move map.
+        currentReassignments.entrySet().forEach(e -> {
+            TopicPartition part = e.getKey();
+            PartitionReassignment reassignment = e.getValue();
+
+            List<Integer> allReplicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+
+            // The addingReplicas is included in the replicas during reassignment
+            Set<Integer> sources = new HashSet<>(allReplicas);
+            sources.removeAll(addingReplicas);
+
+            Set<Integer> destinations = new HashSet<>(addingReplicas);
+
+            Map<Integer, PartitionMove> 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<String, Map<Integer, PartitionMove>> calculateProposedMoveMap(Map<TopicPartition, PartitionReassignment> currentReassignments,
+                                                                             Map<TopicPartition, List<Integer>> proposedParts,
+                                                                             Map<TopicPartition, List<Integer>> currentParts) {
+        Map<String, Map<Integer, PartitionMove>> moveMap = calculateCurrentMoveMap(currentReassignments);
+        for (Map.Entry<TopicPartition, List<Integer>> e : proposedParts.entrySet()) {
+            TopicPartition part = e.getKey();
+            List<Integer> replicas = e.getValue();
+            Map<Integer, PartitionMove> 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<Integer> 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<Integer> 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<String, String> calculateLeaderThrottles(Map<String, Map<Integer, PartitionMove>> moveMap) {
+        Map<String, String> results = new HashMap<>();
+        moveMap.forEach((topicName, partMoveMap) -> {
+            Set<String> 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<String, String> calculateFollowerThrottles(Map<String, Map<Integer, PartitionMove>> moveMap) {
+        Map<String, String> results = new HashMap<>();
+        moveMap.forEach((topicName, partMoveMap) -> {
+            Set<String> 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<Integer> calculateReassigningBrokers(Map<String, Map<Integer, PartitionMove>> moveMap) {
+        Set<Integer> 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<Integer> calculateMovingBrokers(Set<TopicPartitionReplica> 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<String, String> leaderThrottles,
+                                     Map<String, String> followerThrottles) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configs = new HashMap<>();
+        Set<String> topicNames = new HashSet<>(leaderThrottles.keySet());
+        topicNames.addAll(followerThrottles.keySet());
+        topicNames.forEach(topicName -> {
+            List<AlterConfigOp> 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<String, Map<Integer, PartitionMove>> moveMap, Long interBrokerThrottle) throws ExecutionException, InterruptedException {
+        Map<String, String> leaderThrottles = calculateLeaderThrottles(moveMap);
+        Map<String, String> followerThrottles = calculateFollowerThrottles(moveMap);
+        modifyTopicThrottles(admin, leaderThrottles, followerThrottles);
+
+        Set<Integer> 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<Integer> reassigningBrokers,
+                                          long interBrokerThrottle) throws ExecutionException, InterruptedException {
+        if (interBrokerThrottle >= 0) {
+            Map<ConfigResource, Collection<AlterConfigOp>> configs = new HashMap<>();
+            reassigningBrokers.forEach(brokerId -> {
+                List<AlterConfigOp> 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<Integer> movingBrokers,
+                                     long logDirThrottle) throws ExecutionException, InterruptedException {
+        if (logDirThrottle >= 0) {
+            Map<ConfigResource, Collection<AlterConfigOp>> configs = new HashMap<>();
+            movingBrokers.forEach(brokerId -> {
+                List<AlterConfigOp> 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 Tuple<Map<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> parseExecuteAssignmentArgs(String reassignmentJson) throws JsonProcessingException {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(reassignmentJson);
+
+        List<Tuple<TopicPartition, List<Integer>>> partitionsToBeReassigned = t0.v1;
+        Map<TopicPartitionReplica, String> 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<TopicPartition> 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<Tuple<TopicPartition, Set<Integer>>> duplicateEntries = partitionsToBeReassigned.stream()
+            .map(t -> new Tuple<>(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 Tuple<>(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 Tuple<Set<TopicPartition>, Set<TopicPartitionReplica>> cancelAssignment(Admin adminClient,
+                                                                                   String jsonString,
+                                                                                   Boolean preserveThrottles,
+                                                                                   Long timeoutMs,
+                                                                                   Time time) throws ExecutionException, InterruptedException, JsonProcessingException, TerseException {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetReplicas = t0.v2;
+        Set<TopicPartition> targetPartsSet = targetParts.stream().map(t -> t.v1).collect(Collectors.toSet());
+        Set<TopicPartition> 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<TopicPartition, Throwable> 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",

Review Comment:
   Good catch. Thanks! Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on a diff in pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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;
+
+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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                              List<Tuple<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.print(partitionReassignmentStatesToString(partStates));
+        return new Tuple<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                           List<Tuple<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+
+            if (currentReassignments.containsKey(part)) {
+                PartitionReassignment reassignment = currentReassignments.get(part);
+                notFoundResults.add(new Tuple<>(part, new PartitionReassignmentState(
+                    reassignment.replicas(),
+                    targetReplicas,
+                    false)));
+            } else {
+                notFoundResults.add(new Tuple<>(part, topicDescriptionFutureToState(part.partition(),
+                    topicDescriptions.get(part.topic()), targetReplicas)));
+            }
+        }
+
+        Map<TopicPartition, PartitionReassignmentState> allResults = new HashMap<>();
+        foundResults.forEach(e -> allResults.put(e.v1, e.v2));
+        notFoundResults.forEach(e -> allResults.put(e.v1, e.v2));
+
+        return new Tuple<>(allResults, currentReassignments.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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 Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                  Map<TopicPartitionReplica, String> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> moveStates = findLogDirMoveStates(adminClient, targetReassignments);
+        System.out.println(replicaMoveStatesToString(moveStates));
+        return new Tuple<>(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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient.
+            describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple<TopicPartition, List<Integer>>> targetParts) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        targetParts.forEach(t -> brokers.addAll(t.v2));
+
+        System.out.printf("Clearing broker-level throttles on broker%s %s%n",
+            brokers.size() == 1 ? "" : "s", brokers.stream().map(Object::toString).collect(Collectors.joining(",")));
+        clearBrokerLevelThrottles(adminClient, brokers);
+
+        Set<String> 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", topics.stream().map(Object::toString).collect(Collectors.joining(",")));
+        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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, 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 Tuple<Map<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                            String reassignmentJson,
+                                                                                                            String brokerListString,
+                                                                                                            Boolean enableRackAwareness) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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",
+            formatAsReassignmentJson(proposedAssignments, Collections.emptyMap()));
+        return new Tuple<>(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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        HashMap<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                       Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics) throws ExecutionException, InterruptedException {
+
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                        Set<TopicPartition> partitions) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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 Tuple<List<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                                  String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> duplicateTopicsToReassign = ToolsUtils.duplicates(topicsToReassign);
+        if (!duplicateTopicsToReassign.isEmpty())
+            throw new AdminCommandFailedException(String.format("List of topics to reassign contains duplicate entries: %s",
+                duplicateTopicsToReassign));
+        return new Tuple<>(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 {
+        Tuple<Map<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> t0 = parseExecuteAssignmentArgs(reassignmentJson);
+
+        Map<TopicPartition, List<Integer>> proposedParts = t0.v1;
+        Map<TopicPartitionReplica, String> proposedReplicas = t0.v2;
+        Map<TopicPartition, PartitionReassignment> 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<Integer> brokers = new HashSet<>();
+        proposedParts.values().forEach(brokers::addAll);
+
+        verifyBrokerIds(adminClient, brokers);
+        Map<TopicPartition, List<Integer>> 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<String, Map<Integer, PartitionMove>> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts);
+                modifyReassignmentThrottle(adminClient, moveMap, interBrokerThrottle);
+            }
+
+            if (logDirThrottle >= 0) {
+                Set<Integer> movingBrokers = calculateMovingBrokers(proposedReplicas.keySet());
+                modifyLogDirThrottle(adminClient, movingBrokers, logDirThrottle);
+            }
+        }
+
+        // Execute the partition reassignments.
+        Map<TopicPartition, Throwable> 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<TopicPartitionReplica, String> proposedReplicas,
+                                     Long timeoutMs,
+                                     Time time) throws InterruptedException, TerseException {
+        long startTimeMs = time.milliseconds();
+        Map<TopicPartitionReplica, String> pendingReplicas = new HashMap<>();
+        pendingReplicas.putAll(proposedReplicas);
+        boolean done = false;
+        do {
+            Set<TopicPartitionReplica> completed = alterReplicaLogDirs(adminClient, pendingReplicas);
+            if (!completed.isEmpty()) {
+                System.out.printf("Successfully started log directory move%s for: %s",
+                    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<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+        String text = currentReassignments.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).map(part -> {
+            PartitionReassignment reassignment = currentReassignments.get(part);
+            List<Integer> replicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+            List<Integer> 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.length() == 0
+            ? "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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Set<Integer> allNodeIds = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Optional<Integer> unknown = brokers.stream().
+            filter(i -> !allNodeIds.contains(i)).

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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

   @mimaison Tests in #14456 are OK. If you have some time, please, join the review.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1501 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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";

Review Comment:
   I think it's a gread idea. Let's discuss it on the dev-list.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1523 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient
+            .describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, 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<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                             String reassignmentJson,
+                                                                                                             String brokerListString,
+                                                                                                             Boolean enableRackAwareness
+    ) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple2<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        Map<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                                                Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics
+    ) throws ExecutionException, InterruptedException {
+
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                Set<TopicPartition> partitions
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                           String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> 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,

Review Comment:
   I don't think so. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
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<V1, V2> {
+    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 "Tuple{v1=" + v1 + ", v2=" + v2 + '}';

Review Comment:
   What about using class name `Tuple2` at the `toString()` after rename the class?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1501 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {

Review Comment:
   Too long line. Could we format the method like this:
   ```suggestion
       private static Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(
           Admin adminClient,
           List<Tuple2<TopicPartition, List<Integer>>> targets
       ) throws ExecutionException, InterruptedException {
   ```
   or this style is disallow by Kafka code conventions?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1501 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {

Review Comment:
   Long line. Please format arguments



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala:
##########
@@ -1,340 +0,0 @@
-/**

Review Comment:
   This looks like a tool that would be run externally. You could run it locally to test, but looks like you need 40GB(!) to do so. 
   
   Not sure how many folks use it regularly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   Hello @mimaison 
   I merged trunk into branch. Conflict resolved.
   Tests for command pass locally.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

   @mimaison Please, share your feedback. Are you ready to review and merge current changes (only case classes and options moved to java code) or should I continue work and rewrite the whole command in Java?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] fvaleri commented on a diff in pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandOptions.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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 joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+
+public class ReassignPartitionsCommandOptions extends CommandDefaultOptions {
+    // Actions
+    private final OptionSpecBuilder verifyOpt;
+    private final OptionSpecBuilder generateOpt;
+    private final OptionSpecBuilder executeOpt;
+    private final OptionSpecBuilder cancelOpt;
+    private final OptionSpecBuilder listOpt;
+
+    // Arguments
+    private final ArgumentAcceptingOptionSpec<String> bootstrapServerOpt;

Review Comment:
   For consistency with the other tools, I would use the `OptionSpec` superclass.



##########
tools/src/main/java/org/apache/kafka/tools/reassign/TerseReassignmentFailureException.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * 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 org.apache.kafka.common.KafkaException;
+
+/**
+ * An exception thrown to indicate that the command has failed, but we don't want to
+ * print a stack trace.
+ */
+public class TerseReassignmentFailureException extends KafkaException {

Review Comment:
   Do we really need this? Can't we simply use `TerseException`?



##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandOptions.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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 joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+
+public class ReassignPartitionsCommandOptions extends CommandDefaultOptions {
+    // Actions
+    private final OptionSpecBuilder verifyOpt;

Review Comment:
   For consistency with the other tools, I would use the `OptionSpec` superclass.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1513 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                             String reassignmentJson,
+                                                                                                             String brokerListString,
+                                                                                                             Boolean enableRackAwareness
+    ) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple2<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        Map<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                                                Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                Set<TopicPartition> partitions
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                           String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> 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<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> t0 = parseExecuteAssignmentArgs(reassignmentJson);
+
+        Map<TopicPartition, List<Integer>> proposedParts = t0.v1;
+        Map<TopicPartitionReplica, String> proposedReplicas = t0.v2;
+        Map<TopicPartition, PartitionReassignment> 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<Integer> brokers = new HashSet<>();
+        proposedParts.values().forEach(brokers::addAll);
+
+        verifyBrokerIds(adminClient, brokers);
+        Map<TopicPartition, List<Integer>> 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<String, Map<Integer, PartitionMove>> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts);
+                modifyReassignmentThrottle(adminClient, moveMap, interBrokerThrottle);
+            }
+
+            if (logDirThrottle >= 0) {
+                Set<Integer> movingBrokers = calculateMovingBrokers(proposedReplicas.keySet());
+                modifyLogDirThrottle(adminClient, movingBrokers, logDirThrottle);
+            }
+        }
+
+        // Execute the partition reassignments.
+        Map<TopicPartition, Throwable> 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<TopicPartitionReplica, String> proposedReplicas,
+                                     Long timeoutMs,
+                                     Time time
+    ) throws InterruptedException, TerseException {
+        long startTimeMs = time.milliseconds();
+        Map<TopicPartitionReplica, String> pendingReplicas = new HashMap<>();
+        pendingReplicas.putAll(proposedReplicas);
+        boolean done = false;
+        do {
+            Set<TopicPartitionReplica> 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<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+        String text = currentReassignments.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).map(part -> {
+            PartitionReassignment reassignment = currentReassignments.get(part);
+            List<Integer> replicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+            List<Integer> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Set<Integer> allNodeIds = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Optional<Integer> 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<TopicPartition, List<Integer>> proposedParts,
+                                                            Map<TopicPartition, List<Integer>> currentParts) throws JsonProcessingException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, Throwable> alterPartitionReassignments(Admin adminClient,
+                                                                      Map<TopicPartition, List<Integer>> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.entrySet().forEach(e -> {
+            TopicPartition part = e.getKey();
+            List<Integer> replicas = e.getValue();
+
+            args.put(part, Optional.of(new NewPartitionReassignment(replicas)));
+        });
+        Map<TopicPartition, KafkaFuture<Void>> results = adminClient.alterPartitionReassignments(args).values();
+        Map<TopicPartition, Throwable> errors = new HashMap<>();
+        for (Map.Entry<TopicPartition, KafkaFuture<Void>> 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<TopicPartition, Throwable> cancelPartitionReassignments(Admin adminClient,
+                                                                       Set<TopicPartition> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.forEach(part -> args.put(part, Optional.empty()));
+
+        Map<TopicPartition, KafkaFuture<Void>> results = adminClient.alterPartitionReassignments(args).values();
+        Map<TopicPartition, Throwable> errors = new HashMap<>();
+        for (Map.Entry<TopicPartition, KafkaFuture<Void>> 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<String, Map<Integer, PartitionMove>> calculateCurrentMoveMap(Map<TopicPartition, PartitionReassignment> currentReassignments) {
+        Map<String, Map<Integer, PartitionMove>> moveMap = new HashMap<>();
+        // Add the current reassignments to the move map.
+        currentReassignments.entrySet().forEach(e -> {
+            TopicPartition part = e.getKey();
+            PartitionReassignment reassignment = e.getValue();
+
+            List<Integer> allReplicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+
+            // The addingReplicas is included in the replicas during reassignment
+            Set<Integer> sources = new HashSet<>(allReplicas);
+            sources.removeAll(addingReplicas);
+
+            Set<Integer> destinations = new HashSet<>(addingReplicas);
+
+            Map<Integer, PartitionMove> partMoves = moveMap.computeIfAbsent(part.topic(), k -> new HashMap<>());
+            partMoves.put(part.partition(), new PartitionMove(sources, destinations));
+        });

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1501 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.println(partitionReassignmentStatesToString(partStates));
+        return new Tuple2<>(partStates, partsOngoing);

Review Comment:
   Looks like the code is similar to:
   ```suggestion
           Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
   
           Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
           Boolean partsOngoing = t0.v2;
   
           System.out.println(partitionReassignmentStatesToString(partStates));
           return new Tuple2<>(partStates, partsOngoing);
   ```
   ```suggestion
           Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
   
           System.out.println(partitionReassignmentStatesToString(t0.v1));
   
           return t0;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1523 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient
+            .describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, 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<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                             String reassignmentJson,
+                                                                                                             String brokerListString,
+                                                                                                             Boolean enableRackAwareness
+    ) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple2<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        Map<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                                                Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics
+    ) throws ExecutionException, InterruptedException {
+
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                Set<TopicPartition> partitions
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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());

Review Comment:
   Please move .collect to new line to align with other Stream methods.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1523 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient
+            .describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, topicName),
+            TOPIC_LEVEL_THROTTLES.stream().map(throttle -> new AlterConfigOp(new ConfigEntry(throttle, null),
+                AlterConfigOp.OpType.DELETE)).collect(Collectors.toList())
+
+        ));

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java:
##########
@@ -769,22 +762,12 @@ public void testPropagateInvalidJsonError() {
     }
 
     @SuppressWarnings("unchecked")
-    private static <T> scala.collection.immutable.Set<T> set(final T... set) {
-        return mutableSet(set).toSet();
-    }
-
-    @SuppressWarnings({"deprecation", "unchecked"})
-    private static <T> scala.collection.mutable.Set<T> mutableSet(final T...set) {
-        return JavaConverters.asScalaSet(new HashSet<>(Arrays.asList(set)));
+    private static <T> Set<T> set(final T... set) {

Review Comment:
   Fixed. Yes, we able to remove these method. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java:
##########
@@ -857,68 +864,13 @@ public void close() {
         }
     }
 
-    private ReassignPartitionsCommand.VerifyAssignmentResult asScala(VerifyAssignmentResult res) {
-        Map<TopicPartition, ReassignPartitionsCommand.PartitionReassignmentState> partStates = new HashMap<>();
-        res.partStates.forEach((tp, state) -> partStates.put(tp, asScala(state)));
-
-        Map<TopicPartitionReplica, ReassignPartitionsCommand.LogDirMoveState> 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 <T> scala.collection.immutable.Set<T> set(final T... set) {
-        return mutableSet(set).toSet();
-    }
-
     @SuppressWarnings({"deprecation", "unchecked"})
     private static <T> scala.collection.mutable.Set<T> mutableSet(final T...set) {
-        return JavaConverters.asScalaSet(new HashSet<>(Arrays.asList(set)));
-    }
-
-    @SuppressWarnings({"unchecked"})
-    private static <T> Seq<T> seq(T... seq) {
-        return seq(Arrays.asList(seq));
+        return JavaConverters.asScalaSet(new HashSet<>(asList(set)));

Review Comment:
   Yes. We need either use other parent class for this test or java collections in QuorumTestHarness



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java:
##########
@@ -769,22 +762,12 @@ public void testPropagateInvalidJsonError() {
     }
 
     @SuppressWarnings("unchecked")
-    private static <T> scala.collection.immutable.Set<T> set(final T... set) {
-        return mutableSet(set).toSet();
-    }
-
-    @SuppressWarnings({"deprecation", "unchecked"})
-    private static <T> scala.collection.mutable.Set<T> mutableSet(final T...set) {
-        return JavaConverters.asScalaSet(new HashSet<>(Arrays.asList(set)));
+    private static <T> Set<T> set(final T... set) {
+        return new HashSet<>(Arrays.asList(set));
     }
 
-    @SuppressWarnings({"deprecation", "unchecked"})
-    private static <T> Seq<T> seq(T... seq) {
-        return JavaConverters.asScalaIteratorConverter(Arrays.asList(seq).iterator()).asScala().toSeq();
-    }
-
-    @SuppressWarnings("deprecation")
-    private static <K, V> scala.collection.Map<K, V> asScala(Map<K, V> jmap) {
-        return JavaConverters.mapAsScalaMap(jmap);
+    @SuppressWarnings("unchecked")

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java:
##########
@@ -16,7 +16,7 @@
  */

Review Comment:
   You can create a JIRA ticket just mentioning the tests (maybe linking to the code that was disabled) and saying we should update when JBOD is available. Doesn't need to be too formal. 
   
   You may want to target 3.7 since we want to get JBOD done by then.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   `reassign_partitions_test.py` is OK. Will check second in a minute.
   
   ```
   $ git rev-parse HEAD
   53e767a75f2d48c4e5419fdf83033f881f21771e
   
   docker exec ducker01 bash -c "cd /opt/kafka-dev && ducktape --cluster-file /opt/kafka-dev/tests/docker/build/cluster.json  ./tests/kafkatest/tests/core/reassign_partitions_test.py "
   /usr/local/lib/python3.9/dist-packages/paramiko/transport.py:236: CryptographyDeprecationWarning: Blowfish has been deprecated
     "class": algorithms.Blowfish,
   [INFO:2023-10-20 09:36:39,559]: starting test run with session id 2023-10-20--001...
   [INFO:2023-10-20 09:36:39,559]: running 8 tests...
   [INFO:2023-10-20 09:36:39,560]: Triggering test 1 of 8...
   [INFO:2023-10-20 09:36:39,572]: RunnerClient: Loading test {'directory': '/opt/kafka-dev/tests/kafkatest/tests/core', 'file_name': 'reassign_partitions_test.py', 'cls_name': 'ReassignPartitionsTest', 'method_name': 'test_reassign_partitions', 'injected_args': {'bounce_brokers': False, 'reassign_from_offset_zero': False, 'metadata_quorum': 'ISOLATED_KRAFT'}}
   [INFO:2023-10-20 09:36:39,576]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=False.metadata_quorum=ISOLATED_KRAFT: on run 1/1
   [INFO:2023-10-20 09:36:39,578]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=False.metadata_quorum=ISOLATED_KRAFT: Setting up...
   [INFO:2023-10-20 09:36:39,578]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=False.metadata_quorum=ISOLATED_KRAFT: Running...
   [INFO:2023-10-20 09:39:03,424]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=False.metadata_quorum=ISOLATED_KRAFT: Tearing down...
   [INFO:2023-10-20 09:39:27,516]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=False.metadata_quorum=ISOLATED_KRAFT: PASS
   [INFO:2023-10-20 09:39:27,517]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=False.metadata_quorum=ISOLATED_KRAFT: Data: None
   [INFO:2023-10-20 09:39:27,534]: ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
   [INFO:2023-10-20 09:39:27,534]: Triggering test 2 of 8...
   [INFO:2023-10-20 09:39:27,545]: RunnerClient: Loading test {'directory': '/opt/kafka-dev/tests/kafkatest/tests/core', 'file_name': 'reassign_partitions_test.py', 'cls_name': 'ReassignPartitionsTest', 'method_name': 'test_reassign_partitions', 'injected_args': {'bounce_brokers': False, 'reassign_from_offset_zero': False, 'metadata_quorum': 'ZK'}}
   [INFO:2023-10-20 09:39:27,551]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=False.metadata_quorum=ZK: on run 1/1
   [INFO:2023-10-20 09:39:27,553]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=False.metadata_quorum=ZK: Setting up...
   [INFO:2023-10-20 09:39:31,390]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=False.metadata_quorum=ZK: Running...
   [INFO:2023-10-20 09:41:44,682]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=False.metadata_quorum=ZK: Tearing down...
   [INFO:2023-10-20 09:42:20,942]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=False.metadata_quorum=ZK: PASS
   [INFO:2023-10-20 09:42:20,944]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=False.metadata_quorum=ZK: Data: None
   [INFO:2023-10-20 09:42:20,962]: ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
   [INFO:2023-10-20 09:42:20,963]: Triggering test 3 of 8...
   [INFO:2023-10-20 09:42:20,983]: RunnerClient: Loading test {'directory': '/opt/kafka-dev/tests/kafkatest/tests/core', 'file_name': 'reassign_partitions_test.py', 'cls_name': 'ReassignPartitionsTest', 'method_name': 'test_reassign_partitions', 'injected_args': {'bounce_brokers': False, 'reassign_from_offset_zero': True, 'metadata_quorum': 'ISOLATED_KRAFT'}}
   [INFO:2023-10-20 09:42:20,987]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: on run 1/1
   [INFO:2023-10-20 09:42:20,989]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: Setting up...
   [INFO:2023-10-20 09:42:20,989]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: Running...
   [INFO:2023-10-20 09:45:01,375]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: Tearing down...
   [INFO:2023-10-20 09:45:24,418]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: PASS
   [WARNING - 2023-10-20 09:45:24,419 - runner_client - log - lineno:294]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: Test requested 8 nodes, used only 7
   [WARNING:2023-10-20 09:45:24,419]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: Test requested 8 nodes, used only 7
   [INFO:2023-10-20 09:45:24,420]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: Data: None
   [INFO:2023-10-20 09:45:24,441]: ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
   [INFO:2023-10-20 09:45:24,441]: Triggering test 4 of 8...
   [INFO:2023-10-20 09:45:24,456]: RunnerClient: Loading test {'directory': '/opt/kafka-dev/tests/kafkatest/tests/core', 'file_name': 'reassign_partitions_test.py', 'cls_name': 'ReassignPartitionsTest', 'method_name': 'test_reassign_partitions', 'injected_args': {'bounce_brokers': False, 'reassign_from_offset_zero': True, 'metadata_quorum': 'ZK'}}
   [INFO:2023-10-20 09:45:24,461]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ZK: on run 1/1
   [INFO:2023-10-20 09:45:24,464]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ZK: Setting up...
   [INFO:2023-10-20 09:45:28,235]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ZK: Running...
   [INFO:2023-10-20 09:48:01,185]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ZK: Tearing down...
   [INFO:2023-10-20 09:48:36,870]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ZK: PASS
   [WARNING - 2023-10-20 09:48:36,871 - runner_client - log - lineno:294]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ZK: Test requested 8 nodes, used only 7
   [WARNING:2023-10-20 09:48:36,871]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ZK: Test requested 8 nodes, used only 7
   [INFO:2023-10-20 09:48:36,872]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ZK: Data: None
   [INFO:2023-10-20 09:48:36,887]: ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
   [INFO:2023-10-20 09:48:36,888]: Triggering test 5 of 8...
   [INFO:2023-10-20 09:48:36,902]: RunnerClient: Loading test {'directory': '/opt/kafka-dev/tests/kafkatest/tests/core', 'file_name': 'reassign_partitions_test.py', 'cls_name': 'ReassignPartitionsTest', 'method_name': 'test_reassign_partitions', 'injected_args': {'bounce_brokers': True, 'reassign_from_offset_zero': False, 'metadata_quorum': 'ISOLATED_KRAFT'}}
   [INFO:2023-10-20 09:48:36,907]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=False.metadata_quorum=ISOLATED_KRAFT: on run 1/1
   [INFO:2023-10-20 09:48:36,909]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=False.metadata_quorum=ISOLATED_KRAFT: Setting up...
   [INFO:2023-10-20 09:48:36,911]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=False.metadata_quorum=ISOLATED_KRAFT: Running...
   [INFO:2023-10-20 09:51:20,336]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=False.metadata_quorum=ISOLATED_KRAFT: Tearing down...
   [INFO:2023-10-20 09:51:44,207]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=False.metadata_quorum=ISOLATED_KRAFT: PASS
   [INFO:2023-10-20 09:51:44,208]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=False.metadata_quorum=ISOLATED_KRAFT: Data: None
   [INFO:2023-10-20 09:51:44,226]: ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
   [INFO:2023-10-20 09:51:44,227]: Triggering test 6 of 8...
   [INFO:2023-10-20 09:51:44,240]: RunnerClient: Loading test {'directory': '/opt/kafka-dev/tests/kafkatest/tests/core', 'file_name': 'reassign_partitions_test.py', 'cls_name': 'ReassignPartitionsTest', 'method_name': 'test_reassign_partitions', 'injected_args': {'bounce_brokers': True, 'reassign_from_offset_zero': False, 'metadata_quorum': 'ZK'}}
   [INFO:2023-10-20 09:51:44,244]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=False.metadata_quorum=ZK: on run 1/1
   [INFO:2023-10-20 09:51:44,247]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=False.metadata_quorum=ZK: Setting up...
   [INFO:2023-10-20 09:51:47,848]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=False.metadata_quorum=ZK: Running...
   [INFO:2023-10-20 09:54:10,056]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=False.metadata_quorum=ZK: Tearing down...
   [INFO:2023-10-20 09:54:46,389]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=False.metadata_quorum=ZK: PASS
   [INFO:2023-10-20 09:54:46,389]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=False.metadata_quorum=ZK: Data: None
   [INFO:2023-10-20 09:54:46,412]: ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
   [INFO:2023-10-20 09:54:46,413]: Triggering test 7 of 8...
   [INFO:2023-10-20 09:54:46,429]: RunnerClient: Loading test {'directory': '/opt/kafka-dev/tests/kafkatest/tests/core', 'file_name': 'reassign_partitions_test.py', 'cls_name': 'ReassignPartitionsTest', 'method_name': 'test_reassign_partitions', 'injected_args': {'bounce_brokers': True, 'reassign_from_offset_zero': True, 'metadata_quorum': 'ISOLATED_KRAFT'}}
   [INFO:2023-10-20 09:54:46,436]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: on run 1/1
   [INFO:2023-10-20 09:54:46,439]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: Setting up...
   [INFO:2023-10-20 09:54:46,440]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: Running...
   [INFO:2023-10-20 09:57:44,797]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: Tearing down...
   [INFO:2023-10-20 09:58:09,838]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: PASS
   [WARNING - 2023-10-20 09:58:09,838 - runner_client - log - lineno:294]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: Test requested 8 nodes, used only 7
   [WARNING:2023-10-20 09:58:09,838]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: Test requested 8 nodes, used only 7
   [INFO:2023-10-20 09:58:09,839]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT: Data: None
   [INFO:2023-10-20 09:58:09,853]: ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
   [INFO:2023-10-20 09:58:09,853]: Triggering test 8 of 8...
   [INFO:2023-10-20 09:58:09,864]: RunnerClient: Loading test {'directory': '/opt/kafka-dev/tests/kafkatest/tests/core', 'file_name': 'reassign_partitions_test.py', 'cls_name': 'ReassignPartitionsTest', 'method_name': 'test_reassign_partitions', 'injected_args': {'bounce_brokers': True, 'reassign_from_offset_zero': True, 'metadata_quorum': 'ZK'}}
   [INFO:2023-10-20 09:58:09,867]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ZK: on run 1/1
   [INFO:2023-10-20 09:58:09,869]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ZK: Setting up...
   [INFO:2023-10-20 09:58:13,587]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ZK: Running...
   [INFO:2023-10-20 09:59:58,469]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ZK: Tearing down...
   [INFO:2023-10-20 10:00:36,352]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ZK: PASS
   [WARNING - 2023-10-20 10:00:36,352 - runner_client - log - lineno:294]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ZK: Test requested 8 nodes, used only 7
   [WARNING:2023-10-20 10:00:36,353]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ZK: Test requested 8 nodes, used only 7
   [INFO:2023-10-20 10:00:36,353]: RunnerClient: kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ZK: Data: None
   ================================================================================
   SESSION REPORT (ALL TESTS)
   ducktape version: 0.11.4
   session_id:       2023-10-20--001
   run time:         23 minutes 56.810 seconds
   tests run:        8
   passed:           8
   flaky:            0
   failed:           0
   ignored:          0
   ================================================================================
   test_id:    kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=False.metadata_quorum=ISOLATED_KRAFT
   status:     PASS
   run time:   2 minutes 47.940 seconds
   --------------------------------------------------------------------------------
   test_id:    kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=False.metadata_quorum=ZK
   status:     PASS
   run time:   2 minutes 53.391 seconds
   --------------------------------------------------------------------------------
   test_id:    kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT
   status:     PASS
   run time:   3 minutes 3.432 seconds
   --------------------------------------------------------------------------------
   test_id:    kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=False.reassign_from_offset_zero=True.metadata_quorum=ZK
   status:     PASS
   run time:   3 minutes 12.410 seconds
   --------------------------------------------------------------------------------
   test_id:    kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=False.metadata_quorum=ISOLATED_KRAFT
   status:     PASS
   run time:   3 minutes 7.300 seconds
   --------------------------------------------------------------------------------
   test_id:    kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=False.metadata_quorum=ZK
   status:     PASS
   run time:   3 minutes 2.145 seconds
   --------------------------------------------------------------------------------
   test_id:    kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ISOLATED_KRAFT
   status:     PASS
   run time:   3 minutes 23.402 seconds
   --------------------------------------------------------------------------------
   test_id:    kafkatest.tests.core.reassign_partitions_test.ReassignPartitionsTest.test_reassign_partitions.bounce_brokers=True.reassign_from_offset_zero=True.metadata_quorum=ZK
   status:     PASS
   run time:   2 minutes 26.485 seconds
   --------------------------------------------------------------------------------
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   @jolshan Looks like we good to go. Are you ready to merge this PR?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   Hello @mimaison 
   Can you, please, share your feeback for PR?
   Are we ready to merge this?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] mimaison commented on pull request #13247: KAFKA-14595 [WIP] Move ReassignPartitionsCommand to java

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

   I'll try to take a look at #14096 this week. Thanks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] mimaison commented on pull request #13247: KAFKA-14595 [WIP] Move ReassignPartitionsCommand to java

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

   Thanks @nizhikov for the updates. In its current shape it's a pretty big PR. I really prefer when we moved one thing at a time, it's much easier to review. If you can think of ways to split it in at least 2 PRs that would be preferable and I'd recommend not adding any more changes to this!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on a diff in pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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;
+
+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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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 {
+        Tuple<List<Tuple<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                              List<Tuple<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.print(partitionReassignmentStatesToString(partStates));
+        return new Tuple<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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 Tuple<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                           List<Tuple<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+
+            if (currentReassignments.containsKey(part)) {
+                PartitionReassignment reassignment = currentReassignments.get(part);
+                notFoundResults.add(new Tuple<>(part, new PartitionReassignmentState(
+                    reassignment.replicas(),
+                    targetReplicas,
+                    false)));
+            } else {
+                notFoundResults.add(new Tuple<>(part, topicDescriptionFutureToState(part.partition(),
+                    topicDescriptions.get(part.topic()), targetReplicas)));
+            }
+        }
+
+        Map<TopicPartition, PartitionReassignmentState> allResults = new HashMap<>();
+        foundResults.forEach(e -> allResults.put(e.v1, e.v2));
+        notFoundResults.forEach(e -> allResults.put(e.v1, e.v2));
+
+        return new Tuple<>(allResults, currentReassignments.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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 Tuple<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                  Map<TopicPartitionReplica, String> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> moveStates = findLogDirMoveStates(adminClient, targetReassignments);
+        System.out.println(replicaMoveStatesToString(moveStates));
+        return new Tuple<>(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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient.
+            describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple<TopicPartition, List<Integer>>> targetParts) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        targetParts.forEach(t -> brokers.addAll(t.v2));
+
+        System.out.printf("Clearing broker-level throttles on broker%s %s%n",
+            brokers.size() == 1 ? "" : "s", brokers.stream().map(Object::toString).collect(Collectors.joining(",")));
+        clearBrokerLevelThrottles(adminClient, brokers);
+
+        Set<String> 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", topics.stream().map(Object::toString).collect(Collectors.joining(",")));
+        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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, 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 Tuple<Map<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                            String reassignmentJson,
+                                                                                                            String brokerListString,
+                                                                                                            Boolean enableRackAwareness) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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",
+            formatAsReassignmentJson(proposedAssignments, Collections.emptyMap()));
+        return new Tuple<>(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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        HashMap<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   @mimaison 
   @ijuma 
   
   It seems we reduced changes as much as possible. 
   So it's time to do final review of command transfer from scala to java.
   
   Can you, please, take a look.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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

   @mimaison I create #14217 which include `ReassignPartitionsCommandArgsTest` rewritten in java. It independent from the big PR and can be reviewed separately. Can you, please, take a look?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on a diff in pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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


##########
checkstyle/checkstyle.xml:
##########
@@ -119,7 +119,7 @@
     </module>
     <module name="ClassDataAbstractionCoupling">
       <!-- default is 7 -->
-      <property name="max" value="25"/>
+      <property name="max" value="30"/>

Review Comment:
   Good catch. Thanks! Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on a diff in pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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


##########
checkstyle/import-control.xml:
##########
@@ -292,7 +294,11 @@
     <allow pkg="scala.collection" />
 
     <subpackage name="reassign">
-      <allow pkg="kafka.admin" />
+        <allow pkg="kafka.admin" />

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] tledkov commented on a diff in pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/Tuple.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 Tuple<V1, V2> {

Review Comment:
   Minor naming issue. What about  `Tuple2` / `BiTuple` etc? It's just that the name "tuple" suggests a tuple with an arbitrary number of fields.
   Also, can we afford to move this class to `org.apache.kafka.common` / `org.apache.kafka.utils` or `org.apache.kafka.lang`. I guess it is not the last place where a tuple abstraction is used.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on a diff in pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/Tuple.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 Tuple<V1, V2> {

Review Comment:
   AFAICU the plan is to omit usage of "Tuple" classes completely.
   So, I must take some time to investigate if we can do it.
   But, for now, more simple steps can be performed - like moving tests one by one to java.
   
   After that, Tuple usage will become more clear.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] fvaleri commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

   @nizhikov I don't have any preference here, I'm fine with whatever is more convenient for you. Multiple PRs may be convenient, but if you open them in parallel, merging one could easily break the others with the need of multiple rebase operations. If you decide to continue with this PR, maybe try to group the various migration phases using commits, so that the review is easier.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

   Hello @ijuma 
   
   > If you want to do it in small steps
   
   It's more about simplify review then my personal preferences :)
   
   For now, I introduced java classes from `ReassignPartitionsCommand` in core module.
   It seems we can review and merge current changes and then move to the actual command scala -> java transformation.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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

   @jolshan Please, join the final review to rewrite `ReassignPartitionsCommand` to java if you have spare cycles :)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1501 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.println(partitionReassignmentStatesToString(partStates));
+        return new Tuple2<>(partStates, partsOngoing);

Review Comment:
   Good catch. Thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1500 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> 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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                             String reassignmentJson,
+                                                                                                             String brokerListString,
+                                                                                                             Boolean enableRackAwareness
+    ) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple2<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        Map<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                                                Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                Set<TopicPartition> partitions
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                           String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> 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<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> t0 = parseExecuteAssignmentArgs(reassignmentJson);
+
+        Map<TopicPartition, List<Integer>> proposedParts = t0.v1;
+        Map<TopicPartitionReplica, String> proposedReplicas = t0.v2;
+        Map<TopicPartition, PartitionReassignment> 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<Integer> brokers = new HashSet<>();
+        proposedParts.values().forEach(brokers::addAll);
+
+        verifyBrokerIds(adminClient, brokers);
+        Map<TopicPartition, List<Integer>> 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<String, Map<Integer, PartitionMove>> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts);
+                modifyReassignmentThrottle(adminClient, moveMap, interBrokerThrottle);
+            }
+
+            if (logDirThrottle >= 0) {
+                Set<Integer> movingBrokers = calculateMovingBrokers(proposedReplicas.keySet());
+                modifyLogDirThrottle(adminClient, movingBrokers, logDirThrottle);
+            }
+        }
+
+        // Execute the partition reassignments.
+        Map<TopicPartition, Throwable> 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<TopicPartitionReplica, String> proposedReplicas,
+                                     Long timeoutMs,
+                                     Time time
+    ) throws InterruptedException, TerseException {
+        long startTimeMs = time.milliseconds();
+        Map<TopicPartitionReplica, String> pendingReplicas = new HashMap<>(proposedReplicas);
+        boolean done = false;
+        do {
+            Set<TopicPartitionReplica> 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<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+        String text = currentReassignments.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).map(part -> {
+            PartitionReassignment reassignment = currentReassignments.get(part);
+            List<Integer> replicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+            List<Integer> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Set<Integer> allNodeIds = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Optional<Integer> 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<TopicPartition, List<Integer>> proposedParts,
+                                                            Map<TopicPartition, List<Integer>> currentParts) throws JsonProcessingException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, Throwable> alterPartitionReassignments(Admin adminClient,
+                                                                      Map<TopicPartition, List<Integer>> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.forEach((part, replicas) -> args.put(part, Optional.of(new NewPartitionReassignment(replicas))));
+        Map<TopicPartition, KafkaFuture<Void>> results = adminClient.alterPartitionReassignments(args).values();
+        Map<TopicPartition, Throwable> errors = new HashMap<>();
+        for (Map.Entry<TopicPartition, KafkaFuture<Void>> 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<TopicPartition, Throwable> cancelPartitionReassignments(Admin adminClient,
+                                                                       Set<TopicPartition> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.forEach(part -> args.put(part, Optional.empty()));
+
+        Map<TopicPartition, KafkaFuture<Void>> results = adminClient.alterPartitionReassignments(args).values();
+        Map<TopicPartition, Throwable> errors = new HashMap<>();
+        for (Map.Entry<TopicPartition, KafkaFuture<Void>> 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<String, Map<Integer, PartitionMove>> calculateCurrentMoveMap(Map<TopicPartition, PartitionReassignment> currentReassignments) {
+        Map<String, Map<Integer, PartitionMove>> moveMap = new HashMap<>();
+        // Add the current reassignments to the move map.
+        currentReassignments.forEach((part, reassignment) -> {
+

Review Comment:
   Empty line



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1501 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {

Review Comment:
   Too long line. Could we format the method like this:
   ```suggestion
       private static Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(
           Admin adminClient,
           List<Tuple2<TopicPartition, List<Integer>>> targets
       ) throws ExecutionException, InterruptedException {
   ```
   or this style is disallow by Kafka code convetions?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1501 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1501 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t0 = findPartitionReassignmentStates(adminClient, targets);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t0.v1;
+        Boolean partsOngoing = t0.v2;
+
+        System.out.println(partitionReassignmentStatesToString(partStates));
+        return new Tuple2<>(partStates, partsOngoing);
+    }
+
+    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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition, List<Integer>>> targetReassignments) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = new HashSet<>();
+        notFoundReassignments.forEach(e -> {
+            TopicPartition part = e.v1;
+            if (!currentReassignments.containsKey(part))
+                topicNamesToLookUp.add(part.topic());
+        });

Review Comment:
   What about using simple stream API here:
   ```suggestion
           Set<String> topicNamesToLookUp0 = notFoundReassignments.stream()
               .map(e -> e.v1)
               .filter(part -> !currentReassignments.containsKey(part))
               .map(TopicPartition::topic)
               .collect(Collectors.toSet());
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1523 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient
+            .describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, topicName),
+            TOPIC_LEVEL_THROTTLES.stream().map(throttle -> new AlterConfigOp(new ConfigEntry(throttle, null),
+                AlterConfigOp.OpType.DELETE)).collect(Collectors.toList())
+
+        ));

Review Comment:
   ```suggestion
           Map<ConfigResource, Collection<AlterConfigOp>> 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())
               )); 
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1523 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient
+            .describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+
+            LogDirMoveState moveState;
+
+            if (!replicaLogDirInfos.containsKey(replica)) {
+                moveState = new MissingReplicaMoveState(targetLogDir);
+            } else {
+                DescribeReplicaLogDirsResult.ReplicaLogDirInfo info = replicaLogDirInfos.get(replica);
+                if (info.getCurrentReplicaLogDir() == null) {
+                    moveState = new MissingLogDirMoveState(targetLogDir);
+                } else if (info.getFutureReplicaLogDir() == null) {
+                    if (info.getCurrentReplicaLogDir().equals(targetLogDir)) {
+                        moveState = new CompletedMoveState(targetLogDir);
+                    } else {
+                        moveState = new CancelledMoveState(info.getCurrentReplicaLogDir(), targetLogDir);
+                    }
+                } else {
+                    moveState = new ActiveMoveState(info.getCurrentReplicaLogDir(),
+                        targetLogDir,
+                        info.getFutureReplicaLogDir());
+                }
+            }
+            res.put(replica, moveState);
+        });
+
+        return res;
+    }
+
+    /**
+     * 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> configOps = new HashMap<>();
+        topics.forEach(topicName -> configOps.put(
+            new ConfigResource(ConfigResource.Type.TOPIC, 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<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                             String reassignmentJson,
+                                                                                                             String brokerListString,
+                                                                                                             Boolean enableRackAwareness
+    ) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple2<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        Map<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                                                Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics
+    ) throws ExecutionException, InterruptedException {
+
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                Set<TopicPartition> partitions
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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());

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java:
##########
@@ -857,68 +864,18 @@ public void close() {
         }
     }
 
-    private ReassignPartitionsCommand.VerifyAssignmentResult asScala(VerifyAssignmentResult res) {
-        Map<TopicPartition, ReassignPartitionsCommand.PartitionReassignmentState> partStates = new HashMap<>();
-        res.partStates.forEach((tp, state) -> partStates.put(tp, asScala(state)));
-
-        Map<TopicPartitionReplica, ReassignPartitionsCommand.LogDirMoveState> 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 <T> scala.collection.immutable.Set<T> set(final T... set) {
-        return mutableSet(set).toSet();
+    static <T> Set<T> set(final T... set) {
+        return new HashSet<>(Arrays.asList(set));

Review Comment:
   We able to remove `set` method.
   But, `mutableSet` and `seq` has to stay, because, we use `QuorumTestHarness` as parent and `TestUtils` which written in scala.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala:
##########
@@ -1,340 +0,0 @@
-/**

Review Comment:
   Please, write me if we want to migrate this class, also.
   Success run of main method is enough to test it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala:
##########
@@ -1,340 +0,0 @@
-/**

Review Comment:
   OK. Let's migrate those class too. Will prepare PR in the nearest time.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: [WIP] KAFKA-14595 Move value object of ReassignPartitionsCommand to java

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

   @mimaison 
   
   This PR is a first part of moving `ReassignPartitionsCommand` to java and `tools` module.
   Can you, please, take a look?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] mimaison commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

   > Are you suggesting to perform all moving in one PR? Or you have some plan how to performa moving in several steps? If yes, please, share some guidelines on how should I implement it.
   
   We can create the new Java classes in tools and not touch the existing Scala code. Then in a second time, we can rewrite the tool in Java using the new classes and delete all the old Scala code.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

   @mimaison 
   
   > I think we should directly move the classes to the tools module instead of temporarily keeping them in core
   
   Are you suggesting to perform all moving in one PR?
   Or you have some plan how to performa moving in several steps? If yes, please, share som guideline on how should I implement it.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] mimaison commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

   I think we should directly move the classes to the tools module instead of temporarily keeping them in core.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: [WIP] KAFKA-14595 Move value object of ReassignPartitionsCommand to java

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

   @ijuma 
   
   > Is server-common the right module?
   
   When move `ReassignPartitionsCommand` finished all classes must land in `tools` module.
   So the answer is No - `server-common` in not right module.
   
   But, I propose to move with small steps and split moving in several changes.
   To keep changes reviewable.
   As a first step I propose to move case classes from `ReassignPartitionsCommand` to java code.
   
   It seems new java classes can be keep in `core` module so I placed them into it.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] github-actions[bot] commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] fvaleri commented on a diff in pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandOptions.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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 joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+
+public class ReassignPartitionsCommandOptions extends CommandDefaultOptions {
+    // Actions
+    private final OptionSpecBuilder verifyOpt;
+    private final OptionSpecBuilder generateOpt;
+    private final OptionSpecBuilder executeOpt;
+    private final OptionSpecBuilder cancelOpt;
+    private final OptionSpecBuilder listOpt;
+
+    // Arguments
+    private final ArgumentAcceptingOptionSpec<String> bootstrapServerOpt;

Review Comment:
   I would use the `OptionSpec` superclass for consistency with the other tools.



##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandOptions.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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 joptsimple.ArgumentAcceptingOptionSpec;
+import joptsimple.OptionSpecBuilder;
+import org.apache.kafka.server.util.CommandDefaultOptions;
+
+public class ReassignPartitionsCommandOptions extends CommandDefaultOptions {
+    // Actions
+    private final OptionSpecBuilder verifyOpt;

Review Comment:
   I would use the `OptionSpec` superclass for consistency with the other tools.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

   @fvaleri Can you, please, share you opinion? What is the next steps on rewriting `ReassignPartitionsCommand`?
   I can continue rewriting in this PR or there is committer that will commit PR in the trunk as is?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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

   I created a #14355 to rewrite ReassignPartitionsUnitTest separately.
   This will reduce changed even move.
   
   Please, review.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] tledkov commented on a diff in pull request #13247: KAFKA-14595 Move ReassignPartitionsCommand to java

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/Tuple.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 Tuple<V1, V2> {

Review Comment:
   Minor naming issue. What about  `Tuple2` / `BiTuple` etc? It's just that the name "tuple" suggests a tuple with an arbitrary number of fields.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1523 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> replicaLogDirInfos = adminClient
+            .describeReplicaLogDirs(targetMoves.keySet()).all().get();
+
+        Map<TopicPartitionReplica, LogDirMoveState> res = new HashMap<>();
+
+        targetMoves.entrySet().forEach(e -> {
+            TopicPartitionReplica replica = e.getKey();
+            String targetLogDir = e.getValue();
+

Review Comment:
   ```suggestion
           targetMoves.forEach((replica, targetLogDir) -> {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1513 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                             String reassignmentJson,
+                                                                                                             String brokerListString,
+                                                                                                             Boolean enableRackAwareness
+    ) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple2<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        Map<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                                                Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                Set<TopicPartition> partitions
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                           String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> 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<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> t0 = parseExecuteAssignmentArgs(reassignmentJson);
+
+        Map<TopicPartition, List<Integer>> proposedParts = t0.v1;
+        Map<TopicPartitionReplica, String> proposedReplicas = t0.v2;
+        Map<TopicPartition, PartitionReassignment> 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<Integer> brokers = new HashSet<>();
+        proposedParts.values().forEach(brokers::addAll);
+
+        verifyBrokerIds(adminClient, brokers);
+        Map<TopicPartition, List<Integer>> 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<String, Map<Integer, PartitionMove>> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts);
+                modifyReassignmentThrottle(adminClient, moveMap, interBrokerThrottle);
+            }
+
+            if (logDirThrottle >= 0) {
+                Set<Integer> movingBrokers = calculateMovingBrokers(proposedReplicas.keySet());
+                modifyLogDirThrottle(adminClient, movingBrokers, logDirThrottle);
+            }
+        }
+
+        // Execute the partition reassignments.
+        Map<TopicPartition, Throwable> 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<TopicPartitionReplica, String> proposedReplicas,
+                                     Long timeoutMs,
+                                     Time time
+    ) throws InterruptedException, TerseException {
+        long startTimeMs = time.milliseconds();
+        Map<TopicPartitionReplica, String> pendingReplicas = new HashMap<>();
+        pendingReplicas.putAll(proposedReplicas);
+        boolean done = false;
+        do {
+            Set<TopicPartitionReplica> 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<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+        String text = currentReassignments.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).map(part -> {
+            PartitionReassignment reassignment = currentReassignments.get(part);
+            List<Integer> replicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+            List<Integer> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Set<Integer> allNodeIds = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Optional<Integer> 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<TopicPartition, List<Integer>> proposedParts,
+                                                            Map<TopicPartition, List<Integer>> currentParts) throws JsonProcessingException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, Throwable> alterPartitionReassignments(Admin adminClient,
+                                                                      Map<TopicPartition, List<Integer>> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.entrySet().forEach(e -> {
+            TopicPartition part = e.getKey();
+            List<Integer> replicas = e.getValue();
+
+            args.put(part, Optional.of(new NewPartitionReassignment(replicas)));
+        });
+        Map<TopicPartition, KafkaFuture<Void>> results = adminClient.alterPartitionReassignments(args).values();
+        Map<TopicPartition, Throwable> errors = new HashMap<>();
+        for (Map.Entry<TopicPartition, KafkaFuture<Void>> 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<TopicPartition, Throwable> cancelPartitionReassignments(Admin adminClient,
+                                                                       Set<TopicPartition> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.forEach(part -> args.put(part, Optional.empty()));
+
+        Map<TopicPartition, KafkaFuture<Void>> results = adminClient.alterPartitionReassignments(args).values();
+        Map<TopicPartition, Throwable> errors = new HashMap<>();
+        for (Map.Entry<TopicPartition, KafkaFuture<Void>> 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<String, Map<Integer, PartitionMove>> calculateCurrentMoveMap(Map<TopicPartition, PartitionReassignment> currentReassignments) {
+        Map<String, Map<Integer, PartitionMove>> moveMap = new HashMap<>();
+        // Add the current reassignments to the move map.
+        currentReassignments.entrySet().forEach(e -> {
+            TopicPartition part = e.getKey();
+            PartitionReassignment reassignment = e.getValue();
+
+            List<Integer> allReplicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+
+            // The addingReplicas is included in the replicas during reassignment
+            Set<Integer> sources = new HashSet<>(allReplicas);
+            sources.removeAll(addingReplicas);
+
+            Set<Integer> destinations = new HashSet<>(addingReplicas);
+
+            Map<Integer, PartitionMove> partMoves = moveMap.computeIfAbsent(part.topic(), k -> new HashMap<>());
+            partMoves.put(part.partition(), new PartitionMove(sources, destinations));
+        });

Review Comment:
   ```suggestion
    currentReassignments.forEach((part, reassignment) -> {
               List<Integer> allReplicas = reassignment.replicas();
               List<Integer> addingReplicas = reassignment.addingReplicas();
   
               // The addingReplicas is included in the replicas during reassignment
               Set<Integer> sources = new HashSet<>(allReplicas);
               addingReplicas.forEach(sources::remove);
   
               Set<Integer> destinations = new HashSet<>(addingReplicas);
   
               Map<Integer, PartitionMove> partMoves = moveMap.computeIfAbsent(part.topic(), k -> new HashMap<>());
               partMoves.put(part.partition(), new PartitionMove(sources, destinations));
           });
   ```
   Also there is recommendation to avoid `mySet.removeAll(myList);`. Fixed at the suggestion by  `myList.forEach(mySet::remove);`
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -0,0 +1,1513 @@
+/*
+ * 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<List<Integer>> INT_LIST = DecodeJson.decodeList(INT);
+
+    private static final DecodeJson<List<String>> 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<String> 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<String> 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<List<Tuple2<TopicPartition, List<Integer>>>, Map<TopicPartitionReplica, String>> t0 = parsePartitionReassignmentData(jsonString);
+
+        List<Tuple2<TopicPartition, List<Integer>>> targetParts = t0.v1;
+        Map<TopicPartitionReplica, String> targetLogDirs = t0.v2;
+
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, Boolean> t1 = verifyPartitionAssignments(adminClient, targetParts);
+
+        Map<TopicPartition, PartitionReassignmentState> partStates = t1.v1;
+        Boolean partsOngoing = t1.v2;
+
+        Tuple2<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> t2 = verifyReplicaMoves(adminClient, targetLogDirs);
+
+        Map<TopicPartitionReplica, LogDirMoveState> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> verifyPartitionAssignments(Admin adminClient,
+                                                                                                               List<Tuple2<TopicPartition, List<Integer>>> targets
+    ) throws ExecutionException, InterruptedException {
+        Tuple2<Map<TopicPartition, PartitionReassignmentState>, 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<TopicPartition, PartitionReassignmentState> states) {
+        List<String> 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<Map<TopicPartition, PartitionReassignmentState>, Boolean> findPartitionReassignmentStates(Admin adminClient,
+                                                                                                            List<Tuple2<TopicPartition,
+                                                                                                            List<Integer>>> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+
+        List<Tuple2<TopicPartition, List<Integer>>> foundReassignments = new ArrayList<>();
+        List<Tuple2<TopicPartition, List<Integer>>> notFoundReassignments = new ArrayList<>();
+
+        targetReassignments.forEach(reassignment -> {
+            if (currentReassignments.containsKey(reassignment.v1))
+                foundReassignments.add(reassignment);
+            else
+                notFoundReassignments.add(reassignment);
+        });
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> foundResults = foundReassignments.stream().map(e -> {
+            TopicPartition part = e.v1;
+            List<Integer> targetReplicas = e.v2;
+            return new Tuple2<>(part,
+                new PartitionReassignmentState(
+                    currentReassignments.get(part).replicas(),
+                    targetReplicas,
+                    false));
+        }).collect(Collectors.toList());
+
+        Set<String> topicNamesToLookUp = notFoundReassignments.stream()
+            .map(e -> e.v1)
+            .filter(part -> !currentReassignments.containsKey(part))
+            .map(TopicPartition::topic)
+            .collect(Collectors.toSet());
+
+        Map<String, KafkaFuture<TopicDescription>> topicDescriptions = adminClient.
+            describeTopics(topicNamesToLookUp).topicNameValues();
+
+        List<Tuple2<TopicPartition, PartitionReassignmentState>> notFoundResults = new ArrayList<>();
+        for (Tuple2<TopicPartition, List<Integer>> e : notFoundReassignments) {
+            TopicPartition part = e.v1;
+            List<Integer> 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<TopicPartition, PartitionReassignmentState> 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.size() > 0);
+    }
+
+    private static PartitionReassignmentState topicDescriptionFutureToState(int partition,
+                                                                            KafkaFuture<TopicDescription> future,
+                                                                            List<Integer> 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<Map<TopicPartitionReplica, LogDirMoveState>, Boolean> verifyReplicaMoves(Admin adminClient,
+                                                                                                   Map<TopicPartitionReplica, String> targetReassignments
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, LogDirMoveState> 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<TopicPartitionReplica, LogDirMoveState> findLogDirMoveStates(Admin adminClient,
+                                                                            Map<TopicPartitionReplica, String> targetMoves
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartitionReplica, DescribeReplicaLogDirsResult.ReplicaLogDirInfo> 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<TopicPartitionReplica, LogDirMoveState> states) {
+        List<String> 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<Tuple2<TopicPartition, List<Integer>>> targetParts
+    ) throws ExecutionException, InterruptedException {
+        Set<Integer> activeBrokers = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Set<Integer> brokers = new HashSet<>(activeBrokers);
+        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<String> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<String> topics) throws ExecutionException, InterruptedException {
+        Map<ConfigResource, Collection<AlterConfigOp>> 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<TopicPartition, List<Integer>>, Map<TopicPartition, List<Integer>>> generateAssignment(Admin adminClient,
+                                                                                                             String reassignmentJson,
+                                                                                                             String brokerListString,
+                                                                                                             Boolean enableRackAwareness
+    ) throws ExecutionException, InterruptedException, JsonProcessingException {
+        Tuple2<List<Integer>, List<String>> t0 = parseGenerateAssignmentArgs(reassignmentJson, brokerListString);
+
+        List<Integer> brokersToReassign = t0.v1;
+        List<String> topicsToReassign = t0.v2;
+
+        Map<TopicPartition, List<Integer>> currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+        List<BrokerMetadata> brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness);
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> calculateAssignment(Map<TopicPartition, List<Integer>> currentAssignment,
+                                                                          List<BrokerMetadata> brokerMetadatas) {
+        Map<String, List<Map.Entry<TopicPartition, List<Integer>>>> groupedByTopic = new HashMap<>();
+        for (Map.Entry<TopicPartition, List<Integer>> e : currentAssignment.entrySet())
+            groupedByTopic.computeIfAbsent(e.getKey().topic(), k -> new ArrayList<>()).add(e);
+        Map<TopicPartition, List<Integer>> proposedAssignments = new HashMap<>();
+        groupedByTopic.forEach((topic, assignment) -> {
+            List<Integer> replicas = assignment.get(0).getValue();
+            Map<Integer, List<Integer>> assignedReplicas = AdminUtils.
+                assignReplicasToBrokers(brokerMetadatas, assignment.size(), replicas.size());
+            assignedReplicas.forEach((partition, replicas0) ->
+                proposedAssignments.put(new TopicPartition(topic, partition), replicas0));
+        });
+        return proposedAssignments;
+    }
+
+    private static Map<String, TopicDescription> describeTopics(Admin adminClient,
+                                                                Set<String> topics) throws ExecutionException, InterruptedException {
+        Map<String, KafkaFuture<TopicDescription>> futures = adminClient.describeTopics(topics).topicNameValues();
+        Map<String, TopicDescription> res = new HashMap<>();
+        for (Map.Entry<String, KafkaFuture<TopicDescription>> e : futures.entrySet()) {
+            String topicName = e.getKey();
+            KafkaFuture<TopicDescription> 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<TopicPartition, List<Integer>> getReplicaAssignmentForTopics(Admin adminClient,
+                                                                            List<String> topics
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, List<Integer>> getReplicaAssignmentForPartitions(Admin adminClient,
+                                                                                Set<TopicPartition> partitions
+    ) throws ExecutionException, InterruptedException {
+        Map<TopicPartition, List<Integer>> 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<BrokerMetadata> getBrokerMetadata(Admin adminClient, List<Integer> brokers, boolean enableRackAwareness) throws ExecutionException, InterruptedException {
+        Set<Integer> brokerSet = new HashSet<>(brokers);
+        List<BrokerMetadata> 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<Integer>, List<String>> parseGenerateAssignmentArgs(String reassignmentJson,
+                                                                           String brokerList) throws JsonMappingException {
+        List<Integer> brokerListToReassign = Arrays.stream(brokerList.split(",")).map(Integer::parseInt).collect(Collectors.toList());
+        Set<Integer> duplicateReassignments = ToolsUtils.duplicates(brokerListToReassign);
+        if (!duplicateReassignments.isEmpty())
+            throw new AdminCommandFailedException(String.format("Broker list contains duplicate entries: %s", duplicateReassignments));
+        List<String> topicsToReassign = parseTopicsData(reassignmentJson);
+        Set<String> 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<TopicPartition, List<Integer>>, Map<TopicPartitionReplica, String>> t0 = parseExecuteAssignmentArgs(reassignmentJson);
+
+        Map<TopicPartition, List<Integer>> proposedParts = t0.v1;
+        Map<TopicPartitionReplica, String> proposedReplicas = t0.v2;
+        Map<TopicPartition, PartitionReassignment> 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<Integer> brokers = new HashSet<>();
+        proposedParts.values().forEach(brokers::addAll);
+
+        verifyBrokerIds(adminClient, brokers);
+        Map<TopicPartition, List<Integer>> 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<String, Map<Integer, PartitionMove>> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts);
+                modifyReassignmentThrottle(adminClient, moveMap, interBrokerThrottle);
+            }
+
+            if (logDirThrottle >= 0) {
+                Set<Integer> movingBrokers = calculateMovingBrokers(proposedReplicas.keySet());
+                modifyLogDirThrottle(adminClient, movingBrokers, logDirThrottle);
+            }
+        }
+
+        // Execute the partition reassignments.
+        Map<TopicPartition, Throwable> 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<TopicPartitionReplica, String> proposedReplicas,
+                                     Long timeoutMs,
+                                     Time time
+    ) throws InterruptedException, TerseException {
+        long startTimeMs = time.milliseconds();
+        Map<TopicPartitionReplica, String> pendingReplicas = new HashMap<>();
+        pendingReplicas.putAll(proposedReplicas);
+        boolean done = false;
+        do {
+            Set<TopicPartitionReplica> 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<TopicPartition, PartitionReassignment> currentReassignments = adminClient.
+            listPartitionReassignments().reassignments().get();
+        String text = currentReassignments.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).map(part -> {
+            PartitionReassignment reassignment = currentReassignments.get(part);
+            List<Integer> replicas = reassignment.replicas();
+            List<Integer> addingReplicas = reassignment.addingReplicas();
+            List<Integer> 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<Integer> brokers) throws ExecutionException, InterruptedException {
+        Set<Integer> allNodeIds = adminClient.describeCluster().nodes().get().stream().map(Node::id).collect(Collectors.toSet());
+        Optional<Integer> 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<TopicPartition, List<Integer>> proposedParts,
+                                                            Map<TopicPartition, List<Integer>> currentParts) throws JsonProcessingException {
+        Map<TopicPartition, List<Integer>> 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<TopicPartition, Throwable> alterPartitionReassignments(Admin adminClient,
+                                                                      Map<TopicPartition, List<Integer>> reassignments) throws InterruptedException {
+        Map<TopicPartition, Optional<NewPartitionReassignment>> args = new HashMap<>();
+        reassignments.entrySet().forEach(e -> {
+            TopicPartition part = e.getKey();
+            List<Integer> replicas = e.getValue();
+
+            args.put(part, Optional.of(new NewPartitionReassignment(replicas)));
+        });

Review Comment:
   ```suggestion
           reassignments.forEach((part, replicas) -> args.put(part, Optional.of(new NewPartitionReassignment(replicas))));
   
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


Re: [PR] KAFKA-14595 Move ReassignPartitionsCommand to java [kafka]

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


##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java:
##########
@@ -857,68 +864,18 @@ public void close() {
         }
     }
 
-    private ReassignPartitionsCommand.VerifyAssignmentResult asScala(VerifyAssignmentResult res) {
-        Map<TopicPartition, ReassignPartitionsCommand.PartitionReassignmentState> partStates = new HashMap<>();
-        res.partStates.forEach((tp, state) -> partStates.put(tp, asScala(state)));
-
-        Map<TopicPartitionReplica, ReassignPartitionsCommand.LogDirMoveState> 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 <T> scala.collection.immutable.Set<T> set(final T... set) {
-        return mutableSet(set).toSet();
+    static <T> Set<T> set(final T... set) {
+        return new HashSet<>(Arrays.asList(set));

Review Comment:
   Got it. Thanks for clarifying. (That's going to be a fun one to migrate too 😅 )



##########
tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsIntegrationTest.java:
##########
@@ -857,68 +864,18 @@ public void close() {
         }
     }
 
-    private ReassignPartitionsCommand.VerifyAssignmentResult asScala(VerifyAssignmentResult res) {
-        Map<TopicPartition, ReassignPartitionsCommand.PartitionReassignmentState> partStates = new HashMap<>();
-        res.partStates.forEach((tp, state) -> partStates.put(tp, asScala(state)));
-
-        Map<TopicPartitionReplica, ReassignPartitionsCommand.LogDirMoveState> 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 <T> scala.collection.immutable.Set<T> set(final T... set) {
-        return mutableSet(set).toSet();
+    static <T> Set<T> set(final T... set) {
+        return new HashSet<>(Arrays.asList(set));

Review Comment:
   Got it. Thanks for clarifying. (That's going to be a fun one to migrate too 😅 )



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

   @mimaison I resumed work on this PR to rewrite `ReassignPartitionsCommand` in one change so if you have any preferences regarding splitting work into several changes, please, write me :)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 Move value objects of ReassignPartitionsCommand to java

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

   Hello, @fvaleri thanks for the review.
   
   1. trunk merge to PR
   2. all your comments fixed.
   
   I'm ready to continue and finish work on moving `ReassignPartitionsCommand` to java code.
   So if this PR can be merge to tunk, let's do it.
   
   If you or @mimaison want to go another way, please, write me the next steps.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 [WIP] Move ReassignPartitionsCommand to java

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

   Hello @mimaison 
   
   > If you can think of ways to split it in at least 2 PRs that would be preferable
   
   My plan is the following:
   
   1. PR to rewrite AdminUtils  - https://github.com/apache/kafka/pull/14096 . It independent from the others and can be reviewed right now. I checked all tests failures and it seems like all of them is just flacky tests from trunk. Can you, please, join the review?
   
   2. PR to create a copy of POJOs and custom exceptions of `ReassignPartitionsCommand`.
   
   3. PR to rewrite command itself and it's tests.
   
   To finish all three I need to rewrite `ReassignPartitionsIntegrationTest` which I do right now.
   I will split PR in three after it.
   
   What do you think? Can you start review of #14096 ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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


[GitHub] [kafka] nizhikov commented on pull request #13247: KAFKA-14595 [WIP] Move ReassignPartitionsCommand to java

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

   @ijuma 
   @tombentley 
   @mimaison I rewrite `ReassignPartitionsCommand` and all tests belong to it. All tests regarding `ReassignPartitionsCommand.java` pass. 
   
   The only link to `ReassignPartitionsCommand.scala` is `ReplicationQuotasTestRig` which provides some `main` method and it never used in kafka project.
   
   I'm not sure what is supposed action here. Can you advise me how to unlink `ReplicationQuotasTestRig` and `ReassignPartitionsCommand.scala`?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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