You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by cm...@apache.org on 2020/03/20 03:45:06 UTC

[kafka] branch trunk updated: KAFKA-8820: kafka-reassign-partitions.sh should support the KIP-455 API (#8244)

This is an automated email from the ASF dual-hosted git repository.

cmccabe pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 56051e7  KAFKA-8820: kafka-reassign-partitions.sh should support the KIP-455 API (#8244)
56051e7 is described below

commit 56051e763965d439f11f20f876475732eed7b307
Author: Colin Patrick McCabe <cm...@confluent.io>
AuthorDate: Thu Mar 19 20:44:34 2020 -0700

    KAFKA-8820: kafka-reassign-partitions.sh should support the KIP-455 API (#8244)
    
    Rewrite ReassignPartitionsCommand to use the KIP-455 API when possible, rather
    than direct communication with ZooKeeper.  Direct ZK access is still supported,
    but deprecated, as described in KIP-455.
    
    As specified in KIP-455, the tool has several new flags.  --cancel stops
    an assignment which is in progress.  --preserve-throttle causes the
    --verify and --cancel commands to leave the throttles alone.
    --additional allows users to execute another partition assignment even
    if there is already one in progress.  Finally, --show displays all of
    the current partition reassignments.
    
    Reorganize the reassignment code and tests somewhat to rely more on unit
    testing using the MockAdminClient and less on integration testing.  Each
    integration test where we bring up a cluster seems to take about 5 seconds, so
    it's good when we can get similar coverage from unit tests.  To enable this,
    MockAdminClient now supports incrementalAlterConfigs, alterReplicaLogDirs,
    describeReplicaLogDirs, and some other APIs.  MockAdminClient is also now
    thread-safe, to match the real AdminClient implementation.
    
    In DeleteTopicTest, use the KIP-455 API rather than invoking the reassignment
    command.
---
 checkstyle/suppressions.xml                        |    2 +-
 .../kafka/clients/admin/MockAdminClient.java       |  481 ++++-
 .../kafka/connect/util/ConnectUtilsTest.java       |   11 +-
 .../kafka/admin/ReassignPartitionsCommand.scala    | 2108 +++++++++++++++-----
 .../admin/ReassignPartitionsIntegrationTest.scala  |  701 +++++--
 .../other/kafka/ReplicationQuotasTestRig.scala     |   31 +-
 .../scala/unit/kafka/admin/DeleteTopicTest.scala   |   65 +-
 .../admin/ReassignPartitionsClusterTest.scala      | 1304 ------------
 .../admin/ReassignPartitionsCommandArgsTest.scala  |  166 +-
 .../admin/ReassignPartitionsCommandTest.scala      |  582 ------
 .../kafka/admin/ReassignPartitionsUnitTest.scala   |  643 ++++++
 .../processor/internals/StreamThreadTest.java      |    3 +-
 .../internals/metrics/ThreadMetricsTest.java       |    2 +-
 13 files changed, 3426 insertions(+), 2673 deletions(-)

diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index 5cd6e52..4a06262 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -51,7 +51,7 @@
               files="Sender.java"/>
 
     <suppress checks="ClassDataAbstractionCoupling"
-              files="(KafkaConsumer|ConsumerCoordinator|Fetcher|KafkaProducer|AbstractRequest|AbstractResponse|TransactionManager|Admin|KafkaAdminClient).java"/>
+              files="(KafkaConsumer|ConsumerCoordinator|Fetcher|KafkaProducer|AbstractRequest|AbstractResponse|TransactionManager|Admin|KafkaAdminClient|MockAdminClient).java"/>
     <suppress checks="ClassDataAbstractionCoupling"
               files="(Errors|SaslAuthenticatorTest|AgentTest|CoordinatorTest).java"/>
 
diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java
index c6d9d14..468aa17 100644
--- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java
+++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.clients.admin;
 
+import org.apache.kafka.clients.admin.DescribeReplicaLogDirsResult.ReplicaLogDirInfo;
 import org.apache.kafka.common.KafkaFuture;
 import org.apache.kafka.common.Metric;
 import org.apache.kafka.common.MetricName;
@@ -29,10 +30,14 @@ import org.apache.kafka.common.acl.AclBinding;
 import org.apache.kafka.common.acl.AclBindingFilter;
 import org.apache.kafka.common.acl.AclOperation;
 import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.KafkaStorageException;
+import org.apache.kafka.common.errors.ReplicaNotAvailableException;
 import org.apache.kafka.common.errors.TimeoutException;
 import org.apache.kafka.common.errors.TopicExistsException;
 import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
 import org.apache.kafka.common.internals.KafkaFutureImpl;
+import org.apache.kafka.common.requests.DescribeLogDirsResponse;
 import org.apache.kafka.common.quota.ClientQuotaAlteration;
 import org.apache.kafka.common.quota.ClientQuotaFilter;
 
@@ -49,51 +54,109 @@ import java.util.Set;
 public class MockAdminClient extends AdminClient {
     public static final String DEFAULT_CLUSTER_ID = "I4ZmrWqfT2e-upky_4fdPA";
 
+    public static final List<String> DEFAULT_LOG_DIRS =
+        Collections.singletonList("/tmp/kafka-logs");
+
     private final List<Node> brokers;
     private final Map<String, TopicMetadata> allTopics = new HashMap<>();
+    private final Map<TopicPartition, NewPartitionReassignment> reassignments =
+        new HashMap<>();
+    private final Map<TopicPartitionReplica, ReplicaLogDirInfo> replicaMoves =
+        new HashMap<>();
     private final String clusterId;
+    private final List<List<String>> brokerLogDirs;
+    private final List<Map<String, String>> brokerConfigs;
 
     private Node controller;
     private int timeoutNextRequests = 0;
 
     private Map<MetricName, Metric> mockMetrics = new HashMap<>();
 
-    /**
-     * Creates MockAdminClient for a cluster with the given brokers. The Kafka cluster ID uses the default value from
-     * DEFAULT_CLUSTER_ID.
-     *
-     * @param brokers list of brokers in the cluster
-     * @param controller node that should start as the controller
-     */
-    public MockAdminClient(List<Node> brokers, Node controller) {
-        this(brokers, controller, DEFAULT_CLUSTER_ID);
-    }
-
-    /**
-     * Creates MockAdminClient for a cluster with the given brokers.
-     * @param brokers list of brokers in the cluster
-     * @param controller node that should start as the controller
-     */
-    public MockAdminClient(List<Node> brokers, Node controller, String clusterId) {
+    public static class Builder {
+        private String clusterId = DEFAULT_CLUSTER_ID;
+        private List<Node> brokers = new ArrayList<>();
+        private Node controller = null;
+        private List<List<String>> brokerLogDirs = new ArrayList<>();
+
+        public Builder() {
+            numBrokers(1);
+        }
+
+        public Builder clusterId(String clusterId) {
+            this.clusterId = clusterId;
+            return this;
+        }
+
+        public Builder brokers(List<Node> brokers) {
+            numBrokers(brokers.size());
+            this.brokers = brokers;
+            return this;
+        }
+
+        public Builder numBrokers(int numBrokers) {
+            if (brokers.size() >= numBrokers) {
+                brokers = brokers.subList(0, numBrokers);
+                brokerLogDirs = brokerLogDirs.subList(0, numBrokers);
+            } else {
+                for (int id = brokers.size(); id < numBrokers; id++) {
+                    brokers.add(new Node(id, "localhost", 1000 + id));
+                    brokerLogDirs.add(DEFAULT_LOG_DIRS);
+                }
+            }
+            return this;
+        }
+
+        public Builder controller(int index) {
+            this.controller = brokers.get(index);
+            return this;
+        }
+
+        public Builder brokerLogDirs(List<List<String>> brokerLogDirs) {
+            this.brokerLogDirs = brokerLogDirs;
+            return this;
+        }
+
+        public MockAdminClient build() {
+            return new MockAdminClient(brokers,
+                controller == null ? brokers.get(0) : controller,
+                clusterId,
+                brokerLogDirs);
+        }
+    }
+
+    public MockAdminClient(List<Node> brokers,
+                           Node controller) {
+        this(brokers, controller, DEFAULT_CLUSTER_ID,
+            Collections.nCopies(brokers.size(), DEFAULT_LOG_DIRS));
+    }
+
+    private MockAdminClient(List<Node> brokers,
+                            Node controller,
+                            String clusterId,
+                            List<List<String>> brokerLogDirs) {
         this.brokers = brokers;
         controller(controller);
         this.clusterId = clusterId;
+        this.brokerLogDirs = brokerLogDirs;
+        this.brokerConfigs = new ArrayList<>();
+        for (int i = 0; i < brokers.size(); i++) {
+            this.brokerConfigs.add(new HashMap<>());
+        }
     }
 
-    public void controller(Node controller) {
+    synchronized public void controller(Node controller) {
         if (!brokers.contains(controller))
             throw new IllegalArgumentException("The controller node must be in the list of brokers");
         this.controller = controller;
     }
 
-    public void addTopic(boolean internal,
+    synchronized public void addTopic(boolean internal,
                          String name,
                          List<TopicPartitionInfo> partitions,
                          Map<String, String> configs) {
         if (allTopics.containsKey(name)) {
             throw new IllegalArgumentException(String.format("Topic %s was already added.", name));
         }
-        List<Node> replicas = null;
         for (TopicPartitionInfo partition : partitions) {
             if (!brokers.contains(partition.leader())) {
                 throw new IllegalArgumentException("Leader broker unknown");
@@ -104,18 +167,15 @@ public class MockAdminClient extends AdminClient {
             if (!brokers.containsAll(partition.isr())) {
                 throw new IllegalArgumentException("Unknown brokers in isr list");
             }
-
-            if (replicas == null) {
-                replicas = partition.replicas();
-            } else if (!replicas.equals(partition.replicas())) {
-                throw new IllegalArgumentException("All partitions need to have the same replica nodes.");
-            }
         }
-
-        allTopics.put(name, new TopicMetadata(internal, partitions, configs));
+        ArrayList<String> logDirs = new ArrayList<>();
+        for (int i = 0; i < partitions.size(); i++) {
+            logDirs.add(brokerLogDirs.get(partitions.get(i).leader().id()).get(0));
+        }
+        allTopics.put(name, new TopicMetadata(internal, partitions, logDirs, configs));
     }
 
-    public void markTopicForDeletion(final String name) {
+    synchronized public void markTopicForDeletion(final String name) {
         if (!allTopics.containsKey(name)) {
             throw new IllegalArgumentException(String.format("Topic %s did not exist.", name));
         }
@@ -123,12 +183,12 @@ public class MockAdminClient extends AdminClient {
         allTopics.get(name).markedForDeletion = true;
     }
 
-    public void timeoutNextRequest(int numberOfRequest) {
+    synchronized public void timeoutNextRequest(int numberOfRequest) {
         timeoutNextRequests = numberOfRequest;
     }
 
     @Override
-    public DescribeClusterResult describeCluster(DescribeClusterOptions options) {
+    synchronized public DescribeClusterResult describeCluster(DescribeClusterOptions options) {
         KafkaFutureImpl<Collection<Node>> nodesFuture = new KafkaFutureImpl<>();
         KafkaFutureImpl<Node> controllerFuture = new KafkaFutureImpl<>();
         KafkaFutureImpl<String> brokerIdFuture = new KafkaFutureImpl<>();
@@ -151,7 +211,7 @@ public class MockAdminClient extends AdminClient {
     }
 
     @Override
-    public CreateTopicsResult createTopics(Collection<NewTopic> newTopics, CreateTopicsOptions options) {
+    synchronized public CreateTopicsResult createTopics(Collection<NewTopic> newTopics, CreateTopicsOptions options) {
         Map<String, KafkaFuture<CreateTopicsResult.TopicMetadataAndConfig>> createTopicResult = new HashMap<>();
 
         if (timeoutNextRequests > 0) {
@@ -189,10 +249,13 @@ public class MockAdminClient extends AdminClient {
 
             int numberOfPartitions = newTopic.numPartitions();
             List<TopicPartitionInfo> partitions = new ArrayList<>(numberOfPartitions);
-            for (int p = 0; p < numberOfPartitions; ++p) {
-                partitions.add(new TopicPartitionInfo(p, brokers.get(0), replicas, Collections.emptyList()));
+            // Partitions start off on the first log directory of each broker, for now.
+            List<String> logDirs = new ArrayList<>(numberOfPartitions);
+            for (int i = 0; i < numberOfPartitions; i++) {
+                partitions.add(new TopicPartitionInfo(i, brokers.get(0), replicas, Collections.emptyList()));
+                logDirs.add(brokerLogDirs.get(partitions.get(i).leader().id()).get(0));
             }
-            allTopics.put(topicName, new TopicMetadata(false, partitions, newTopic.configs()));
+            allTopics.put(topicName, new TopicMetadata(false, partitions, logDirs, newTopic.configs()));
             future.complete(null);
             createTopicResult.put(topicName, future);
         }
@@ -201,7 +264,7 @@ public class MockAdminClient extends AdminClient {
     }
 
     @Override
-    public ListTopicsResult listTopics(ListTopicsOptions options) {
+    synchronized public ListTopicsResult listTopics(ListTopicsOptions options) {
         Map<String, TopicListing> topicListings = new HashMap<>();
 
         if (timeoutNextRequests > 0) {
@@ -227,7 +290,7 @@ public class MockAdminClient extends AdminClient {
     }
 
     @Override
-    public DescribeTopicsResult describeTopics(Collection<String> topicNames, DescribeTopicsOptions options) {
+    synchronized public DescribeTopicsResult describeTopics(Collection<String> topicNames, DescribeTopicsOptions options) {
         Map<String, KafkaFuture<TopicDescription>> topicDescriptions = new HashMap<>();
 
         if (timeoutNextRequests > 0) {
@@ -268,7 +331,7 @@ public class MockAdminClient extends AdminClient {
     }
 
     @Override
-    public DeleteTopicsResult deleteTopics(Collection<String> topicsToDelete, DeleteTopicsOptions options) {
+    synchronized public DeleteTopicsResult deleteTopics(Collection<String> topicsToDelete, DeleteTopicsOptions options) {
         Map<String, KafkaFuture<Void>> deleteTopicsResult = new HashMap<>();
 
         if (timeoutNextRequests > 0) {
@@ -297,12 +360,12 @@ public class MockAdminClient extends AdminClient {
     }
 
     @Override
-    public CreatePartitionsResult createPartitions(Map<String, NewPartitions> newPartitions, CreatePartitionsOptions options) {
+    synchronized public CreatePartitionsResult createPartitions(Map<String, NewPartitions> newPartitions, CreatePartitionsOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public DeleteRecordsResult deleteRecords(Map<TopicPartition, RecordsToDelete> recordsToDelete, DeleteRecordsOptions options) {
+    synchronized public DeleteRecordsResult deleteRecords(Map<TopicPartition, RecordsToDelete> recordsToDelete, DeleteRecordsOptions options) {
         Map<TopicPartition, KafkaFuture<DeletedRecords>> deletedRecordsResult = new HashMap<>();
         if (recordsToDelete.isEmpty()) {
             return new DeleteRecordsResult(deletedRecordsResult);
@@ -312,58 +375,58 @@ public class MockAdminClient extends AdminClient {
     }
 
     @Override
-    public CreateDelegationTokenResult createDelegationToken(CreateDelegationTokenOptions options) {
+    synchronized public CreateDelegationTokenResult createDelegationToken(CreateDelegationTokenOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public RenewDelegationTokenResult renewDelegationToken(byte[] hmac, RenewDelegationTokenOptions options) {
+    synchronized public RenewDelegationTokenResult renewDelegationToken(byte[] hmac, RenewDelegationTokenOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public ExpireDelegationTokenResult expireDelegationToken(byte[] hmac, ExpireDelegationTokenOptions options) {
+    synchronized public ExpireDelegationTokenResult expireDelegationToken(byte[] hmac, ExpireDelegationTokenOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public DescribeDelegationTokenResult describeDelegationToken(DescribeDelegationTokenOptions options) {
+    synchronized public DescribeDelegationTokenResult describeDelegationToken(DescribeDelegationTokenOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public DescribeConsumerGroupsResult describeConsumerGroups(Collection<String> groupIds, DescribeConsumerGroupsOptions options) {
+    synchronized public DescribeConsumerGroupsResult describeConsumerGroups(Collection<String> groupIds, DescribeConsumerGroupsOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public ListConsumerGroupsResult listConsumerGroups(ListConsumerGroupsOptions options) {
+    synchronized public ListConsumerGroupsResult listConsumerGroups(ListConsumerGroupsOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, ListConsumerGroupOffsetsOptions options) {
+    synchronized public ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, ListConsumerGroupOffsetsOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public DeleteConsumerGroupsResult deleteConsumerGroups(Collection<String> groupIds, DeleteConsumerGroupsOptions options) {
+    synchronized public DeleteConsumerGroupsResult deleteConsumerGroups(Collection<String> groupIds, DeleteConsumerGroupsOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsets(String groupId, Set<TopicPartition> partitions, DeleteConsumerGroupOffsetsOptions options) {
+    synchronized public DeleteConsumerGroupOffsetsResult deleteConsumerGroupOffsets(String groupId, Set<TopicPartition> partitions, DeleteConsumerGroupOffsetsOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Deprecated
     @Override
-    public ElectPreferredLeadersResult electPreferredLeaders(Collection<TopicPartition> partitions, ElectPreferredLeadersOptions options) {
+    synchronized public ElectPreferredLeadersResult electPreferredLeaders(Collection<TopicPartition> partitions, ElectPreferredLeadersOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public ElectLeadersResult electLeaders(
+    synchronized public ElectLeadersResult electLeaders(
             ElectionType electionType,
             Set<TopicPartition> partitions,
             ElectLeadersOptions options) {
@@ -371,142 +434,372 @@ public class MockAdminClient extends AdminClient {
     }
 
     @Override
-    public RemoveMembersFromConsumerGroupResult removeMembersFromConsumerGroup(String groupId, RemoveMembersFromConsumerGroupOptions options) {
+    synchronized public RemoveMembersFromConsumerGroupResult removeMembersFromConsumerGroup(String groupId, RemoveMembersFromConsumerGroupOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public CreateAclsResult createAcls(Collection<AclBinding> acls, CreateAclsOptions options) {
+    synchronized public CreateAclsResult createAcls(Collection<AclBinding> acls, CreateAclsOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public DescribeAclsResult describeAcls(AclBindingFilter filter, DescribeAclsOptions options) {
+    synchronized public DescribeAclsResult describeAcls(AclBindingFilter filter, DescribeAclsOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public DeleteAclsResult deleteAcls(Collection<AclBindingFilter> filters, DeleteAclsOptions options) {
+    synchronized public DeleteAclsResult deleteAcls(Collection<AclBindingFilter> filters, DeleteAclsOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public DescribeConfigsResult describeConfigs(Collection<ConfigResource> resources, DescribeConfigsOptions options) {
-        Map<ConfigResource, KafkaFuture<Config>> configescriptions = new HashMap<>();
-
+    synchronized public DescribeConfigsResult describeConfigs(Collection<ConfigResource> resources, DescribeConfigsOptions options) {
+        Map<ConfigResource, KafkaFuture<Config>> results = new HashMap<>();
         for (ConfigResource resource : resources) {
-            if (resource.type() == ConfigResource.Type.TOPIC) {
-                Map<String, String> configs = allTopics.get(resource.name()).configs;
-                List<ConfigEntry> configEntries = new ArrayList<>();
-                for (Map.Entry<String, String> entry : configs.entrySet()) {
-                    configEntries.add(new ConfigEntry(entry.getKey(), entry.getValue()));
+            KafkaFutureImpl<Config> future = new KafkaFutureImpl<>();
+            results.put(resource, future);
+            try {
+                future.complete(getResourceDescription(resource));
+            } catch (Throwable e) {
+                future.completeExceptionally(e);
+            }
+        }
+        return new DescribeConfigsResult(results);
+    }
+
+    synchronized private Config getResourceDescription(ConfigResource resource) {
+        switch (resource.type()) {
+            case BROKER: {
+                Map<String, String> map =
+                    brokerConfigs.get(Integer.valueOf(resource.name()));
+                if (map == null) {
+                    throw new InvalidRequestException("Broker " + resource.name() +
+                        " not found.");
                 }
-                KafkaFutureImpl<Config> future = new KafkaFutureImpl<>();
-                future.complete(new Config(configEntries));
-                configescriptions.put(resource, future);
-            } else {
-                throw new UnsupportedOperationException("Not implemented yet");
+                return toConfigObject(map);
+            }
+            case TOPIC: {
+                TopicMetadata topicMetadata = allTopics.get(resource.name());
+                if (topicMetadata == null) {
+                    throw new UnknownTopicOrPartitionException();
+                }
+                return toConfigObject(topicMetadata.configs);
             }
+            default:
+                throw new UnsupportedOperationException("Not implemented yet");
         }
+    }
 
-        return new DescribeConfigsResult(configescriptions);
+    private static Config toConfigObject(Map<String, String> map) {
+        List<ConfigEntry> configEntries = new ArrayList<>();
+        for (Map.Entry<String, String> entry : map.entrySet()) {
+            configEntries.add(new ConfigEntry(entry.getKey(), entry.getValue()));
+        }
+        return new Config(configEntries);
     }
 
     @Override
     @Deprecated
-    public AlterConfigsResult alterConfigs(Map<ConfigResource, Config> configs, AlterConfigsOptions options) {
+    synchronized public AlterConfigsResult alterConfigs(Map<ConfigResource, Config> configs, AlterConfigsOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public AlterConfigsResult incrementalAlterConfigs(Map<ConfigResource, Collection<AlterConfigOp>> configs,
-                                                      AlterConfigsOptions options) {
-        throw new UnsupportedOperationException("Not implemented yet");
+    synchronized public AlterConfigsResult incrementalAlterConfigs(
+            Map<ConfigResource, Collection<AlterConfigOp>> configs,
+            AlterConfigsOptions options) {
+        Map<ConfigResource, KafkaFuture<Void>> futures = new HashMap<>();
+        for (Map.Entry<ConfigResource, Collection<AlterConfigOp>> entry :
+                configs.entrySet()) {
+            ConfigResource resource = entry.getKey();
+            KafkaFutureImpl<Void> future = new KafkaFutureImpl<>();
+            futures.put(resource, future);
+            Throwable throwable =
+                handleIncrementalResourceAlteration(resource, entry.getValue());
+            if (throwable == null) {
+                future.complete(null);
+            } else {
+                future.completeExceptionally(throwable);
+            }
+        }
+        return new AlterConfigsResult(futures);
+    }
+
+    synchronized private Throwable handleIncrementalResourceAlteration(
+            ConfigResource resource, Collection<AlterConfigOp> ops) {
+        switch (resource.type()) {
+            case BROKER: {
+                int brokerId;
+                try {
+                    brokerId = Integer.valueOf(resource.name());
+                } catch (NumberFormatException e) {
+                    return e;
+                }
+                Map<String, String> map = brokerConfigs.get(brokerId);
+                if (map == null) {
+                    return new InvalidRequestException("no such broker as " + brokerId);
+                }
+                HashMap<String, String> newMap = new HashMap<>(map);
+                for (AlterConfigOp op : ops) {
+                    switch (op.opType()) {
+                        case SET:
+                            newMap.put(op.configEntry().name(), op.configEntry().value());
+                            break;
+                        case DELETE:
+                            newMap.remove(op.configEntry().name());
+                            break;
+                        default:
+                            return new InvalidRequestException(
+                                "Unsupported op type " + op.opType());
+                    }
+                }
+                brokerConfigs.set(brokerId, newMap);
+                return null;
+            }
+            case TOPIC: {
+                TopicMetadata topicMetadata = allTopics.get(resource.name());
+                if (topicMetadata == null) {
+                    return new UnknownTopicOrPartitionException("No such topic as " +
+                        resource.name());
+                }
+                HashMap<String, String> newMap = new HashMap<>(topicMetadata.configs);
+                for (AlterConfigOp op : ops) {
+                    switch (op.opType()) {
+                        case SET:
+                            newMap.put(op.configEntry().name(), op.configEntry().value());
+                            break;
+                        case DELETE:
+                            newMap.remove(op.configEntry().name());
+                            break;
+                        default:
+                            return new InvalidRequestException(
+                                "Unsupported op type " + op.opType());
+                    }
+                }
+                topicMetadata.configs = newMap;
+                return null;
+            }
+            default:
+                return new UnsupportedOperationException();
+        }
     }
 
     @Override
-    public AlterReplicaLogDirsResult alterReplicaLogDirs(Map<TopicPartitionReplica, String> replicaAssignment, AlterReplicaLogDirsOptions options) {
-        throw new UnsupportedOperationException("Not implemented yet");
+    synchronized public AlterReplicaLogDirsResult alterReplicaLogDirs(
+            Map<TopicPartitionReplica, String> replicaAssignment,
+            AlterReplicaLogDirsOptions options) {
+        Map<TopicPartitionReplica, KafkaFuture<Void>> results = new HashMap<>();
+        for (Map.Entry<TopicPartitionReplica, String> entry : replicaAssignment.entrySet()) {
+            TopicPartitionReplica replica = entry.getKey();
+            String newLogDir = entry.getValue();
+            KafkaFutureImpl<Void> future = new KafkaFutureImpl<>();
+            results.put(replica, future);
+            List<String> dirs = brokerLogDirs.get(replica.brokerId());
+            if (dirs == null) {
+                future.completeExceptionally(
+                    new ReplicaNotAvailableException("Can't find " + replica));
+            } else if (!dirs.contains(newLogDir)) {
+                future.completeExceptionally(
+                    new KafkaStorageException("Log directory " + newLogDir + " is offline"));
+            } else {
+                TopicMetadata metadata = allTopics.get(replica.topic());
+                if (metadata == null || metadata.partitions.size() <= replica.partition()) {
+                    future.completeExceptionally(
+                        new ReplicaNotAvailableException("Can't find " + replica));
+                } else {
+                    String currentLogDir = metadata.partitionLogDirs.get(replica.partition());
+                    replicaMoves.put(replica,
+                        new ReplicaLogDirInfo(currentLogDir, 0, newLogDir, 0));
+                    future.complete(null);
+                }
+            }
+        }
+        return new AlterReplicaLogDirsResult(results);
     }
 
     @Override
-    public DescribeLogDirsResult describeLogDirs(Collection<Integer> brokers, DescribeLogDirsOptions options) {
+    synchronized public DescribeLogDirsResult describeLogDirs(Collection<Integer> brokers,
+                                                              DescribeLogDirsOptions options) {
         throw new UnsupportedOperationException("Not implemented yet");
     }
 
     @Override
-    public DescribeReplicaLogDirsResult describeReplicaLogDirs(Collection<TopicPartitionReplica> replicas, DescribeReplicaLogDirsOptions options) {
-        throw new UnsupportedOperationException("Not implemented yet");
+    synchronized public DescribeReplicaLogDirsResult describeReplicaLogDirs(
+            Collection<TopicPartitionReplica> replicas, DescribeReplicaLogDirsOptions options) {
+        Map<TopicPartitionReplica, KafkaFuture<ReplicaLogDirInfo>> results = new HashMap<>();
+        for (TopicPartitionReplica replica : replicas) {
+            TopicMetadata topicMetadata = allTopics.get(replica.topic());
+            if (topicMetadata != null) {
+                KafkaFutureImpl<ReplicaLogDirInfo> future = new KafkaFutureImpl<>();
+                results.put(replica, future);
+                String currentLogDir = currentLogDir(replica);
+                if (currentLogDir == null) {
+                    future.complete(new ReplicaLogDirInfo(null,
+                        DescribeLogDirsResponse.INVALID_OFFSET_LAG,
+                        null,
+                        DescribeLogDirsResponse.INVALID_OFFSET_LAG));
+                } else {
+                    ReplicaLogDirInfo info = replicaMoves.get(replica);
+                    if (info == null) {
+                        future.complete(new ReplicaLogDirInfo(currentLogDir, 0, null, 0));
+                    } else {
+                        future.complete(info);
+                    }
+                }
+            }
+        }
+        return new DescribeReplicaLogDirsResult(results);
     }
 
-    @Override
-    public AlterPartitionReassignmentsResult alterPartitionReassignments(Map<TopicPartition, Optional<NewPartitionReassignment>> reassignments,
-                                                                         AlterPartitionReassignmentsOptions options) {
-        throw new UnsupportedOperationException("Not implemented yet");
+    private synchronized String currentLogDir(TopicPartitionReplica replica) {
+        TopicMetadata topicMetadata = allTopics.get(replica.topic());
+        if (topicMetadata == null) {
+            return null;
+        }
+        if (topicMetadata.partitionLogDirs.size() <= replica.partition()) {
+            return null;
+        }
+        return topicMetadata.partitionLogDirs.get(replica.partition());
+    }
+
+    @Override
+    synchronized public AlterPartitionReassignmentsResult alterPartitionReassignments(
+            Map<TopicPartition, Optional<NewPartitionReassignment>> newReassignments,
+            AlterPartitionReassignmentsOptions options) {
+        Map<TopicPartition, KafkaFuture<Void>> futures = new HashMap<>();
+        for (Map.Entry<TopicPartition, Optional<NewPartitionReassignment>> entry :
+                newReassignments.entrySet()) {
+            TopicPartition partition = entry.getKey();
+            Optional<NewPartitionReassignment> newReassignment = entry.getValue();
+            KafkaFutureImpl<Void> future = new KafkaFutureImpl<Void>();
+            futures.put(partition, future);
+            TopicMetadata topicMetadata = allTopics.get(partition.topic());
+            if (partition.partition() < 0 ||
+                    topicMetadata == null ||
+                    topicMetadata.partitions.size() <= partition.partition()) {
+                future.completeExceptionally(new UnknownTopicOrPartitionException());
+            } else if (newReassignment.isPresent()) {
+                reassignments.put(partition, newReassignment.get());
+                future.complete(null);
+            } else {
+                reassignments.remove(partition);
+                future.complete(null);
+            }
+        }
+        return new AlterPartitionReassignmentsResult(futures);
     }
 
     @Override
-    public ListPartitionReassignmentsResult listPartitionReassignments(Optional<Set<TopicPartition>> partitions, ListPartitionReassignmentsOptions options) {
-        throw new UnsupportedOperationException("Not implemented yet");
+    synchronized public ListPartitionReassignmentsResult listPartitionReassignments(
+            Optional<Set<TopicPartition>> partitions,
+            ListPartitionReassignmentsOptions options) {
+        Map<TopicPartition, PartitionReassignment> map = new HashMap<>();
+        for (TopicPartition partition : partitions.isPresent() ?
+                partitions.get() : reassignments.keySet()) {
+            PartitionReassignment reassignment = findPartitionReassignment(partition);
+            if (reassignment != null) {
+                map.put(partition, reassignment);
+            }
+        }
+        return new ListPartitionReassignmentsResult(KafkaFutureImpl.completedFuture(map));
+    }
+
+    synchronized private PartitionReassignment findPartitionReassignment(TopicPartition partition) {
+        NewPartitionReassignment reassignment = reassignments.get(partition);
+        if (reassignment == null) {
+            return null;
+        }
+        TopicMetadata metadata = allTopics.get(partition.topic());
+        if (metadata == null) {
+            throw new RuntimeException("Internal MockAdminClient logic error: found " +
+                "reassignment for " + partition + ", but no TopicMetadata");
+        }
+        TopicPartitionInfo info = metadata.partitions.get(partition.partition());
+        if (info == null) {
+            throw new RuntimeException("Internal MockAdminClient logic error: found " +
+                "reassignment for " + partition + ", but no TopicPartitionInfo");
+        }
+        List<Integer> replicas = new ArrayList<>();
+        List<Integer> removingReplicas = new ArrayList<>();
+        List<Integer> addingReplicas = new ArrayList<>(reassignment.targetReplicas());
+        for (Node node : info.replicas()) {
+            replicas.add(node.id());
+            if (!reassignment.targetReplicas().contains(node.id())) {
+                removingReplicas.add(node.id());
+            }
+            addingReplicas.remove(Integer.valueOf(node.id()));
+        }
+        return new PartitionReassignment(replicas, addingReplicas, removingReplicas);
     }
 
     @Override
-    public AlterConsumerGroupOffsetsResult alterConsumerGroupOffsets(String groupId, Map<TopicPartition, OffsetAndMetadata> offsets, AlterConsumerGroupOffsetsOptions options) {
+    synchronized public AlterConsumerGroupOffsetsResult alterConsumerGroupOffsets(String groupId, Map<TopicPartition, OffsetAndMetadata> offsets, AlterConsumerGroupOffsetsOptions options) {
         throw new UnsupportedOperationException("Not implement yet");
     }
 
     @Override
-    public ListOffsetsResult listOffsets(Map<TopicPartition, OffsetSpec> topicPartitionOffsets, ListOffsetsOptions options) {
+    synchronized public ListOffsetsResult listOffsets(Map<TopicPartition, OffsetSpec> topicPartitionOffsets, ListOffsetsOptions options) {
         throw new UnsupportedOperationException("Not implement yet");
     }
 
     @Override
     public DescribeClientQuotasResult describeClientQuotas(ClientQuotaFilter filter, DescribeClientQuotasOptions options) {
-        throw new UnsupportedOperationException("Not implemented yet");
+        throw new UnsupportedOperationException("Not implement yet");
     }
 
     @Override
     public AlterClientQuotasResult alterClientQuotas(Collection<ClientQuotaAlteration> entries, AlterClientQuotasOptions options) {
-        throw new UnsupportedOperationException("Not implemented yet");
+        throw new UnsupportedOperationException("Not implement yet");
     }
 
     @Override
-    public void close(Duration timeout) {}
-
+    synchronized public void close(Duration timeout) {}
 
     private final static class TopicMetadata {
         final boolean isInternalTopic;
         final List<TopicPartitionInfo> partitions;
-        final Map<String, String> configs;
+        final List<String> partitionLogDirs;
+        Map<String, String> configs;
         int fetchesRemainingUntilVisible;
 
         public boolean markedForDeletion;
 
         TopicMetadata(boolean isInternalTopic,
                       List<TopicPartitionInfo> partitions,
+                      List<String> partitionLogDirs,
                       Map<String, String> configs) {
             this.isInternalTopic = isInternalTopic;
             this.partitions = partitions;
+            this.partitionLogDirs = partitionLogDirs;
             this.configs = configs != null ? configs : Collections.emptyMap();
             this.markedForDeletion = false;
             this.fetchesRemainingUntilVisible = 0;
         }
     }
 
-    public void setMockMetrics(MetricName name, Metric metric) {
+    synchronized public void setMockMetrics(MetricName name, Metric metric) {
         mockMetrics.put(name, metric);
     }
 
     @Override
-    public Map<MetricName, ? extends Metric> metrics() {
+    synchronized public Map<MetricName, ? extends Metric> metrics() {
         return mockMetrics;
     }
 
-    public void setFetchesRemainingUntilVisible(String topicName, int fetchesRemainingUntilVisible) {
+    synchronized public void setFetchesRemainingUntilVisible(String topicName, int fetchesRemainingUntilVisible) {
         TopicMetadata metadata = allTopics.get(topicName);
         if (metadata == null) {
             throw new RuntimeException("No such topic as " + topicName);
         }
         metadata.fetchesRemainingUntilVisible = fetchesRemainingUntilVisible;
     }
+
+    synchronized public List<Node> brokers() {
+        return new ArrayList<>(brokers);
+    }
+
+    synchronized public Node broker(int index) {
+        return brokers.get(index);
+    }
 }
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/ConnectUtilsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ConnectUtilsTest.java
index 6be3525..f7e092c6 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/ConnectUtilsTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ConnectUtilsTest.java
@@ -34,8 +34,8 @@ public class ConnectUtilsTest {
         final Node broker1 = new Node(0, "dummyHost-1", 1234);
         final Node broker2 = new Node(1, "dummyHost-2", 1234);
         List<Node> cluster = Arrays.asList(broker1, broker2);
-        MockAdminClient adminClient = new MockAdminClient(cluster, broker1);
-
+        MockAdminClient adminClient = new MockAdminClient.Builder().
+            brokers(cluster).build();
         assertEquals(MockAdminClient.DEFAULT_CLUSTER_ID, ConnectUtils.lookupKafkaClusterId(adminClient));
     }
 
@@ -44,8 +44,8 @@ public class ConnectUtilsTest {
         final Node broker1 = new Node(0, "dummyHost-1", 1234);
         final Node broker2 = new Node(1, "dummyHost-2", 1234);
         List<Node> cluster = Arrays.asList(broker1, broker2);
-        MockAdminClient adminClient = new MockAdminClient(cluster, broker1, null);
-
+        MockAdminClient adminClient = new MockAdminClient.Builder().
+            brokers(cluster).clusterId(null).build();
         assertNull(ConnectUtils.lookupKafkaClusterId(adminClient));
     }
 
@@ -54,7 +54,8 @@ public class ConnectUtilsTest {
         final Node broker1 = new Node(0, "dummyHost-1", 1234);
         final Node broker2 = new Node(1, "dummyHost-2", 1234);
         List<Node> cluster = Arrays.asList(broker1, broker2);
-        MockAdminClient adminClient = new MockAdminClient(cluster, broker1);
+        MockAdminClient adminClient = new MockAdminClient.Builder().
+            brokers(cluster).build();
         adminClient.timeoutNextRequest(1);
 
         ConnectUtils.lookupKafkaClusterId(adminClient);
diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala
index bee457d..28b2d01 100755
--- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala
+++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala
@@ -16,233 +16,1575 @@
  */
 package kafka.admin
 
-import java.util.Properties
+import java.util
+import java.util.Optional
 import java.util.concurrent.ExecutionException
 
 import kafka.common.AdminCommandFailedException
 import kafka.log.LogConfig
-import kafka.log.LogConfig._
 import kafka.server.{ConfigType, DynamicConfig}
-import kafka.utils._
+import kafka.utils.{CommandDefaultOptions, CommandLineUtils, CoreUtils, Exit, Json, Logging}
 import kafka.utils.json.JsonValue
 import kafka.zk.{AdminZkClient, KafkaZkClient}
-import org.apache.kafka.clients.admin.DescribeReplicaLogDirsResult.ReplicaLogDirInfo
-import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterReplicaLogDirsOptions}
-import org.apache.kafka.common.errors.ReplicaNotAvailableException
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType
+import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp, ConfigEntry, NewPartitionReassignment, PartitionReassignment, TopicDescription}
+import org.apache.kafka.common.config.ConfigResource
+import org.apache.kafka.common.errors.{ReplicaNotAvailableException, UnknownTopicOrPartitionException}
 import org.apache.kafka.common.security.JaasUtils
 import org.apache.kafka.common.utils.{Time, Utils}
-import org.apache.kafka.common.{TopicPartition, TopicPartitionReplica}
-import org.apache.zookeeper.KeeperException.NodeExistsException
+import org.apache.kafka.common.{KafkaException, KafkaFuture, TopicPartition, TopicPartitionReplica}
 
 import scala.collection.JavaConverters._
-import scala.collection._
+import scala.collection.{Map, Seq, mutable}
+import scala.compat.java8.OptionConverters._
+import scala.math.Ordered.orderingToOrdered
 
 object ReassignPartitionsCommand extends Logging {
-
-  case class Throttle(interBrokerLimit: Long, replicaAlterLogDirsLimit: Long = -1, postUpdateAction: () => Unit = () => ())
-
-  private[admin] val NoThrottle = Throttle(-1, -1)
   private[admin] val AnyLogDir = "any"
 
+  val helpText = "This tool helps to move topic partitions between replicas."
+
+  /**
+   * The earliest version of the partition reassignment JSON.  We will default to this
+   * version if no other version number is given.
+   */
   private[admin] val EarliestVersion = 1
 
-  val helpText = "This tool helps to moves topic partitions between replicas."
+  /**
+   * The earliest version of the JSON for each partition reassignment topic.  We will
+   * default to this version if no other version number is given.
+   */
+  private[admin] val EarliestTopicsJsonVersion = 1
+
+  // Throttles that are set at the level of an individual broker.
+  private[admin] val brokerLevelLeaderThrottle =
+    DynamicConfig.Broker.LeaderReplicationThrottledRateProp
+  private[admin] val brokerLevelFollowerThrottle =
+    DynamicConfig.Broker.FollowerReplicationThrottledRateProp
+  private[admin] val brokerLevelLogDirThrottle =
+    DynamicConfig.Broker.ReplicaAlterLogDirsIoMaxBytesPerSecondProp
+  private[admin] val brokerLevelThrottles = Seq(
+    brokerLevelLeaderThrottle,
+    brokerLevelFollowerThrottle,
+    brokerLevelLogDirThrottle
+  )
+
+  // Throttles that are set at the level of an individual topic.
+  private[admin] val topicLevelLeaderThrottle =
+    LogConfig.LeaderReplicationThrottledReplicasProp
+  private[admin] val topicLevelFollowerThrottle =
+    LogConfig.FollowerReplicationThrottledReplicasProp
+  private[admin] val topicLevelThrottles = Seq(
+    topicLevelLeaderThrottle,
+    topicLevelFollowerThrottle
+  )
+
+  private[admin] val cannotExecuteBecauseOfExistingMessage = "Cannot execute because " +
+    "there is an existing partition assignment.  Use --additional to override this and " +
+    "create a new partition assignment in addition to the existing one."
+
+  private[admin] val youMustRunVerifyPeriodicallyMessage = "Warning: You must run " +
+    "--verify periodically, until the reassignment completes, to ensure the throttle " +
+    "is removed."
+
+  /**
+   * A map from topic names to partition movements.
+   */
+  type MoveMap = mutable.Map[String, mutable.Map[Int, PartitionMove]]
+
+  /**
+   * A partition movement.  The source and destination brokers may overlap.
+   *
+   * @param sources         The source brokers.
+   * @param destinations    The destination brokers.
+   */
+  sealed case class PartitionMove(sources: mutable.Set[Int],
+                                  destinations: mutable.Set[Int]) { }
+
+  /**
+   * The state of a partition reassignment.  The current replicas and target replicas
+   * may overlap.
+   *
+   * @param currentReplicas The current replicas.
+   * @param targetReplicas  The target replicas.
+   * @param done            True if the reassignment is done.
+   */
+  sealed case class PartitionReassignmentState(currentReplicas: Seq[Int],
+                                               targetReplicas: Seq[Int],
+                                               done: Boolean) {}
+
+  /**
+   * The state of a replica log directory movement.
+   */
+  sealed trait LogDirMoveState {
+    /**
+     * True if the move is done without errors.
+     */
+    def done: Boolean
+  }
+
+  /**
+   * A replica log directory move state where the source log directory is missing.
+   *
+   * @param targetLogDir        The log directory that we wanted the replica to move to.
+   */
+  sealed case class MissingReplicaMoveState(targetLogDir: String)
+      extends LogDirMoveState {
+    override def done = false
+  }
+
+  /**
+   * A replica log directory move state where the source replica is missing.
+   *
+   * @param targetLogDir        The log directory that we wanted the replica to move to.
+   */
+  sealed case class MissingLogDirMoveState(targetLogDir: String)
+      extends LogDirMoveState {
+    override def done = false
+  }
+
+  /**
+   * A replica log directory move state where the move is in progress.
+   *
+   * @param currentLogDir       The current log directory.
+   * @param futureLogDir        The log directory that the replica is moving to.
+   * @param targetLogDir        The log directory that we wanted the replica to move to.
+   */
+  sealed case class ActiveMoveState(currentLogDir: String,
+                                    targetLogDir: String,
+                                    futureLogDir: String)
+      extends LogDirMoveState {
+    override def done = false
+  }
+
+  /**
+   * A replica log directory move state where there is no move in progress, but we did not
+   * reach the target log directory.
+   *
+   * @param currentLogDir       The current log directory.
+   * @param targetLogDir        The log directory that we wanted the replica to move to.
+   */
+  sealed case class CancelledMoveState(currentLogDir: String,
+                                       targetLogDir: String)
+      extends LogDirMoveState {
+    override def done = true
+  }
+
+  /**
+   * The completed replica log directory move state.
+   *
+   * @param targetLogDir        The log directory that we wanted the replica to move to.
+   */
+  sealed case class CompletedMoveState(targetLogDir: String)
+      extends LogDirMoveState {
+    override def done = true
+  }
+
+  /**
+   * An exception thrown to indicate that the command has failed, but we don't want to
+   * print a stack trace.
+   *
+   * @param message     The message to print out before exiting.  A stack trace will not
+   *                    be printed.
+   */
+  class TerseReassignmentFailureException(message: String) extends KafkaException(message) {
+  }
 
   def main(args: Array[String]): Unit = {
     val opts = validateAndParseArgs(args)
-    val zkConnect = opts.options.valueOf(opts.zkConnectOpt)
-    val time = Time.SYSTEM
-    val zkClient = KafkaZkClient(zkConnect, JaasUtils.isZkSaslEnabled, 30000, 30000, Int.MaxValue, time)
-
-    val adminClientOpt = createAdminClient(opts)
+    var toClose: Option[AutoCloseable] = None
+    var failed = true
 
     try {
-      if(opts.options.has(opts.verifyOpt))
-        verifyAssignment(zkClient, adminClientOpt, opts)
-      else if(opts.options.has(opts.generateOpt))
-        generateAssignment(zkClient, opts)
-      else if (opts.options.has(opts.executeOpt))
-        executeAssignment(zkClient, adminClientOpt, opts)
+      if (opts.options.has(opts.bootstrapServerOpt)) {
+        if (opts.options.has(opts.zkConnectOpt)) {
+          println("Warning: ignoring deprecated --zookeeper option because " +
+            "--bootstrap-server was specified.  The --zookeeper option will " +
+            "be removed in a future version of Kafka.")
+        }
+        val props = if (opts.options.has(opts.commandConfigOpt))
+          Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt))
+        else
+          new util.Properties()
+        props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt))
+        props.putIfAbsent(AdminClientConfig.CLIENT_ID_CONFIG, "reassign-partitions-tool")
+        val adminClient = Admin.create(props)
+        toClose = Some(adminClient)
+        handleAction(adminClient, opts)
+      } else {
+        println("Warning: --zookeeper is deprecated, and will be removed in a future " +
+          "version of Kafka.")
+        val zkClient = KafkaZkClient(opts.options.valueOf(opts.zkConnectOpt),
+          JaasUtils.isZkSaslEnabled, 30000, 30000, Int.MaxValue, Time.SYSTEM)
+        toClose = Some(zkClient)
+        handleAction(zkClient, opts)
+      }
+      failed = false
     } catch {
+      case e: TerseReassignmentFailureException =>
+        println(e.getMessage)
       case e: Throwable =>
-        println("Partitions reassignment failed due to " + e.getMessage)
+        println("Error: " + e.getMessage)
         println(Utils.stackTrace(e))
-    } finally zkClient.close()
-  }
-
-  private def createAdminClient(opts: ReassignPartitionsCommandOptions): Option[Admin] = {
-    if (opts.options.has(opts.bootstrapServerOpt)) {
-      val props = if (opts.options.has(opts.commandConfigOpt))
-        Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt))
-      else
-        new Properties()
-      props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt))
-      props.putIfAbsent(AdminClientConfig.CLIENT_ID_CONFIG, "reassign-partitions-tool")
-      Some(Admin.create(props))
+    } finally {
+      // Close the AdminClient or ZooKeeper client, as appropriate.
+      // It's good to do this after printing any error stack trace.
+      toClose.foreach(_.close())
+    }
+    // If the command failed, exit with a non-zero exit code.
+    if (failed) {
+      Exit.exit(1)
+    }
+  }
+
+  private def handleAction(adminClient: Admin,
+                           opts: ReassignPartitionsCommandOptions): Unit = {
+    if (opts.options.has(opts.verifyOpt)) {
+      verifyAssignment(adminClient,
+        Utils.readFileAsString(opts.options.valueOf(opts.reassignmentJsonFileOpt)),
+        opts.options.has(opts.preserveThrottlesOpt))
+    } else if (opts.options.has(opts.generateOpt)) {
+      generateAssignment(adminClient,
+        Utils.readFileAsString(opts.options.valueOf(opts.topicsToMoveJsonFileOpt)),
+        opts.options.valueOf(opts.brokerListOpt),
+        !opts.options.has(opts.disableRackAware))
+    } else if (opts.options.has(opts.executeOpt)) {
+      executeAssignment(adminClient,
+        opts.options.has(opts.additionalOpt),
+        Utils.readFileAsString(opts.options.valueOf(opts.reassignmentJsonFileOpt)),
+        opts.options.valueOf(opts.interBrokerThrottleOpt),
+        opts.options.valueOf(opts.replicaAlterLogDirsThrottleOpt),
+        opts.options.valueOf(opts.timeoutOpt))
+    } else if (opts.options.has(opts.cancelOpt)) {
+      cancelAssignment(adminClient,
+        Utils.readFileAsString(opts.options.valueOf(opts.reassignmentJsonFileOpt)),
+        opts.options.has(opts.preserveThrottlesOpt),
+        opts.options.valueOf(opts.timeoutOpt))
+    } else if (opts.options.has(opts.listOpt)) {
+      listReassignments(adminClient)
     } else {
-      None
+      throw new RuntimeException("Unsupported action.")
     }
   }
 
-  def verifyAssignment(zkClient: KafkaZkClient, adminClientOpt: Option[Admin], opts: ReassignPartitionsCommandOptions): Unit = {
-    val jsonFile = opts.options.valueOf(opts.reassignmentJsonFileOpt)
-    val jsonString = Utils.readFileAsString(jsonFile)
-    verifyAssignment(zkClient, adminClientOpt, jsonString)
+  private def handleAction(zkClient: KafkaZkClient,
+                           opts: ReassignPartitionsCommandOptions): Unit = {
+    if (opts.options.has(opts.verifyOpt)) {
+      verifyAssignment(zkClient,
+        Utils.readFileAsString(opts.options.valueOf(opts.reassignmentJsonFileOpt)),
+        opts.options.has(opts.preserveThrottlesOpt))
+    } else if (opts.options.has(opts.generateOpt)) {
+      generateAssignment(zkClient,
+        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(zkClient,
+        Utils.readFileAsString(opts.options.valueOf(opts.reassignmentJsonFileOpt)),
+        opts.options.valueOf(opts.interBrokerThrottleOpt))
+    } else {
+      throw new RuntimeException("Unsupported action.")
+    }
   }
 
-  def verifyAssignment(zkClient: KafkaZkClient, adminClientOpt: Option[Admin], jsonString: String): Unit = {
-    println("Status of partition reassignment: ")
-    val adminZkClient = new AdminZkClient(zkClient)
-    val (partitionsToBeReassigned, replicaAssignment) = parsePartitionReassignmentData(jsonString)
-    val reassignedPartitionsStatus = checkIfPartitionReassignmentSucceeded(zkClient, partitionsToBeReassigned.toMap)
-    val replicasReassignmentStatus = checkIfReplicaReassignmentSucceeded(adminClientOpt, replicaAssignment)
-
-    reassignedPartitionsStatus.foreach { case (topicPartition, status) =>
-      status match {
-        case ReassignmentCompleted =>
-          println("Reassignment of partition %s completed successfully".format(topicPartition))
-        case ReassignmentFailed =>
-          println("Reassignment of partition %s failed".format(topicPartition))
-        case ReassignmentInProgress =>
-          println("Reassignment of partition %s is still in progress".format(topicPartition))
-      }
+  /**
+   * A result returned from verifyAssignment.
+   *
+   * @param partStates    A map from partitions to reassignment states.
+   * @param partsOngoing  True if there are any ongoing partition reassignments.
+   * @param moveStates    A map from log directories to movement states.
+   * @param movesOngoing  True if there are any ongoing moves that we know about.
+   */
+  case class VerifyAssignmentResult(partStates: Map[TopicPartition, PartitionReassignmentState],
+                                    partsOngoing: Boolean = false,
+                                    moveStates: Map[TopicPartitionReplica, LogDirMoveState] = Map.empty,
+                                    movesOngoing: Boolean = false)
+
+  /**
+   * The entry point for the --verify command.
+   *
+   * @param adminClient           The AdminClient to use.
+   * @param jsonString            The JSON string to use for the topics and partitions to verify.
+   * @param preserveThrottles     True if we should avoid changing topic or broker throttles.
+   *
+   * @returns                     A result that is useful for testing.
+   */
+  def verifyAssignment(adminClient: Admin, jsonString: String, preserveThrottles: Boolean)
+                      : VerifyAssignmentResult = {
+    val (targetParts, targetLogDirs) = parsePartitionReassignmentData(jsonString)
+    val (partStates, partsOngoing) = verifyPartitionAssignments(adminClient, targetParts)
+    val (moveStates, movesOngoing) = verifyReplicaMoves(adminClient, targetLogDirs)
+    if (!partsOngoing && !movesOngoing && !preserveThrottles) {
+      // If the partition assignments and replica assignments are done, clear any throttles
+      // that were set.  We have to clear all throttles, because we don't have enough
+      // information to know all of the source brokers that might have been involved in the
+      // previous reassignments.
+      clearAllThrottles(adminClient, targetParts)
     }
+    VerifyAssignmentResult(partStates, partsOngoing, moveStates, movesOngoing)
+  }
+
+  /**
+   * Verify the partition reassignments specified by the user.
+   *
+   * @param adminClient           The AdminClient to use.
+   * @param targets               The partition reassignments specified by the user.
+   *
+   * @return                      A tuple of the partition reassignment states, and a
+   *                              boolean which is true if there are no ongoing
+   *                              reassignments (including reassignments not described
+   *                              in the JSON file.)
+   */
+  def verifyPartitionAssignments(adminClient: Admin,
+                                 targets: Seq[(TopicPartition, Seq[Int])])
+                                 : (Map[TopicPartition, PartitionReassignmentState], Boolean) = {
+    val (partStates, partsOngoing) = findPartitionReassignmentStates(adminClient, targets)
+    println(partitionReassignmentStatesToString(partStates))
+    (partStates, partsOngoing)
+  }
 
-    replicasReassignmentStatus.foreach { case (replica, status) =>
-      status match {
-        case ReassignmentCompleted =>
-          println("Reassignment of replica %s completed successfully".format(replica))
-        case ReassignmentFailed =>
-          println("Reassignment of replica %s failed".format(replica))
-        case ReassignmentInProgress =>
-          println("Reassignment of replica %s is still in progress".format(replica))
+  /**
+   * The deprecated entry point for the --verify command.
+   *
+   * @param zkClient              The ZooKeeper client 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.
+   *
+   * @returns                     A result that is useful for testing.  Note that anything that
+   *                              would require AdminClient to see will be left out of this result.
+   */
+  def verifyAssignment(zkClient: KafkaZkClient, jsonString: String, preserveThrottles: Boolean)
+                       : VerifyAssignmentResult = {
+    val (targetParts, targetLogDirs) = parsePartitionReassignmentData(jsonString)
+    if (targetLogDirs.nonEmpty) {
+      throw new AdminCommandFailedException("bootstrap-server needs to be provided when " +
+        "replica reassignments are present.")
+    }
+    println("Warning: because you are using the deprecated --zookeeper option, the results " +
+      "may be incomplete.  Use --bootstrap-server instead for more accurate results.")
+    val (partStates, partsOngoing) = verifyPartitionAssignments(zkClient, targetParts.toMap)
+    if (!partsOngoing && !preserveThrottles) {
+      clearAllThrottles(zkClient, targetParts)
+    }
+    VerifyAssignmentResult(partStates, partsOngoing, Map.empty, false)
+  }
+
+  /**
+   * Verify the partition reassignments specified by the user.
+   *
+   * @param zkClient              The ZooKeeper client to use.
+   * @param targets               The partition reassignments specified by the user.
+   *
+   * @returns                     A tuple of partition states and whether there are any
+   *                              ongoing reassignments found in the legacy reassign
+   *                              partitions ZNode.
+   */
+  def verifyPartitionAssignments(zkClient: KafkaZkClient,
+                                 targets: Map[TopicPartition, Seq[Int]])
+                                 : (Map[TopicPartition, PartitionReassignmentState], Boolean) = {
+    val (partStates, partsOngoing) = findPartitionReassignmentStates(zkClient, targets)
+    println(partitionReassignmentStatesToString(partStates))
+    (partStates, partsOngoing)
+  }
+
+  def compareTopicPartitions(a: TopicPartition, b: TopicPartition): Boolean = {
+    (a.topic(), a.partition()) < (b.topic(), b.partition())
+  }
+
+  def compareTopicPartitionReplicas(a: TopicPartitionReplica, b: TopicPartitionReplica): Boolean = {
+    (a.brokerId(), a.topic(), a.partition()) < (b.brokerId(), b.topic(), b.partition())
+  }
+
+  /**
+   * Convert partition reassignment states to a human-readable string.
+   *
+   * @param states      A map from topic partitions to states.
+   * @return            A string summarizing the partition reassignment states.
+   */
+  def partitionReassignmentStatesToString(states: Map[TopicPartition, PartitionReassignmentState])
+                                          : String = {
+    val bld = new mutable.ArrayBuffer[String]()
+    bld.append("Status of partition reassignment:")
+    states.keySet.toBuffer.sortWith(compareTopicPartitions).foreach {
+      topicPartition => {
+        val state = states(topicPartition)
+        if (state.done) {
+          if (state.currentReplicas.equals(state.targetReplicas)) {
+            bld.append("Reassignment of partition %s is complete.".
+              format(topicPartition.toString))
+          } else {
+            bld.append(s"There is no active reassignment of partition ${topicPartition}, " +
+              s"but replica set is ${state.currentReplicas.mkString(",")} rather than " +
+              s"${state.targetReplicas.mkString(",")}.")
+          }
+        } else {
+          bld.append("Reassignment of partition %s is still in progress.".format(topicPartition))
+        }
       }
     }
-    removeThrottle(zkClient, reassignedPartitionsStatus, replicasReassignmentStatus, adminZkClient)
-  }
-
-  private[admin] def removeThrottle(zkClient: KafkaZkClient,
-                                    reassignedPartitionsStatus: Map[TopicPartition, ReassignmentStatus],
-                                    replicasReassignmentStatus: Map[TopicPartitionReplica, ReassignmentStatus],
-                                    adminZkClient: AdminZkClient): Unit = {
-
-    //If both partition assignment and replica reassignment have completed remove both the inter-broker and replica-alter-dir throttle
-    if (reassignedPartitionsStatus.forall { case (_, status) => status == ReassignmentCompleted } &&
-        replicasReassignmentStatus.forall { case (_, status) => status == ReassignmentCompleted }) {
-      var changed = false
-
-      //Remove the throttle limit from all brokers in the cluster
-      //(as we no longer know which specific brokers were involved in the move)
-      for (brokerId <- zkClient.getAllBrokersInCluster.map(_.id)) {
-        val configs = adminZkClient.fetchEntityConfig(ConfigType.Broker, brokerId.toString)
-        // bitwise OR as we don't want to short-circuit
-        if (configs.remove(DynamicConfig.Broker.LeaderReplicationThrottledRateProp) != null
-          | configs.remove(DynamicConfig.Broker.FollowerReplicationThrottledRateProp) != null
-          | configs.remove(DynamicConfig.Broker.ReplicaAlterLogDirsIoMaxBytesPerSecondProp) != null){
-          adminZkClient.changeBrokerConfig(Seq(brokerId), configs)
-          changed = true
+    bld.mkString(System.lineSeparator())
+  }
+
+  /**
+   * Find the state of the specified partition reassignments.
+   *
+   * @param adminClient          The Admin client to use.
+   * @param targetReassignments  The reassignments we want to learn about.
+   *
+   * @return                     A tuple containing the reassignment states for each topic
+   *                             partition, plus whether there are any ongoing reassignments.
+   */
+  def findPartitionReassignmentStates(adminClient: Admin,
+                                      targetReassignments: Seq[(TopicPartition, Seq[Int])])
+                                      : (Map[TopicPartition, PartitionReassignmentState], Boolean) = {
+    val currentReassignments = adminClient.
+      listPartitionReassignments().reassignments().get().asScala
+    val (foundReassignments, notFoundReassignments) = targetReassignments.partition {
+      case (part, _) => currentReassignments.contains(part)
+    }
+    val foundResults: Seq[(TopicPartition, PartitionReassignmentState)] = foundReassignments.map {
+      case (part, targetReplicas) => (part,
+        new PartitionReassignmentState(
+          currentReassignments.get(part).get.replicas().
+            asScala.map(i => i.asInstanceOf[Int]),
+          targetReplicas,
+          false))
+    }
+    val topicNamesToLookUp = new mutable.HashSet[String]()
+    notFoundReassignments.foreach {
+      case (part, targetReplicas) =>
+        if (!currentReassignments.contains(part))
+          topicNamesToLookUp.add(part.topic())
+    }
+    val topicDescriptions = adminClient.
+      describeTopics(topicNamesToLookUp.asJava).values().asScala
+    val notFoundResults: Seq[(TopicPartition, PartitionReassignmentState)] = notFoundReassignments.map {
+      case (part, targetReplicas) =>
+        currentReassignments.get(part) match {
+          case Some(reassignment) => (part,
+            new PartitionReassignmentState(
+              reassignment.replicas().asScala.map(_.asInstanceOf[Int]),
+              targetReplicas,
+              false))
+          case None => {
+            (part, topicDescriptionFutureToState(part.partition(),
+              topicDescriptions(part.topic()), targetReplicas))
+          }
         }
+    }
+    val allResults = foundResults ++ notFoundResults
+    (allResults.toMap, currentReassignments.nonEmpty)
+  }
+
+  private def topicDescriptionFutureToState(partition: Int,
+                                            future: KafkaFuture[TopicDescription],
+                                            targetReplicas: Seq[Int])
+                                            : PartitionReassignmentState = {
+    try {
+      val topicDescription = future.get()
+      if (topicDescription.partitions().size() < partition) {
+        throw new ExecutionException("Too few partitions found", new UnknownTopicOrPartitionException())
       }
+      new PartitionReassignmentState(
+        topicDescription.partitions().get(partition).replicas().asScala.map(_.id),
+        targetReplicas,
+        true)
+    } catch {
+      case t: ExecutionException =>
+        t.getCause match {
+          case _: UnknownTopicOrPartitionException =>
+            new PartitionReassignmentState(Seq(), targetReplicas, true)
+        }
+    }
+  }
 
-      //Remove the list of throttled replicas from all topics with partitions being moved
-      val topics = (reassignedPartitionsStatus.keySet.map(tp => tp.topic) ++ replicasReassignmentStatus.keySet.map(replica => replica.topic)).toSeq.distinct
-      for (topic <- topics) {
-        val configs = adminZkClient.fetchEntityConfig(ConfigType.Topic, topic)
-        // bitwise OR as we don't want to short-circuit
-        if (configs.remove(LogConfig.LeaderReplicationThrottledReplicasProp) != null
-          | configs.remove(LogConfig.FollowerReplicationThrottledReplicasProp) != null) {
-          adminZkClient.changeTopicConfig(topic, configs)
-          changed = true
+  /**
+   * Find the state of the specified partition reassignments.
+   *
+   * @param zkClient              The ZooKeeper 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
+   *                              found in the legacy reassign partitions znode.
+   */
+  def findPartitionReassignmentStates(zkClient: KafkaZkClient,
+                                      targetReassignments: Map[TopicPartition, Seq[Int]])
+                                      : (Map[TopicPartition, PartitionReassignmentState], Boolean) = {
+    val partitionsBeingReassigned = zkClient.getPartitionReassignment
+    val results = new mutable.HashMap[TopicPartition, PartitionReassignmentState]()
+    targetReassignments.groupBy(_._1.topic).foreach {
+      case (topic, partitions) =>
+        val replicasForTopic = zkClient.getReplicaAssignmentForTopics(Set(topic))
+        partitions.foreach {
+          case (partition, targetReplicas) =>
+            val currentReplicas = replicasForTopic.getOrElse(partition, Seq())
+            results.put(partition, new PartitionReassignmentState(
+              currentReplicas, targetReplicas, !partitionsBeingReassigned.contains(partition)))
         }
+    }
+    (results, partitionsBeingReassigned.nonEmpty)
+  }
+
+  /**
+   * Verify the replica reassignments specified by the user.
+   *
+   * @param adminClient           The AdminClient to use.
+   * @param targetReassignments   The replica reassignments specified by the user.
+   *
+   * @return                      A tuple of the replica states, and a boolean which is true
+   *                              if there are any ongoing replica moves.
+   *
+   *                              Note: Unlike in verifyPartitionAssignments, we will
+   *                              return false here even if there are unrelated ongoing
+   *                              reassignments. (We don't have an efficient API that
+   *                              returns all ongoing replica reassignments.)
+   */
+  def verifyReplicaMoves(adminClient: Admin,
+                         targetReassignments: Map[TopicPartitionReplica, String])
+                         : (Map[TopicPartitionReplica, LogDirMoveState], Boolean) = {
+    val moveStates = findLogDirMoveStates(adminClient, targetReassignments)
+    println(replicaMoveStatesToString(moveStates))
+    (moveStates, !moveStates.values.forall(_.done))
+  }
+
+  /**
+   * Find the state of the specified partition reassignments.
+   *
+   * @param adminClient           The AdminClient to use.
+   * @param targetMoves           The movements we want to learn about.  The map is keyed
+   *                              by TopicPartitionReplica, and its values are target log
+   *                              directories.
+   *
+   * @return                      The states for each replica movement.
+   */
+  def findLogDirMoveStates(adminClient: Admin,
+                           targetMoves: Map[TopicPartitionReplica, String])
+                           : Map[TopicPartitionReplica, LogDirMoveState] = {
+    val replicaLogDirInfos = adminClient.describeReplicaLogDirs(
+      targetMoves.keySet.asJava).all().get().asScala
+    targetMoves.map { case (replica, targetLogDir) =>
+      val moveState: LogDirMoveState = replicaLogDirInfos.get(replica) match {
+        case None => MissingReplicaMoveState(targetLogDir)
+        case Some(info) => if (info.getCurrentReplicaLogDir == null) {
+            MissingLogDirMoveState(targetLogDir)
+          } else if (info.getFutureReplicaLogDir == null) {
+            if (info.getCurrentReplicaLogDir.equals(targetLogDir)) {
+              CompletedMoveState(targetLogDir)
+            } else {
+              CancelledMoveState(info.getCurrentReplicaLogDir, targetLogDir)
+            }
+          } else {
+            ActiveMoveState(info.getCurrentReplicaLogDir(),
+              targetLogDir,
+              info.getFutureReplicaLogDir)
+          }
       }
-      if (changed)
-        println("Throttle was removed.")
+      (replica, moveState)
     }
   }
 
-  def generateAssignment(zkClient: KafkaZkClient, opts: ReassignPartitionsCommandOptions): Unit = {
-    val topicsToMoveJsonFile = opts.options.valueOf(opts.topicsToMoveJsonFileOpt)
-    val brokerListToReassign = opts.options.valueOf(opts.brokerListOpt).split(',').map(_.toInt)
-    val duplicateReassignments = CoreUtils.duplicates(brokerListToReassign)
-    if (duplicateReassignments.nonEmpty)
-      throw new AdminCommandFailedException("Broker list contains duplicate entries: %s".format(duplicateReassignments.mkString(",")))
-    val topicsToMoveJsonString = Utils.readFileAsString(topicsToMoveJsonFile)
-    val disableRackAware = opts.options.has(opts.disableRackAware)
-    val (proposedAssignments, currentAssignments) = generateAssignment(zkClient, brokerListToReassign, topicsToMoveJsonString, disableRackAware)
-    println("Current partition replica assignment\n%s\n".format(formatAsReassignmentJson(currentAssignments, Map.empty)))
-    println("Proposed partition reassignment configuration\n%s".format(formatAsReassignmentJson(proposedAssignments, Map.empty)))
+  /**
+   * Convert replica move states to a human-readable string.
+   *
+   * @param states          A map from topic partition replicas to states.
+   * @return                A tuple of a summary string, and a boolean describing
+   *                        whether there are any active replica moves.
+   */
+  def replicaMoveStatesToString(states: Map[TopicPartitionReplica, LogDirMoveState])
+                                : String = {
+    val bld = new mutable.ArrayBuffer[String]
+    states.keySet.toBuffer.sortWith(compareTopicPartitionReplicas).foreach {
+      case replica =>
+        val state = states(replica)
+        state match {
+          case MissingLogDirMoveState(targetLogDir) =>
+            bld.append(s"Partition ${replica.topic()}-${replica.partition()} is not found " +
+              s"in any live log dir on broker ${replica.brokerId()}. There is likely an " +
+              s"offline log directory on the broker.")
+          case MissingReplicaMoveState(targetLogDir) =>
+            bld.append(s"Partition ${replica.topic()}-${replica.partition()} cannot be found " +
+              s"in any live log directory on broker ${replica.brokerId()}.")
+          case ActiveMoveState(currentLogDir, targetLogDir, futureLogDir) =>
+            if (targetLogDir.equals(futureLogDir)) {
+              bld.append(s"Reassignment of replica ${replica} is still in progress.")
+            } else {
+              bld.append(s"Partition ${replica.topic()}-${replica.partition()} on broker " +
+                s"${replica.brokerId()} is being moved to log dir ${futureLogDir} " +
+                s"instead of ${targetLogDir}.")
+            }
+          case CancelledMoveState(currentLogDir, targetLogDir) =>
+            bld.append(s"Partition ${replica.topic()}-${replica.partition()} on broker " +
+              s"${replica.brokerId()} is not being moved from log dir ${currentLogDir} to " +
+              s"${targetLogDir}.")
+          case CompletedMoveState(targetLogDir) =>
+            bld.append(s"Reassignment of replica ${replica} completed successfully.")
+        }
+    }
+    bld.mkString(System.lineSeparator())
   }
 
-  def generateAssignment(zkClient: KafkaZkClient, brokerListToReassign: Seq[Int], topicsToMoveJsonString: String, disableRackAware: Boolean): (Map[TopicPartition, Seq[Int]], Map[TopicPartition, Seq[Int]]) = {
-    val topicsToReassign = parseTopicsData(topicsToMoveJsonString)
-    val duplicateTopicsToReassign = CoreUtils.duplicates(topicsToReassign)
-    if (duplicateTopicsToReassign.nonEmpty)
-      throw new AdminCommandFailedException("List of topics to reassign contains duplicate entries: %s".format(duplicateTopicsToReassign.mkString(",")))
-    val currentAssignment = zkClient.getReplicaAssignmentForTopics(topicsToReassign.toSet)
+  /**
+   * Clear all topic-level and broker-level throttles.
+   *
+   * @param adminClient     The AdminClient to use.
+   * @param targetParts     The target partitions loaded from the JSON file.
+   */
+  def clearAllThrottles(adminClient: Admin,
+                        targetParts: Seq[(TopicPartition, Seq[Int])]): Unit = {
+    val activeBrokers = adminClient.describeCluster().nodes().get().asScala.map(_.id()).toSet
+    val brokers = activeBrokers ++ targetParts.map(_._2).flatten.toSet
+    println("Clearing broker-level throttles on broker%s %s".format(
+      if (brokers.size == 1) "" else "s", brokers.mkString(",")))
+    clearBrokerLevelThrottles(adminClient, brokers)
+
+    val topics = targetParts.map(_._1.topic()).toSet
+    println("Clearing topic-level throttles on topic%s %s".format(
+      if (topics.size == 1) "" else "s", topics.mkString(",")))
+    clearTopicLevelThrottles(adminClient, topics)
+  }
 
-    val groupedByTopic = currentAssignment.groupBy { case (tp, _) => tp.topic }
-    val rackAwareMode = if (disableRackAware) RackAwareMode.Disabled else RackAwareMode.Enforced
+  /**
+   * Clear all topic-level and broker-level throttles.
+   *
+   * @param zkClient        The ZooKeeper client to use.
+   * @param targetParts     The target partitions loaded from the JSON file.
+   */
+  def clearAllThrottles(zkClient: KafkaZkClient,
+                        targetParts: Seq[(TopicPartition, Seq[Int])]): Unit = {
+    val activeBrokers = zkClient.getAllBrokersInCluster.map(_.id).toSet
+    val brokers = activeBrokers ++ targetParts.map(_._2).flatten.toSet
+    println("Clearing broker-level throttles on broker%s %s".format(
+      if (brokers.size == 1) "" else "s", brokers.mkString(",")))
+    clearBrokerLevelThrottles(zkClient, brokers)
+
+    val topics = targetParts.map(_._1.topic()).toSet
+    println("Clearing topic-level throttles on topic%s %s".format(
+      if (topics.size == 1) "" else "s", topics.mkString(",")))
+    clearTopicLevelThrottles(zkClient, topics)
+  }
+
+  /**
+   * Clear all throttles which have been set at the broker level.
+   *
+   * @param adminClient       The AdminClient to use.
+   * @param brokers           The brokers to clear the throttles for.
+   */
+  def clearBrokerLevelThrottles(adminClient: Admin, brokers: Set[Int]): Unit = {
+    val configOps = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]()
+    brokers.foreach {
+      case brokerId => configOps.put(
+        new ConfigResource(ConfigResource.Type.BROKER, brokerId.toString),
+        brokerLevelThrottles.map(throttle => new AlterConfigOp(
+          new ConfigEntry(throttle, null), OpType.DELETE)).asJava)
+    }
+    adminClient.incrementalAlterConfigs(configOps).all().get()
+  }
+
+  /**
+   * Clear all throttles which have been set at the broker level.
+   *
+   * @param zkClient          The ZooKeeper client to use.
+   * @param brokers           The brokers to clear the throttles for.
+   */
+  def clearBrokerLevelThrottles(zkClient: KafkaZkClient, brokers: Set[Int]): Unit = {
+    val adminZkClient = new AdminZkClient(zkClient)
+    for (brokerId <- brokers) {
+      val configs = adminZkClient.fetchEntityConfig(ConfigType.Broker, brokerId.toString)
+      if (brokerLevelThrottles.flatMap(throttle => Option(configs.remove(throttle))).nonEmpty) {
+        adminZkClient.changeBrokerConfig(Seq(brokerId), configs)
+      }
+    }
+  }
+
+  /**
+   * Clear the reassignment throttles for the specified topics.
+   *
+   * @param adminClient           The AdminClient to use.
+   * @param topics                The topics to clear the throttles for.
+   */
+  def clearTopicLevelThrottles(adminClient: Admin, topics: Set[String]): Unit = {
+    val configOps = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]()
+    topics.foreach {
+      topicName => configOps.put(
+        new ConfigResource(ConfigResource.Type.TOPIC, topicName),
+        topicLevelThrottles.map(throttle => new AlterConfigOp(new ConfigEntry(throttle, null),
+          OpType.DELETE)).asJava)
+    }
+    adminClient.incrementalAlterConfigs(configOps).all().get()
+  }
+
+  /**
+   * Clear the reassignment throttles for the specified topics.
+   *
+   * @param zkClient              The ZooKeeper client to use.
+   * @param topics                The topics to clear the throttles for.
+   */
+  def clearTopicLevelThrottles(zkClient: KafkaZkClient, topics: Set[String]): Unit = {
     val adminZkClient = new AdminZkClient(zkClient)
-    val brokerMetadatas = adminZkClient.getBrokerMetadatas(rackAwareMode, Some(brokerListToReassign))
+    for (topic <- topics) {
+      val configs = adminZkClient.fetchEntityConfig(ConfigType.Topic, topic)
+      if (topicLevelThrottles.flatMap(throttle => Option(configs.remove(throttle))).nonEmpty) {
+        adminZkClient.changeTopicConfig(topic, configs)
+      }
+    }
+  }
+
+  /**
+   * The entry point for the --generate command.
+   *
+   * @param adminClient           The AdminClient to use.
+   * @param reassignmentJson      The JSON string to use for the topics to reassign.
+   * @param brokerListString      The comma-separated string of broker IDs to use.
+   * @param enableRackAwareness   True if rack-awareness should be enabled.
+   *
+   * @return                      A tuple containing the proposed assignment and the
+   *                              current assignment.
+   */
+  def generateAssignment(adminClient: Admin,
+                         reassignmentJson: String,
+                         brokerListString: String,
+                         enableRackAwareness: Boolean)
+                         : (Map[TopicPartition, Seq[Int]], Map[TopicPartition, Seq[Int]]) = {
+    val (brokersToReassign, topicsToReassign) =
+      parseGenerateAssignmentArgs(reassignmentJson, brokerListString)
+    val currentAssignments = getReplicaAssignmentForTopics(adminClient, topicsToReassign)
+    val brokerMetadatas = getBrokerMetadata(adminClient, brokersToReassign, enableRackAwareness)
+    val proposedAssignments = calculateAssignment(currentAssignments, brokerMetadatas)
+    println("Current partition replica assignment\n%s\n".
+      format(formatAsReassignmentJson(currentAssignments, Map.empty)))
+    println("Proposed partition reassignment configuration\n%s".
+      format(formatAsReassignmentJson(proposedAssignments, Map.empty)))
+    (proposedAssignments, currentAssignments)
+  }
 
-    val partitionsToBeReassigned = mutable.Map[TopicPartition, Seq[Int]]()
+  /**
+   * The legacy entry point for the --generate command.
+   *
+   * @param zkClient              The ZooKeeper client to use.
+   * @param reassignmentJson      The JSON string to use for the topics to reassign.
+   * @param brokerListString      The comma-separated string of broker IDs to use.
+   * @param enableRackAwareness   True if rack-awareness should be enabled.
+   *
+   * @return                      A tuple containing the proposed assignment and the
+   *                              current assignment.
+   */
+  def generateAssignment(zkClient: KafkaZkClient,
+                         reassignmentJson: String,
+                         brokerListString: String,
+                         enableRackAwareness: Boolean)
+                         : (Map[TopicPartition, Seq[Int]], Map[TopicPartition, Seq[Int]]) = {
+    val (brokersToReassign, topicsToReassign) =
+      parseGenerateAssignmentArgs(reassignmentJson, brokerListString)
+    val currentAssignments = zkClient.getReplicaAssignmentForTopics(topicsToReassign.toSet)
+    val brokerMetadatas = getBrokerMetadata(zkClient, brokersToReassign, enableRackAwareness)
+    val proposedAssignments = calculateAssignment(currentAssignments, brokerMetadatas)
+    println("Current partition replica assignment\n%s\n".
+      format(formatAsReassignmentJson(currentAssignments, Map.empty)))
+    println("Proposed partition reassignment configuration\n%s".
+      format(formatAsReassignmentJson(proposedAssignments, Map.empty)))
+    (proposedAssignments, currentAssignments)
+  }
+
+  /**
+   * Calculate the new partition assignments to suggest in --generate.
+   *
+   * @param currentAssignment  The current partition assignments.
+   * @param brokerMetadatas    The rack information for each broker.
+   *
+   * @return                   A map from partitions to the proposed assignments for each.
+   */
+  def calculateAssignment(currentAssignment: Map[TopicPartition, Seq[Int]],
+                          brokerMetadatas: Seq[BrokerMetadata])
+                          : Map[TopicPartition, Seq[Int]] = {
+    val groupedByTopic = currentAssignment.groupBy { case (tp, _) => tp.topic }
+    val proposedAssignments = mutable.Map[TopicPartition, Seq[Int]]()
     groupedByTopic.foreach { case (topic, assignment) =>
       val (_, replicas) = assignment.head
-      val assignedReplicas = AdminUtils.assignReplicasToBrokers(brokerMetadatas, assignment.size, replicas.size)
-      partitionsToBeReassigned ++= assignedReplicas.map { case (partition, replicas) =>
+      val assignedReplicas = AdminUtils.
+        assignReplicasToBrokers(brokerMetadatas, assignment.size, replicas.size)
+      proposedAssignments ++= assignedReplicas.map { case (partition, replicas) =>
         new TopicPartition(topic, partition) -> replicas
       }
     }
-    (partitionsToBeReassigned, currentAssignment)
+    proposedAssignments
   }
 
-  def executeAssignment(zkClient: KafkaZkClient, adminClientOpt: Option[Admin], opts: ReassignPartitionsCommandOptions): Unit = {
-    val reassignmentJsonFile =  opts.options.valueOf(opts.reassignmentJsonFileOpt)
-    val reassignmentJsonString = Utils.readFileAsString(reassignmentJsonFile)
-    val interBrokerThrottle = opts.options.valueOf(opts.interBrokerThrottleOpt)
-    val replicaAlterLogDirsThrottle = opts.options.valueOf(opts.replicaAlterLogDirsThrottleOpt)
-    val timeoutMs = opts.options.valueOf(opts.timeoutOpt)
-    executeAssignment(zkClient, adminClientOpt, reassignmentJsonString, Throttle(interBrokerThrottle, replicaAlterLogDirsThrottle), timeoutMs)
+  /**
+   * Get the current replica assignments for some topics.
+   *
+   * @param adminClient     The AdminClient to use.
+   * @param topics          The topics to get information about.
+   * @return                A map from partitions to broker assignments.
+   *                        If any topic can't be found, an exception will be thrown.
+   */
+  def getReplicaAssignmentForTopics(adminClient: Admin,
+                                    topics: Seq[String])
+                                    : Map[TopicPartition, Seq[Int]] = {
+    adminClient.describeTopics(topics.asJava).all().get().asScala.flatMap {
+      case (topicName, topicDescription) => {
+        topicDescription.partitions().asScala.map {
+          info => (new TopicPartition(topicName, info.partition()),
+            info.replicas().asScala.map(_.id()))
+        }
+      }
+    }
   }
 
-  def executeAssignment(zkClient: KafkaZkClient, adminClientOpt: Option[Admin], reassignmentJsonString: String, throttle: Throttle, timeoutMs: Long = 10000L): Unit = {
-    val (partitionAssignment, replicaAssignment) = parseAndValidate(zkClient, reassignmentJsonString)
+  /**
+   * Get the current replica assignments for some partitions.
+   *
+   * @param adminClient     The AdminClient to use.
+   * @param partitions      The partitions to get information about.
+   * @return                A map from partitions to broker assignments.
+   *                        If any topic can't be found, an exception will be thrown.
+   */
+  def getReplicaAssignmentForPartitions(adminClient: Admin,
+                                        partitions: Set[TopicPartition])
+                                        : Map[TopicPartition, Seq[Int]] = {
+    adminClient.describeTopics(partitions.map(_.topic).asJava).all().get().asScala.flatMap {
+      case (topicName, topicDescription) => {
+        topicDescription.partitions().asScala.flatMap {
+          info => if (partitions.contains(new TopicPartition(topicName, info.partition()))) {
+            Some(new TopicPartition(topicName, info.partition()),
+                info.replicas().asScala.map(_.id()))
+          } else {
+            None
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Find the rack information for some brokers.
+   *
+   * @param adminClient         The AdminClient object.
+   * @param brokers             The brokers to gather metadata about.
+   * @param enableRackAwareness True if we should return rack information, and throw an
+   *                            exception if it is inconsistent.
+   *
+   * @return                    The metadata for each broker that was found.
+   *                            Brokers that were not found will be omitted.
+   */
+  def getBrokerMetadata(adminClient: Admin,
+                        brokers: Seq[Int],
+                        enableRackAwareness: Boolean): Seq[BrokerMetadata] = {
+    val brokerSet = brokers.toSet
+    val results = adminClient.describeCluster().nodes().get().asScala.
+      filter(node => brokerSet.contains(node.id())).
+      map {
+        node => if (enableRackAwareness && node.rack() != null) {
+          new BrokerMetadata(node.id(), Some(node.rack()))
+        } else {
+          new BrokerMetadata(node.id(), None)
+        }
+      }.toSeq
+    val numRackless = results.count(_.rack.isEmpty)
+    if (enableRackAwareness && numRackless != 0 && numRackless != results.size) {
+      throw new AdminOperationException("Not all brokers have rack information. Add " +
+        "--disable-rack-aware in command line to make replica assignment without rack " +
+        "information.")
+    }
+    results
+  }
+
+  /**
+   * Find the metadata for some brokers.
+   *
+   * @param zkClient              The ZooKeeper client to use.
+   * @param brokers               The brokers to gather metadata about.
+   * @param enableRackAwareness   True if we should return rack information, and throw an
+   *                              exception if it is inconsistent.
+   *
+   * @return                      The metadata for each broker that was found.
+   *                              Brokers that were not found will be omitted.
+   */
+  def getBrokerMetadata(zkClient: KafkaZkClient,
+                        brokers: Seq[Int],
+                        enableRackAwareness: Boolean): Seq[BrokerMetadata] = {
     val adminZkClient = new AdminZkClient(zkClient)
-    val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, adminClientOpt, partitionAssignment.toMap, replicaAssignment, adminZkClient)
+    adminZkClient.getBrokerMetadatas(if (enableRackAwareness)
+      RackAwareMode.Enforced else RackAwareMode.Disabled, Some(brokers))
+  }
+
+  /**
+   * Parse and validate data gathered from the command-line for --generate
+   * In particular, we parse the JSON and validate that duplicate brokers and
+   * topics don't appear.
+   *
+   * @param reassignmentJson       The JSON passed to --generate .
+   * @param brokerList             A list of brokers passed to --generate.
+   *
+   * @return                       A tuple of brokers to reassign, topics to reassign
+   */
+  def parseGenerateAssignmentArgs(reassignmentJson: String,
+                                  brokerList: String): (Seq[Int], Seq[String]) = {
+    val brokerListToReassign = brokerList.split(',').map(_.toInt)
+    val duplicateReassignments = CoreUtils.duplicates(brokerListToReassign)
+    if (duplicateReassignments.nonEmpty)
+      throw new AdminCommandFailedException("Broker list contains duplicate entries: %s".
+        format(duplicateReassignments.mkString(",")))
+    val topicsToReassign = parseTopicsData(reassignmentJson)
+    val duplicateTopicsToReassign = CoreUtils.duplicates(topicsToReassign)
+    if (duplicateTopicsToReassign.nonEmpty)
+      throw new AdminCommandFailedException("List of topics to reassign contains duplicate entries: %s".
+        format(duplicateTopicsToReassign.mkString(",")))
+    (brokerListToReassign, topicsToReassign)
+  }
+
+  /**
+   * The entry point for the --execute and --execute-additional commands.
+   *
+   * @param adminClient                 The AdminClient to use.
+   * @param additional                  Whether --additional was passed.
+   * @param reassignmentJson            The JSON string to use for the topics to reassign.
+   * @param interBrokerThrottle         The inter-broker throttle to use, or a negative
+   *                                    number to skip using a throttle.
+   * @param logDirThrottle              The replica log directory throttle to use, or a
+   *                                    negative number to skip using a throttle.
+   * @param timeoutMs                   The maximum time in ms to wait for log directory
+   *                                    replica assignment to begin.
+   * @param time                        The Time object to use.
+   */
+  def executeAssignment(adminClient: Admin,
+                        additional: Boolean,
+                        reassignmentJson: String,
+                        interBrokerThrottle: Long = -1L,
+                        logDirThrottle: Long = -1L,
+                        timeoutMs: Long = 10000L,
+                        time: Time = Time.SYSTEM): Unit = {
+    val (proposedParts, proposedReplicas) = parseExecuteAssignmentArgs(reassignmentJson)
+    val currentReassignments = adminClient.
+      listPartitionReassignments().reassignments().get().asScala
+    // If there is an existing assignment, check for --additional before proceeding.
+    // This helps avoid surprising users.
+    if (!additional && currentReassignments.nonEmpty) {
+      throw new TerseReassignmentFailureException(cannotExecuteBecauseOfExistingMessage)
+    }
+    verifyBrokerIds(adminClient, proposedParts.values.flatten.toSet)
+    val currentParts = getReplicaAssignmentForPartitions(adminClient, proposedParts.keySet.toSet)
+    println(currentPartitionReplicaAssignmentToString(proposedParts, currentParts))
+    if (interBrokerThrottle >= 0 || logDirThrottle >= 0) {
+      println(youMustRunVerifyPeriodicallyMessage)
+      val moveMap = calculateMoveMap(currentReassignments, proposedParts, currentParts)
+      val leaderThrottles = calculateLeaderThrottles(moveMap)
+      val followerThrottles = calculateFollowerThrottles(moveMap)
+      modifyTopicThrottles(adminClient, leaderThrottles, followerThrottles)
+      val reassigningBrokers = calculateReassigningBrokers(moveMap)
+      val movingBrokers = calculateMovingBrokers(proposedReplicas.keySet.toSet)
+      modifyBrokerThrottles(adminClient,
+        reassigningBrokers, interBrokerThrottle,
+        movingBrokers, logDirThrottle)
+      if (interBrokerThrottle >= 0) {
+        println(s"The inter-broker throttle limit was set to ${interBrokerThrottle} B/s")
+      }
+      if (logDirThrottle >= 0) {
+        println(s"The replica-alter-dir throttle limit was set to ${logDirThrottle} B/s")
+      }
+    }
+
+    // Execute the partition reassignments.
+    val errors = alterPartitionReassignments(adminClient, proposedParts)
+    if (errors.nonEmpty) {
+      throw new TerseReassignmentFailureException(
+        "Error reassigning partition(s):%n%s".format(
+          errors.keySet.toBuffer.sortWith(compareTopicPartitions).map {
+            case part => s"${part}: ${errors(part).getMessage}"
+          }.mkString(System.lineSeparator())))
+    }
+    println("Successfully started partition reassignment%s for %s".format(
+      if (proposedParts.size == 1) "" else "s",
+      proposedParts.keySet.toBuffer.sortWith(compareTopicPartitions).mkString(",")))
+    if (proposedReplicas.nonEmpty) {
+      executeMoves(adminClient, proposedReplicas, timeoutMs, time)
+    }
+  }
 
-    // If there is an existing rebalance running, attempt to change its throttle
-    if (zkClient.reassignPartitionsInProgress()) {
-      println("There is an existing assignment running.")
-      reassignPartitionsCommand.maybeLimit(throttle)
+  /**
+   * Execute some partition log directory movements.
+   *
+   * @param adminClient                 The AdminClient to use.
+   * @param proposedReplicas            A map from TopicPartitionReplicas to the
+   *                                    directories to move them to.
+   * @param timeoutMs                   The maximum time in ms to wait for log directory
+   *                                    replica assignment to begin.
+   * @param time                        The Time object to use.
+   */
+  def executeMoves(adminClient: Admin,
+                   proposedReplicas: Map[TopicPartitionReplica, String],
+                   timeoutMs: Long,
+                   time: Time): Unit = {
+    val startTimeMs = time.milliseconds()
+    val pendingReplicas = new mutable.HashMap[TopicPartitionReplica, String]()
+    pendingReplicas ++= proposedReplicas
+    var done = false
+    do {
+      val completed = alterReplicaLogDirs(adminClient, pendingReplicas)
+      if (completed.nonEmpty) {
+        println("Successfully started log directory move%s for: %s".format(
+          if (completed.size == 1) "" else "s",
+          completed.toBuffer.sortWith(compareTopicPartitionReplicas).mkString(",")))
+      }
+      pendingReplicas --= completed
+      if (pendingReplicas.isEmpty) {
+        done = true
+      } else if (time.milliseconds() >= startTimeMs + timeoutMs) {
+        throw new TerseReassignmentFailureException(
+          "Timed out before log directory move%s could be started for: %s".format(
+            if (pendingReplicas.size == 1) "" else "s",
+            pendingReplicas.keySet.toBuffer.sortWith(compareTopicPartitionReplicas).
+              mkString(",")))
+      } else {
+        // If a replica has been moved to a new host and we also specified a particular
+        // log directory, we will have to keep retrying the alterReplicaLogDirs
+        // call.  It can't take effect until the replica is moved to that host.
+        time.sleep(100)
+      }
+    } while (!done)
+  }
+
+  /**
+   * Entry point for the --list command.
+   *
+   * @param adminClient   The AdminClient to use.
+   */
+  def listReassignments(adminClient: Admin): Unit = {
+    println(curReassignmentsToString(adminClient))
+  }
+
+  /**
+   * Convert the current partition reassignments to text.
+   *
+   * @param adminClient   The AdminClient to use.
+   * @return              A string describing the current partition reassignments.
+   */
+  def curReassignmentsToString(adminClient: Admin): String = {
+    val currentReassignments = adminClient.
+      listPartitionReassignments().reassignments().get().asScala
+    val text = currentReassignments.keySet.toBuffer.sortWith(compareTopicPartitions).map {
+      case part =>
+        val reassignment = currentReassignments(part)
+        val replicas = reassignment.replicas().asScala
+        val addingReplicas = reassignment.addingReplicas().asScala
+        val removingReplicas = reassignment.removingReplicas().asScala
+        "%s: replicas: %s.%s%s".format(part, replicas.mkString(","),
+          if (addingReplicas.isEmpty) "" else
+            " adding: %s.".format(addingReplicas.mkString(",")),
+          if (removingReplicas.isEmpty) "" else
+            " removing: %s.".format(removingReplicas.mkString(",")))
+    }.mkString(System.lineSeparator())
+    if (text.isEmpty) {
+      "No partition reassignments found."
     } else {
-      printCurrentAssignment(zkClient, partitionAssignment.map(_._1.topic))
-      if (throttle.interBrokerLimit >= 0 || throttle.replicaAlterLogDirsLimit >= 0)
-        println(String.format("Warning: You must run Verify periodically, until the reassignment completes, to ensure the throttle is removed. You can also alter the throttle by rerunning the Execute command passing a new value."))
-      if (reassignPartitionsCommand.reassignPartitions(throttle, timeoutMs)) {
-        println("Successfully started reassignment of partitions.")
-      } else
-        println("Failed to reassign partitions %s".format(partitionAssignment))
+      "Current partition reassignments:%n%s".format(text)
     }
   }
 
-  def printCurrentAssignment(zkClient: KafkaZkClient, topics: Seq[String]): Unit = {
-    // before starting assignment, output the current replica assignment to facilitate rollback
-    val currentPartitionReplicaAssignment = zkClient.getReplicaAssignmentForTopics(topics.toSet)
-    println("Current partition replica assignment\n\n%s\n\nSave this to use as the --reassignment-json-file option during rollback"
-      .format(formatAsReassignmentJson(currentPartitionReplicaAssignment, Map.empty)))
+  /**
+   * Verify that all the brokers in an assignment exist.
+   *
+   * @param adminClient                 The AdminClient to use.
+   * @param brokers                     The broker IDs to verify.
+   */
+  def verifyBrokerIds(adminClient: Admin, brokers: Set[Int]): Unit = {
+    val allNodeIds = adminClient.describeCluster().nodes().get().asScala.map(_.id).toSet
+    brokers.find(!allNodeIds.contains(_)).map {
+      id => throw new AdminCommandFailedException(s"Unknown broker id ${id}")
+    }
+  }
+
+  /**
+   * The entry point for the --execute command.
+   *
+   * @param zkClient                    The ZooKeeper client to use.
+   * @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.
+   */
+  def executeAssignment(zkClient: KafkaZkClient,
+                        reassignmentJson: String,
+                        interBrokerThrottle: Long): Unit = {
+    val (proposedParts, proposedReplicas) = parseExecuteAssignmentArgs(reassignmentJson)
+    if (proposedReplicas.nonEmpty) {
+      throw new AdminCommandFailedException("bootstrap-server needs to be provided when " +
+        "replica reassignments are present.")
+    }
+    verifyReplicasAndBrokersInAssignment(zkClient, proposedParts)
+
+    // Check for the presence of the legacy partition reassignment ZNode.  This actually
+    // won't detect all rebalances... only ones initiated by the legacy method.
+    // This is a limitation of the legacy ZK API.
+    val reassignPartitionsInProgress = zkClient.reassignPartitionsInProgress()
+    if (reassignPartitionsInProgress) {
+      // Note: older versions of this tool would modify the broker quotas here (but not
+      // topic quotas, for some reason).  This behavior wasn't documented in the --execute
+      // command line help.  Since it might interfere with other ongoing reassignments,
+      // this behavior was dropped as part of the KIP-455 changes.
+      throw new TerseReassignmentFailureException(cannotExecuteBecauseOfExistingMessage)
+    }
+    val currentParts = zkClient.getReplicaAssignmentForTopics(
+      proposedParts.map(_._1.topic()).toSet)
+    println(currentPartitionReplicaAssignmentToString(proposedParts, currentParts))
+
+    if (interBrokerThrottle >= 0) {
+      println(youMustRunVerifyPeriodicallyMessage)
+      val moveMap = calculateMoveMap(Map.empty, proposedParts, currentParts)
+      val leaderThrottles = calculateLeaderThrottles(moveMap)
+      val followerThrottles = calculateFollowerThrottles(moveMap)
+      modifyTopicThrottles(zkClient, leaderThrottles, followerThrottles)
+      val reassigningBrokers = calculateReassigningBrokers(moveMap)
+      modifyBrokerThrottles(zkClient, reassigningBrokers, interBrokerThrottle)
+      println(s"The inter-broker throttle limit was set to ${interBrokerThrottle} B/s")
+    }
+    zkClient.createPartitionReassignment(proposedParts)
+    println("Successfully started partition reassignment%s for %s".format(
+      if (proposedParts.size == 1) "" else "s",
+      proposedParts.keySet.toBuffer.sortWith(compareTopicPartitions).mkString(",")))
+  }
+
+  /**
+   * Return the string which we want to print to describe the current partition assignment.
+   *
+   * @param proposedParts               The proposed partition assignment.
+   * @param currentParts                The current partition assignment.
+   *
+   * @return                            The string to print.  We will only print information about
+   *                                    partitions that appear in the proposed partition assignment.
+   */
+  def currentPartitionReplicaAssignmentToString(proposedParts: Map[TopicPartition, Seq[Int]],
+                                                currentParts: Map[TopicPartition, Seq[Int]]): String = {
+    "Current partition replica assignment%n%n%s%n%nSave this to use as the %s".
+        format(formatAsReassignmentJson(currentParts.filterKeys(proposedParts.contains(_)).toMap, Map.empty),
+              "--reassignment-json-file option during rollback")
+  }
+
+  /**
+   * Verify that the replicas and brokers referenced in the given partition assignment actually
+   * exist.  This is necessary when using the deprecated ZK API, since ZooKeeper itself can't
+   * validate what we're applying.
+   *
+   * @param zkClient                    The ZooKeeper client to use.
+   * @param proposedParts               The partition assignment.
+   */
+  def verifyReplicasAndBrokersInAssignment(zkClient: KafkaZkClient,
+                                           proposedParts: Map[TopicPartition, Seq[Int]]): Unit = {
+    // check that all partitions in the proposed assignment exist in the cluster
+    val proposedTopics = proposedParts.map { case (tp, _) => tp.topic }
+    val existingAssignment = zkClient.getReplicaAssignmentForTopics(proposedTopics.toSet)
+    val nonExistentPartitions = proposedParts.map { case (tp, _) => tp }.filterNot(existingAssignment.contains)
+    if (nonExistentPartitions.nonEmpty)
+      throw new AdminCommandFailedException("The proposed assignment contains non-existent partitions: " +
+        nonExistentPartitions)
+
+    // check that all brokers in the proposed assignment exist in the cluster
+    val existingBrokerIDs = zkClient.getSortedBrokerList
+    val nonExistingBrokerIDs = proposedParts.toMap.values.flatten.filterNot(existingBrokerIDs.contains).toSet
+    if (nonExistingBrokerIDs.nonEmpty)
+      throw new AdminCommandFailedException("The proposed assignment contains non-existent brokerIDs: " + nonExistingBrokerIDs.mkString(","))
+  }
+
+  /**
+   * Execute the given partition reassignments.
+   *
+   * @param adminClient       The admin client object to use.
+   * @param reassignments     A map from topic names to target replica assignments.
+   * @return                  A map from partition objects to error strings.
+   */
+  def alterPartitionReassignments(adminClient: Admin,
+                                  reassignments: Map[TopicPartition, Seq[Int]])
+                                  : Map[TopicPartition, Throwable] = {
+    val results: Map[TopicPartition, KafkaFuture[Void]] =
+      adminClient.alterPartitionReassignments(reassignments.map {
+        case (part, replicas) => {
+          (part, Optional.of(new NewPartitionReassignment(replicas.map(Integer.valueOf(_)).asJava)))
+        }
+      }.asJava).values().asScala
+    results.flatMap {
+      case (part, future) => {
+        try {
+          future.get()
+          None
+        } catch {
+          case t: ExecutionException => Some(part, t.getCause())
+        }
+      }
+    }
+  }
+
+  /**
+   * Cancel the given partition reassignments.
+   *
+   * @param adminClient       The admin client object to use.
+   * @param reassignments     The partition reassignments to cancel.
+   * @return                  A map from partition objects to error strings.
+   */
+  def cancelPartitionReassignments(adminClient: Admin,
+                                  reassignments: Set[TopicPartition])
+  : Map[TopicPartition, Throwable] = {
+    val results: Map[TopicPartition, KafkaFuture[Void]] =
+      adminClient.alterPartitionReassignments(reassignments.map {
+          (_, (None: Option[NewPartitionReassignment]).asJava)
+        }.toMap.asJava).values().asScala
+    results.flatMap {
+      case (part, future) => {
+        try {
+          future.get()
+          None
+        } catch {
+          case t: ExecutionException => Some(part, t.getCause())
+        }
+      }
+    }
+  }
+
+  /**
+   * Calculate the global map of all partitions that are moving.
+   *
+   * @param currentReassignments    The currently active reassignments.
+   * @param proposedReassignments   The proposed reassignments (destinations replicas only).
+   * @param currentParts            The current location of the partitions that we are
+   *                                proposing to move.
+   * @return                        A map from topic name to partition map.
+   *                                The partition map is keyed on partition index and contains
+   *                                the movements for that partition.
+   */
+  def calculateMoveMap(currentReassignments: Map[TopicPartition, PartitionReassignment],
+                       proposedReassignments: Map[TopicPartition, Seq[Int]],
+                       currentParts: Map[TopicPartition, Seq[Int]]): MoveMap = {
+    val moveMap = new mutable.HashMap[String, mutable.Map[Int, PartitionMove]]()
+    // Add the current reassignments to the move map.
+    currentReassignments.foreach {
+      case (part, reassignment) => {
+        val move = PartitionMove(new mutable.HashSet[Int](), new mutable.HashSet[Int]())
+        reassignment.replicas().asScala.foreach {
+          replica => move.sources += replica
+            move.destinations += replica
+        }
+        reassignment.addingReplicas().asScala.foreach(move.destinations += _)
+        reassignment.removingReplicas().asScala.foreach(move.destinations -= _)
+        val partMoves = moveMap.getOrElseUpdate(part.topic(), new mutable.HashMap[Int, PartitionMove])
+        partMoves.put(part.partition(), move)
+      }
+    }
+    // Add the proposed reassignments to the move map.  The proposals will overwrite
+    // the current reassignments.
+    proposedReassignments.foreach {
+      case (part, replicas) => {
+        val move = PartitionMove(new mutable.HashSet[Int](), new mutable.HashSet[Int]())
+        move.destinations ++= replicas
+        val partMoves = moveMap.getOrElseUpdate(part.topic(), new mutable.HashMap[Int, PartitionMove])
+        partMoves.put(part.partition(), move)
+      }
+    }
+    // For partitions we are moving, add the current replica locations as sources.
+    // Ignore partitions that are not being moved.
+    moveMap.foreach {
+      case (topicName, partMap) =>
+        partMap.foreach {
+          case (partitionIndex, moves) =>
+            currentParts.get(new TopicPartition(topicName, partitionIndex)) match {
+              case None =>
+              case Some(replicas) => moves.sources ++= replicas
+            }
+        }
+    }
+    // Remove sources from destinations.  If something is a source, the data is already there,
+    // so it doesn't need to be treated as a destination (by having follower throttle applied, etc.)
+    moveMap.foreach {
+      case (_, partMap) =>
+        partMap.foreach {
+          case (_, moves) =>
+            moves.destinations --= moves.sources
+        }
+    }
+    moveMap
+  }
+
+  /**
+   * Calculate the leader throttle configurations to use.
+   *
+   * @param moveMap   The movements.
+   * @return          A map from topic names to leader throttle configurations.
+   */
+  def calculateLeaderThrottles(moveMap: MoveMap): Map[String, String] = {
+    moveMap.map {
+      case (topicName, partMoveMap) => {
+        val components = new mutable.TreeSet[String]
+        partMoveMap.foreach {
+          case (partId, move) =>
+            move.sources.foreach(source => components.add("%d:%d".format(partId, source)))
+        }
+        (topicName, components.mkString(","))
+      }
+    }
+  }
+
+  /**
+   * Calculate the follower throttle configurations to use.
+   *
+   * @param moveMap   The movements.
+   * @return          A map from topic names to follower throttle configurations.
+   */
+  def calculateFollowerThrottles(moveMap: MoveMap): Map[String, String] = {
+    moveMap.map {
+      case (topicName, partMoveMap) => {
+        val components = new mutable.TreeSet[String]
+        partMoveMap.foreach {
+          case (partId, move) =>
+            move.destinations.foreach(destination =>
+              if (!move.sources.contains(destination)) {
+                components.add("%d:%d".format(partId, destination))
+              })
+        }
+        (topicName, components.mkString(","))
+      }
+    }
+  }
+
+  /**
+   * Calculate all the brokers which are involved in the given partition reassignments.
+   *
+   * @param moveMap       The partition movements.
+   * @return              A set of all the brokers involved.
+   */
+  def calculateReassigningBrokers(moveMap: MoveMap): Set[Int] = {
+    val reassigningBrokers = new mutable.TreeSet[Int]
+    moveMap.values.foreach {
+      _.values.foreach {
+        partMove =>
+          partMove.sources.foreach(reassigningBrokers.add(_))
+          partMove.destinations.foreach(reassigningBrokers.add(_))
+      }
+    }
+    reassigningBrokers.toSet
+  }
+
+  /**
+   * Calculate all the brokers which are involved in the given directory movements.
+   *
+   * @param replicaMoves  The replica movements.
+   * @return              A set of all the brokers involved.
+   */
+  def calculateMovingBrokers(replicaMoves: Set[TopicPartitionReplica]): Set[Int] = {
+    replicaMoves.map(_.brokerId()).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.
+   */
+  def modifyTopicThrottles(adminClient: Admin,
+                           leaderThrottles: Map[String, String],
+                           followerThrottles: Map[String, String]): Unit = {
+    val configs = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]()
+    val topicNames = leaderThrottles.keySet ++ followerThrottles.keySet
+    topicNames.foreach {
+      topicName =>
+        val ops = new util.ArrayList[AlterConfigOp]
+        leaderThrottles.get(topicName) match {
+          case None =>
+          case Some(value) => ops.add(new AlterConfigOp(new ConfigEntry(topicLevelLeaderThrottle,
+            value), OpType.SET))
+        }
+        followerThrottles.get(topicName) match {
+          case None =>
+          case Some(value) => ops.add(new AlterConfigOp(new ConfigEntry(topicLevelFollowerThrottle,
+            value), OpType.SET))
+        }
+        if (!ops.isEmpty) {
+          configs.put(new ConfigResource(ConfigResource.Type.TOPIC, topicName), ops)
+        }
+    }
+    adminClient.incrementalAlterConfigs(configs).all().get()
+  }
+
+  /**
+   * Modify the topic configurations that control inter-broker throttling.
+   *
+   * @param zkClient            The ZooKeeper client to use.
+   * @param leaderThrottles     A map from topic names to leader throttle configurations.
+   * @param followerThrottles   A map from topic names to follower throttle configurations.
+   */
+  def modifyTopicThrottles(zkClient: KafkaZkClient,
+                           leaderThrottles: Map[String, String],
+                           followerThrottles: Map[String, String]): Unit = {
+    val adminZkClient = new AdminZkClient(zkClient)
+    val topicNames = leaderThrottles.keySet ++ followerThrottles.keySet
+    topicNames.foreach {
+      topicName =>
+        val configs = adminZkClient.fetchEntityConfig(ConfigType.Topic, topicName)
+        leaderThrottles.get(topicName).map(configs.put(topicLevelLeaderThrottle, _))
+        followerThrottles.get(topicName).map(configs.put(topicLevelFollowerThrottle, _))
+        adminZkClient.changeTopicConfig(topicName, configs)
+    }
+  }
+
+  /**
+   * Modify the broker-level configurations for leader and follower throttling.
+   *
+   * @param adminClient                   The adminClient object to use.
+   * @param reassigningBrokers            The brokers that are involved in reassignments.
+   * @param interBrokerThrottle           The inter-broker throttle value to set, or a
+   *                                      negative number if none should be set.
+   * @param movingBrokers                 The brokers that are involved in movements.
+   * @param logDirThrottle                The replica log dir throttle value to set, or a
+   *                                      negative number if none should be set.
+   */
+  def modifyBrokerThrottles(adminClient: Admin,
+                            reassigningBrokers: Set[Int],
+                            interBrokerThrottle: Long,
+                            movingBrokers: Set[Int],
+                            logDirThrottle: Long): Unit = {
+    val configs = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]()
+    (reassigningBrokers ++ movingBrokers).foreach {
+      brokerId =>
+        val ops = new util.ArrayList[AlterConfigOp]
+        if (interBrokerThrottle >= 0 && reassigningBrokers.contains(brokerId)) {
+          ops.add(new AlterConfigOp(new ConfigEntry(brokerLevelLeaderThrottle,
+            interBrokerThrottle.toString), OpType.SET))
+          ops.add(new AlterConfigOp(new ConfigEntry(brokerLevelFollowerThrottle,
+            interBrokerThrottle.toString), OpType.SET))
+        }
+        if (logDirThrottle >= 0 && movingBrokers.contains(brokerId)) {
+          ops.add(new AlterConfigOp(new ConfigEntry(brokerLevelLogDirThrottle,
+            logDirThrottle.toString), OpType.SET))
+        }
+        if (!ops.isEmpty) {
+          configs.put(new ConfigResource(ConfigResource.Type.BROKER, brokerId.toString), ops)
+        }
+    }
+    adminClient.incrementalAlterConfigs(configs).all().get()
+  }
+
+  /**
+   * Modify the broker-level configurations for leader and follower throttling.
+   *
+   * @param zkClient            The ZooKeeper client to use.
+   * @param reassigningBrokers  The brokers to reconfigure.
+   * @param interBrokerThrottle The throttle value to set.
+   */
+  def modifyBrokerThrottles(zkClient: KafkaZkClient,
+                            reassigningBrokers: Set[Int],
+                            interBrokerThrottle: Long): Unit = {
+    val adminZkClient = new AdminZkClient(zkClient)
+    for (id <- reassigningBrokers) {
+      val configs = adminZkClient.fetchEntityConfig(ConfigType.Broker, id.toString)
+      configs.put(brokerLevelLeaderThrottle, interBrokerThrottle.toString)
+      configs.put(brokerLevelFollowerThrottle, interBrokerThrottle.toString)
+      adminZkClient.changeBrokerConfig(Seq(id), configs)
+    }
+  }
+
+  /**
+   * Parse the reassignment JSON string passed to the --execute command.
+   *
+   * @param reassignmentJson  The JSON string.
+   * @return                  A tuple of the partitions to be reassigned and the replicas
+   *                          to be reassigned.
+   */
+  def parseExecuteAssignmentArgs(reassignmentJson: String)
+      : (Map[TopicPartition, Seq[Int]], Map[TopicPartitionReplica, String]) = {
+    val (partitionsToBeReassigned, replicaAssignment) = parsePartitionReassignmentData(reassignmentJson)
+    if (partitionsToBeReassigned.isEmpty)
+      throw new AdminCommandFailedException("Partition reassignment list cannot be empty")
+    if (partitionsToBeReassigned.exists(_._2.isEmpty)) {
+      throw new AdminCommandFailedException("Partition replica list cannot be empty")
+    }
+    val duplicateReassignedPartitions = CoreUtils.duplicates(partitionsToBeReassigned.map { case (tp, _) => tp })
+    if (duplicateReassignedPartitions.nonEmpty)
+      throw new AdminCommandFailedException("Partition reassignment contains duplicate topic partitions: %s".format(duplicateReassignedPartitions.mkString(",")))
+    val duplicateEntries = partitionsToBeReassigned
+      .map { case (tp, replicas) => (tp, CoreUtils.duplicates(replicas))}
+      .filter { case (_, duplicatedReplicas) => duplicatedReplicas.nonEmpty }
+    if (duplicateEntries.nonEmpty) {
+      val duplicatesMsg = duplicateEntries
+        .map { case (tp, duplicateReplicas) => "%s contains multiple entries for %s".format(tp, duplicateReplicas.mkString(",")) }
+        .mkString(". ")
+      throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: %s".format(duplicatesMsg))
+    }
+    (partitionsToBeReassigned.toMap, replicaAssignment)
+  }
+
+  /**
+   * The entry point for the --cancel command.
+   *
+   * @param adminClient           The AdminClient to use.
+   * @param jsonString            The JSON string to use for the topics and partitions to cancel.
+   * @param preserveThrottles     True if we should avoid changing topic or broker throttles.
+   * @param timeoutMs             The maximum time in ms to wait for log directory
+   *                              replica assignment to begin.
+   * @param time                  The Time object to use.
+   *
+   * @return                      A tuple of the partition reassignments that were cancelled,
+   *                              and the replica movements that were cancelled.
+   */
+  def cancelAssignment(adminClient: Admin,
+                       jsonString: String,
+                       preserveThrottles: Boolean,
+                       timeoutMs: Long = 10000L,
+                       time: Time = Time.SYSTEM)
+                       : (Set[TopicPartition], Set[TopicPartitionReplica]) = {
+    val (targetParts, targetReplicas) = parsePartitionReassignmentData(jsonString)
+    val targetPartsSet = targetParts.map(_._1).toSet
+    val curReassigningParts = adminClient.listPartitionReassignments(targetPartsSet.asJava).
+        reassignments().get().asScala.flatMap {
+      case (part, reassignment) => if (!reassignment.addingReplicas().isEmpty ||
+          !reassignment.removingReplicas().isEmpty) {
+        Some(part)
+      } else {
+        None
+      }
+    }.toSet
+    if (curReassigningParts.nonEmpty) {
+      val errors = cancelPartitionReassignments(adminClient, curReassigningParts)
+      if (errors.nonEmpty) {
+        throw new TerseReassignmentFailureException(
+          "Error cancelling partition reassignment%s for:%n%s".format(
+            if (errors.size == 1) "" else "s",
+            errors.keySet.toBuffer.sortWith(compareTopicPartitions).map {
+              part => s"${part}: ${errors(part).getMessage}"
+            }.mkString(System.lineSeparator())))
+      }
+      println("Successfully cancelled partition reassignment%s for: %s".format(
+        if (curReassigningParts.size == 1) "" else "s",
+        s"${curReassigningParts.toBuffer.sortWith(compareTopicPartitions).mkString(",")}"))
+    } else {
+      println("None of the specified partition reassignments are active.")
+    }
+    val curMovingParts = findLogDirMoveStates(adminClient, targetReplicas).flatMap {
+      case (part, moveState) => moveState match {
+        case state: ActiveMoveState => Some(part, state.currentLogDir)
+        case _ => None
+      }
+    }.toMap
+    if (curMovingParts.isEmpty) {
+      println("None of the specified partition moves are active.")
+    } else {
+      executeMoves(adminClient, curMovingParts, timeoutMs, time)
+    }
+    if (!preserveThrottles) {
+      clearAllThrottles(adminClient, targetParts)
+    }
+    (curReassigningParts, curMovingParts.keySet)
   }
 
   def formatAsReassignmentJson(partitionsToBeReassigned: Map[TopicPartition, Seq[Int]],
                                replicaLogDirAssignment: Map[TopicPartitionReplica, String]): String = {
     Json.encodeAsString(Map(
       "version" -> 1,
-      "partitions" -> partitionsToBeReassigned.map { case (tp, replicas) =>
-        Map(
-          "topic" -> tp.topic,
-          "partition" -> tp.partition,
-          "replicas" -> replicas.asJava,
-          "log_dirs" -> replicas.map(r => replicaLogDirAssignment.getOrElse(new TopicPartitionReplica(tp.topic, tp.partition, r), AnyLogDir)).asJava
-        ).asJava
+      "partitions" -> partitionsToBeReassigned.keySet.toBuffer.sortWith(compareTopicPartitions).map {
+        tp =>
+          val replicas = partitionsToBeReassigned(tp)
+          Map(
+            "topic" -> tp.topic,
+            "partition" -> tp.partition,
+            "replicas" -> replicas.asJava,
+            "log_dirs" -> replicas.map(r => replicaLogDirAssignment.getOrElse(new TopicPartitionReplica(tp.topic, tp.partition, r), AnyLogDir)).asJava
+          ).asJava
       }.asJava
     ).asJava)
   }
@@ -252,7 +1594,7 @@ object ReassignPartitionsCommand extends Logging {
       case Some(js) =>
         val version = js.asJsonObject.get("version") match {
           case Some(jsonValue) => jsonValue.to[Int]
-          case None => EarliestVersion
+          case None => EarliestTopicsJsonVersion
         }
         parseTopicsData(version, js)
       case None => throw new AdminOperationException("The input string is not a valid JSON")
@@ -313,139 +1655,146 @@ object ReassignPartitionsCommand extends Logging {
     }
   }
 
-  def parseAndValidate(zkClient: KafkaZkClient, reassignmentJsonString: String): (Seq[(TopicPartition, Seq[Int])], Map[TopicPartitionReplica, String]) = {
-    val (partitionsToBeReassigned, replicaAssignment) = parsePartitionReassignmentData(reassignmentJsonString)
-
-    if (partitionsToBeReassigned.isEmpty)
-      throw new AdminCommandFailedException("Partition reassignment data file is empty")
-    if (partitionsToBeReassigned.exists(_._2.isEmpty)) {
-      throw new AdminCommandFailedException("Partition replica list cannot be empty")
-    }
-    val duplicateReassignedPartitions = CoreUtils.duplicates(partitionsToBeReassigned.map { case (tp, _) => tp })
-    if (duplicateReassignedPartitions.nonEmpty)
-      throw new AdminCommandFailedException("Partition reassignment contains duplicate topic partitions: %s".format(duplicateReassignedPartitions.mkString(",")))
-    val duplicateEntries = partitionsToBeReassigned
-      .map { case (tp, replicas) => (tp, CoreUtils.duplicates(replicas))}
-      .filter { case (_, duplicatedReplicas) => duplicatedReplicas.nonEmpty }
-    if (duplicateEntries.nonEmpty) {
-      val duplicatesMsg = duplicateEntries
-        .map { case (tp, duplicateReplicas) => "%s contains multiple entries for %s".format(tp, duplicateReplicas.mkString(",")) }
-        .mkString(". ")
-      throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: %s".format(duplicatesMsg))
-    }
-    // check that all partitions in the proposed assignment exist in the cluster
-    val proposedTopics = partitionsToBeReassigned.map { case (tp, _) => tp.topic }.distinct
-    val existingAssignment = zkClient.getReplicaAssignmentForTopics(proposedTopics.toSet)
-    val nonExistentPartitions = partitionsToBeReassigned.map { case (tp, _) => tp }.filterNot(existingAssignment.contains)
-    if (nonExistentPartitions.nonEmpty)
-      throw new AdminCommandFailedException("The proposed assignment contains non-existent partitions: " +
-        nonExistentPartitions)
-
-    // check that all brokers in the proposed assignment exist in the cluster
-    val existingBrokerIDs = zkClient.getSortedBrokerList
-    val nonExistingBrokerIDs = partitionsToBeReassigned.toMap.values.flatten.filterNot(existingBrokerIDs.contains).toSet
-    if (nonExistingBrokerIDs.nonEmpty)
-      throw new AdminCommandFailedException("The proposed assignment contains non-existent brokerIDs: " + nonExistingBrokerIDs.mkString(","))
-
-    (partitionsToBeReassigned, replicaAssignment)
-  }
-
-  def checkIfPartitionReassignmentSucceeded(zkClient: KafkaZkClient, partitionsToBeReassigned: Map[TopicPartition, Seq[Int]])
-  :Map[TopicPartition, ReassignmentStatus] = {
-    val partitionsBeingReassigned = zkClient.getPartitionReassignment
-    val (beingReassigned, notBeingReassigned) = partitionsToBeReassigned.keys.partition { topicAndPartition =>
-      partitionsBeingReassigned.contains(topicAndPartition)
-    }
-    notBeingReassigned.groupBy(_.topic).flatMap { case (topic, partitions) =>
-      val replicasForTopic = zkClient.getReplicaAssignmentForTopics(immutable.Set(topic))
-      partitions.map { topicAndPartition =>
-        val newReplicas = partitionsToBeReassigned(topicAndPartition)
-        val reassignmentStatus = replicasForTopic.get(topicAndPartition) match {
-          case Some(seq) if seq == newReplicas => ReassignmentCompleted
-          case _ => ReassignmentFailed
-        }
-        (topicAndPartition, reassignmentStatus)
-      }
-    } ++ beingReassigned.map { topicAndPartition =>
-      (topicAndPartition, ReassignmentInProgress)
-    }.toMap
-  }
-
-  private def checkIfReplicaReassignmentSucceeded(adminClientOpt: Option[Admin], replicaAssignment: Map[TopicPartitionReplica, String])
-  :Map[TopicPartitionReplica, ReassignmentStatus] = {
-
-    val replicaLogDirInfos = {
-      if (replicaAssignment.nonEmpty) {
-        val adminClient = adminClientOpt.getOrElse(
-          throw new AdminCommandFailedException("bootstrap-server needs to be provided in order to reassign replica to the specified log directory"))
-        adminClient.describeReplicaLogDirs(replicaAssignment.keySet.asJava).all().get().asScala
-      } else {
-        Map.empty[TopicPartitionReplica, ReplicaLogDirInfo]
-      }
-    }
-
-    replicaAssignment.map { case (replica, newLogDir) =>
-      val status: ReassignmentStatus = replicaLogDirInfos.get(replica) match {
-        case Some(replicaLogDirInfo) =>
-          if (replicaLogDirInfo.getCurrentReplicaLogDir == null) {
-            println(s"Partition ${replica.topic()}-${replica.partition()} is not found in any live log dir on " +
-              s"broker ${replica.brokerId()}. There is likely offline log directory on the broker.")
-            ReassignmentFailed
-          } else if (replicaLogDirInfo.getFutureReplicaLogDir == newLogDir) {
-            ReassignmentInProgress
-          } else if (replicaLogDirInfo.getFutureReplicaLogDir != null) {
-            println(s"Partition ${replica.topic()}-${replica.partition()} on broker ${replica.brokerId()} " +
-              s"is being moved to log dir ${replicaLogDirInfo.getFutureReplicaLogDir} instead of $newLogDir")
-            ReassignmentFailed
-          } else if (replicaLogDirInfo.getCurrentReplicaLogDir == newLogDir) {
-            ReassignmentCompleted
-          } else {
-            println(s"Partition ${replica.topic()}-${replica.partition()} on broker ${replica.brokerId()} " +
-              s"is not being moved from log dir ${replicaLogDirInfo.getCurrentReplicaLogDir} to $newLogDir")
-            ReassignmentFailed
-          }
-        case None =>
-          println(s"Partition ${replica.topic()}-${replica.partition()} is not found in any live log dir on broker ${replica.brokerId()}.")
-          ReassignmentFailed
-      }
-      (replica, status)
-    }
-  }
-
   def validateAndParseArgs(args: Array[String]): ReassignPartitionsCommandOptions = {
     val opts = new ReassignPartitionsCommandOptions(args)
 
     CommandLineUtils.printHelpAndExitIfNeeded(opts, helpText)
 
-    // Should have exactly one action
-    val actions = Seq(opts.generateOpt, opts.executeOpt, opts.verifyOpt).count(opts.options.has _)
-    if(actions != 1)
-      CommandLineUtils.printUsageAndDie(opts.parser, "Command must include exactly one action: --generate, --execute or --verify")
-
-    CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.zkConnectOpt)
-
-    //Validate arguments for each action
-    if(opts.options.has(opts.verifyOpt)) {
-      if(!opts.options.has(opts.reassignmentJsonFileOpt))
-        CommandLineUtils.printUsageAndDie(opts.parser, "If --verify option is used, command must include --reassignment-json-file that was used during the --execute option")
-      CommandLineUtils.checkInvalidArgs(opts.parser, opts.options, opts.verifyOpt, Set(opts.interBrokerThrottleOpt, opts.replicaAlterLogDirsThrottleOpt, opts.topicsToMoveJsonFileOpt, opts.disableRackAware, opts.brokerListOpt))
-    }
-    else if(opts.options.has(opts.generateOpt)) {
-      if(!(opts.options.has(opts.topicsToMoveJsonFileOpt) && opts.options.has(opts.brokerListOpt)))
-        CommandLineUtils.printUsageAndDie(opts.parser, "If --generate option is used, command must include both --topics-to-move-json-file and --broker-list options")
-      CommandLineUtils.checkInvalidArgs(opts.parser, opts.options, opts.generateOpt, Set(opts.interBrokerThrottleOpt, opts.replicaAlterLogDirsThrottleOpt, opts.reassignmentJsonFileOpt))
+    // Determine which action we should perform.
+    val validActions = Seq(opts.generateOpt, opts.executeOpt, opts.verifyOpt,
+                           opts.cancelOpt, opts.listOpt)
+    val allActions = validActions.filter(opts.options.has _)
+    if (allActions.size != 1) {
+      CommandLineUtils.printUsageAndDie(opts.parser, "Command must include exactly one action: %s".format(
+        validActions.map("--" + _.options().get(0)).mkString(", ")))
     }
-    else if (opts.options.has(opts.executeOpt)){
-      if(!opts.options.has(opts.reassignmentJsonFileOpt))
-        CommandLineUtils.printUsageAndDie(opts.parser, "If --execute option is used, command must include --reassignment-json-file that was output " + "during the --generate option")
-      CommandLineUtils.checkInvalidArgs(opts.parser, opts.options, opts.executeOpt, Set(opts.topicsToMoveJsonFileOpt, opts.disableRackAware, opts.brokerListOpt))
+    val action = allActions(0)
+
+    // Check that we have either the --zookeeper option or the --bootstrap-server set.
+    // It would be nice to enforce that we can only have one of these options set at once.  Unfortunately,
+    // previous versions of this tool supported setting both options together.  To avoid breaking backwards
+    // compatibility, we will follow suit, for now.  This issue will eventually be resolved when we remove
+    // the --zookeeper option.
+    if (!opts.options.has(opts.zkConnectOpt) && !opts.options.has(opts.bootstrapServerOpt))
+      CommandLineUtils.printUsageAndDie(opts.parser, "Please specify --bootstrap-server")
+
+    // Make sure that we have all the required arguments for our action.
+    val requiredArgs = Map(
+      opts.verifyOpt -> collection.immutable.Seq(
+        opts.reassignmentJsonFileOpt
+      ),
+      opts.generateOpt -> collection.immutable.Seq(
+        opts.topicsToMoveJsonFileOpt,
+        opts.brokerListOpt
+      ),
+      opts.executeOpt -> collection.immutable.Seq(
+        opts.reassignmentJsonFileOpt
+      ),
+      opts.cancelOpt -> collection.immutable.Seq(
+        opts.reassignmentJsonFileOpt
+      ),
+      opts.listOpt -> collection.immutable.Seq(
+      )
+    )
+    CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, requiredArgs.get(action).get: _*)
+
+    // Make sure that we didn't specify any arguments that are incompatible with our chosen action.
+    val permittedArgs = Map(
+      opts.verifyOpt -> Seq(
+        opts.bootstrapServerOpt,
+        opts.commandConfigOpt,
+        opts.preserveThrottlesOpt,
+        opts.zkConnectOpt
+      ),
+      opts.generateOpt -> Seq(
+        opts.bootstrapServerOpt,
+        opts.brokerListOpt,
+        opts.commandConfigOpt,
+        opts.disableRackAware,
+        opts.zkConnectOpt
+      ),
+      opts.executeOpt -> Seq(
+        opts.additionalOpt,
+        opts.bootstrapServerOpt,
+        opts.commandConfigOpt,
+        opts.interBrokerThrottleOpt,
+        opts.replicaAlterLogDirsThrottleOpt,
+        opts.timeoutOpt,
+        opts.zkConnectOpt
+      ),
+      opts.cancelOpt -> Seq(
+        opts.bootstrapServerOpt,
+        opts.commandConfigOpt,
+        opts.preserveThrottlesOpt,
+        opts.timeoutOpt
+      ),
+      opts.listOpt -> Seq(
+        opts.bootstrapServerOpt,
+        opts.commandConfigOpt
+      )
+    )
+    opts.options.specs().asScala.foreach(opt => {
+      if (!opt.equals(action) &&
+          !requiredArgs(action).contains(opt) &&
+          !permittedArgs(action).contains(opt)) {
+        CommandLineUtils.printUsageAndDie(opts.parser,
+          """Option "%s" can't be used with action "%s"""".format(opt, action))
+      }
+    })
+    if (!opts.options.has(opts.bootstrapServerOpt)) {
+      val bootstrapServerOnlyArgs = Seq(
+        opts.additionalOpt,
+        opts.cancelOpt,
+        opts.commandConfigOpt,
+        opts.replicaAlterLogDirsThrottleOpt,
+        opts.listOpt,
+        opts.timeoutOpt
+      )
+      bootstrapServerOnlyArgs.foreach {
+        opt => if (opts.options.has(opt)) {
+          throw new RuntimeException("You must specify --bootstrap-server " +
+            """when using "%s"""".format(opt))
+        }
+      }
     }
     opts
   }
 
-  class ReassignPartitionsCommandOptions(args: Array[String]) extends CommandDefaultOptions(args)  {
+  def alterReplicaLogDirs(adminClient: Admin,
+                          assignment: Map[TopicPartitionReplica, String])
+                          : Set[TopicPartitionReplica] = {
+    adminClient.alterReplicaLogDirs(assignment.asJava).values().asScala.flatMap {
+      case (replica, future) => {
+        try {
+          future.get()
+          Some(replica)
+        } catch {
+          case t: ExecutionException =>
+            t.getCause match {
+              // Ignore ReplicaNotAvailableException.  It is OK if the replica is not
+              // available at this moment.
+              case _: ReplicaNotAvailableException => None
+              case e: Throwable =>
+                throw new AdminCommandFailedException(s"Failed to alter dir for $replica", e)
+            }
+        }
+      }
+    }.toSet
+  }
+
+  sealed class ReassignPartitionsCommandOptions(args: Array[String]) extends CommandDefaultOptions(args)  {
+    // Actions
+    val verifyOpt = parser.accepts("verify", "Verify if the reassignment completed as specified by the --reassignment-json-file option. If there is a throttle engaged for the replicas specified, and the rebalance has completed, the throttle will be removed")
+    val generateOpt = parser.accepts("generate", "Generate a candidate partition reassignment configuration." +
+      " Note that this only generates a candidate assignment, it does not execute it.")
+    val executeOpt = parser.accepts("execute", "Kick off the reassignment as specified by the --reassignment-json-file option.")
+    val cancelOpt = parser.accepts("cancel", "Cancel an active reassignment.")
+    val listOpt = parser.accepts("list", "List all active partition reassignments.")
+
+    // Arguments
     val bootstrapServerOpt = parser.accepts("bootstrap-server", "the server(s) to use for bootstrapping. REQUIRED if " +
-                      "an absolute path of the log directory is specified for any replica in the reassignment json file")
+                      "an absolute path of the log directory is specified for any replica in the reassignment json file, " +
+                      "or if --zookeeper is not given.")
                       .withRequiredArg
                       .describedAs("Server(s) to use for bootstrapping")
                       .ofType(classOf[String])
@@ -453,15 +1802,11 @@ object ReassignPartitionsCommand extends Logging {
                       .withRequiredArg
                       .describedAs("Admin client property file")
                       .ofType(classOf[String])
-    val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the " +
-                      "form host:port. Multiple URLS can be given to allow fail-over.")
+    val zkConnectOpt = parser.accepts("zookeeper", "DEPRECATED: The connection string for the zookeeper connection in the " +
+                      "form host:port. Multiple URLS can be given to allow fail-over.  Please use --bootstrap-server instead.")
                       .withRequiredArg
                       .describedAs("urls")
                       .ofType(classOf[String])
-    val generateOpt = parser.accepts("generate", "Generate a candidate partition reassignment configuration." +
-                      " Note that this only generates a candidate assignment, it does not execute it.")
-    val executeOpt = parser.accepts("execute", "Kick off the reassignment as specified by the --reassignment-json-file option.")
-    val verifyOpt = parser.accepts("verify", "Verify if the reassignment completed as specified by the --reassignment-json-file option. If there is a throttle engaged for the replicas specified, and the rebalance has completed, the throttle will be removed")
     val reassignmentJsonFileOpt = parser.accepts("reassignment-json-file", "The JSON file with the partition reassignment configuration" +
                       "The format to use is - \n" +
                       "{\"partitions\":\n\t[{\"topic\": \"foo\",\n\t  \"partition\": 1,\n\t  \"replicas\": [1,2,3],\n\t  \"log_dirs\": [\"dir1\",\"dir2\",\"dir3\"] }],\n\"version\":1\n}\n" +
@@ -487,198 +1832,17 @@ object ReassignPartitionsCommand extends Logging {
                       .describedAs("throttle")
                       .ofType(classOf[Long])
                       .defaultsTo(-1)
-    val replicaAlterLogDirsThrottleOpt = parser.accepts("replica-alter-log-dirs-throttle", "The movement of replicas between log directories on the same broker will be throttled to this value (bytes/sec). Rerunning with this option, whilst a rebalance is in progress, will alter the throttle value. The throttle rate should be at least 1 KB/s.")
-                      .withRequiredArg()
+    val replicaAlterLogDirsThrottleOpt = parser.accepts("replica-alter-log-dirs-throttle", "The movement of replicas between log directories on the same broker will be throttled to this value (bytes/sec). Rerunning with this option, whilst a rebalance is in progress, will alter the throttle value. The throttle rate should be at least 1 KB/s.") .withRequiredArg()
                       .describedAs("replicaAlterLogDirsThrottle")
                       .ofType(classOf[Long])
                       .defaultsTo(-1)
-    val timeoutOpt = parser.accepts("timeout", "The maximum time in ms allowed to wait for partition reassignment execution to be successfully initiated")
+    val timeoutOpt = parser.accepts("timeout", "The maximum time in ms to wait for log directory replica assignment to begin.")
                       .withRequiredArg()
                       .describedAs("timeout")
                       .ofType(classOf[Long])
                       .defaultsTo(10000)
+    val additionalOpt = parser.accepts("additional", "Execute this reassignment in addition to any other ongoing ones.")
+    val preserveThrottlesOpt = parser.accepts("preserve-throttles", "Do not modify broker or topic throttles.")
     options = parser.parse(args : _*)
   }
 }
-
-class ReassignPartitionsCommand(zkClient: KafkaZkClient,
-                                adminClientOpt: Option[Admin],
-                                proposedPartitionAssignment: Map[TopicPartition, Seq[Int]],
-                                proposedReplicaAssignment: Map[TopicPartitionReplica, String] = Map.empty,
-                                adminZkClient: AdminZkClient)
-  extends Logging {
-
-  import ReassignPartitionsCommand._
-
-  def existingAssignment(): Map[TopicPartition, Seq[Int]] = {
-    val proposedTopics = proposedPartitionAssignment.keySet.map(_.topic).toSeq
-    zkClient.getReplicaAssignmentForTopics(proposedTopics.toSet)
-  }
-
-  private def maybeThrottle(throttle: Throttle): Unit = {
-    if (throttle.interBrokerLimit >= 0)
-      assignThrottledReplicas(existingAssignment(), proposedPartitionAssignment, adminZkClient)
-    maybeLimit(throttle)
-    if (throttle.interBrokerLimit >= 0 || throttle.replicaAlterLogDirsLimit >= 0)
-      throttle.postUpdateAction()
-    if (throttle.interBrokerLimit >= 0)
-      println(s"The inter-broker throttle limit was set to ${throttle.interBrokerLimit} B/s")
-    if (throttle.replicaAlterLogDirsLimit >= 0)
-      println(s"The replica-alter-dir throttle limit was set to ${throttle.replicaAlterLogDirsLimit} B/s")
-  }
-
-  /**
-    * Limit the throttle on currently moving replicas. Note that this command can use used to alter the throttle, but
-    * it may not alter all limits originally set, if some of the brokers have completed their rebalance.
-    */
-  def maybeLimit(throttle: Throttle): Unit = {
-    if (throttle.interBrokerLimit >= 0 || throttle.replicaAlterLogDirsLimit >= 0) {
-      val existingBrokers = existingAssignment().values.flatten.toSeq
-      val proposedBrokers = proposedPartitionAssignment.values.flatten.toSeq ++ proposedReplicaAssignment.keys.toSeq.map(_.brokerId())
-      val brokers = (existingBrokers ++ proposedBrokers).distinct
-
-      for (id <- brokers) {
-        val configs = adminZkClient.fetchEntityConfig(ConfigType.Broker, id.toString)
-        if (throttle.interBrokerLimit >= 0) {
-          configs.put(DynamicConfig.Broker.LeaderReplicationThrottledRateProp, throttle.interBrokerLimit.toString)
-          configs.put(DynamicConfig.Broker.FollowerReplicationThrottledRateProp, throttle.interBrokerLimit.toString)
-        }
-        if (throttle.replicaAlterLogDirsLimit >= 0)
-          configs.put(DynamicConfig.Broker.ReplicaAlterLogDirsIoMaxBytesPerSecondProp, throttle.replicaAlterLogDirsLimit.toString)
-
-        adminZkClient.changeBrokerConfig(Seq(id), configs)
-      }
-    }
-  }
-
-  /** Set throttles to replicas that are moving. Note: this method should only be used when the assignment is initiated. */
-  private[admin] def assignThrottledReplicas(existingPartitionAssignment: Map[TopicPartition, Seq[Int]],
-                                             proposedPartitionAssignment: Map[TopicPartition, Seq[Int]],
-                                             adminZkClient: AdminZkClient): Unit = {
-    for (topic <- proposedPartitionAssignment.keySet.map(_.topic).toSeq.distinct) {
-      val existingPartitionAssignmentForTopic = existingPartitionAssignment.filter { case (tp, _) => tp.topic == topic }
-      val proposedPartitionAssignmentForTopic = proposedPartitionAssignment.filter { case (tp, _) => tp.topic == topic }
-
-      //Apply leader throttle to all replicas that exist before the re-balance.
-      val leader = format(preRebalanceReplicaForMovingPartitions(existingPartitionAssignmentForTopic, proposedPartitionAssignmentForTopic))
-
-      //Apply follower throttle to all "move destinations".
-      val follower = format(postRebalanceReplicasThatMoved(existingPartitionAssignmentForTopic, proposedPartitionAssignmentForTopic))
-
-      val configs = adminZkClient.fetchEntityConfig(ConfigType.Topic, topic)
-      configs.put(LeaderReplicationThrottledReplicasProp, leader)
-      configs.put(FollowerReplicationThrottledReplicasProp, follower)
-      adminZkClient.changeTopicConfig(topic, configs)
-
-      debug(s"Updated leader-throttled replicas for topic $topic with: $leader")
-      debug(s"Updated follower-throttled replicas for topic $topic with: $follower")
-    }
-  }
-
-  private def postRebalanceReplicasThatMoved(existing: Map[TopicPartition, Seq[Int]], proposed: Map[TopicPartition, Seq[Int]]): Map[TopicPartition, Seq[Int]] = {
-    //For each partition in the proposed list, filter out any replicas that exist now, and hence aren't being moved.
-    proposed.map { case (tp, proposedReplicas) =>
-      tp -> (proposedReplicas.toSet -- existing(tp)).toSeq
-    }
-  }
-
-  private def preRebalanceReplicaForMovingPartitions(existing: Map[TopicPartition, Seq[Int]], proposed: Map[TopicPartition, Seq[Int]]): Map[TopicPartition, Seq[Int]] = {
-    def moving(before: Seq[Int], after: Seq[Int]) = (after.toSet -- before.toSet).nonEmpty
-    //For any moving partition, throttle all the original (pre move) replicas (as any one might be a leader)
-    existing.filter { case (tp, preMoveReplicas) =>
-      proposed.contains(tp) && moving(preMoveReplicas, proposed(tp))
-    }
-  }
-
-  def format(moves: Map[TopicPartition, Seq[Int]]): String =
-    moves.flatMap { case (tp, moves) =>
-      moves.map(replicaId => s"${tp.partition}:${replicaId}")
-    }.mkString(",")
-
-  private def alterReplicaLogDirsIgnoreReplicaNotAvailable(replicaAssignment: Map[TopicPartitionReplica, String],
-                                                           adminClient: Admin,
-                                                           timeoutMs: Long): Set[TopicPartitionReplica] = {
-    val alterReplicaLogDirsResult = adminClient.alterReplicaLogDirs(replicaAssignment.asJava, new AlterReplicaLogDirsOptions().timeoutMs(timeoutMs.toInt))
-    val replicasAssignedToFutureDir = alterReplicaLogDirsResult.values().asScala.flatMap { case (replica, future) => {
-      try {
-        future.get()
-        Some(replica)
-      } catch {
-        case t: ExecutionException =>
-          t.getCause match {
-            case _: ReplicaNotAvailableException => None // It is OK if the replica is not available at this moment
-            case e: Throwable => throw new AdminCommandFailedException(s"Failed to alter dir for $replica", e)
-          }
-      }
-    }}
-    replicasAssignedToFutureDir.toSet
-  }
-
-  def reassignPartitions(throttle: Throttle = NoThrottle, timeoutMs: Long = 10000L): Boolean = {
-    maybeThrottle(throttle)
-    try {
-      val validPartitions = proposedPartitionAssignment.groupBy(_._1.topic())
-        .flatMap { case (topic, topicPartitionReplicas) =>
-          validatePartition(zkClient, topic, topicPartitionReplicas)
-        }
-      if (validPartitions.isEmpty) false
-      else {
-        if (proposedReplicaAssignment.nonEmpty && adminClientOpt.isEmpty)
-          throw new AdminCommandFailedException("bootstrap-server needs to be provided in order to reassign replica to the specified log directory")
-        val startTimeMs = System.currentTimeMillis()
-
-        // Send AlterReplicaLogDirsRequest to allow broker to create replica in the right log dir later if the replica has not been created yet.
-        if (proposedReplicaAssignment.nonEmpty)
-          alterReplicaLogDirsIgnoreReplicaNotAvailable(proposedReplicaAssignment, adminClientOpt.get, timeoutMs)
-
-        // Create reassignment znode so that controller will send LeaderAndIsrRequest to create replica in the broker
-        zkClient.createPartitionReassignment(validPartitions.map({case (key, value) => (new TopicPartition(key.topic, key.partition), value)}).toMap)
-
-        // Send AlterReplicaLogDirsRequest again to make sure broker will start to move replica to the specified log directory.
-        // It may take some time for controller to create replica in the broker. Retry if the replica has not been created.
-        var remainingTimeMs = startTimeMs + timeoutMs - System.currentTimeMillis()
-        val replicasAssignedToFutureDir = mutable.Set.empty[TopicPartitionReplica]
-        while (remainingTimeMs > 0 && replicasAssignedToFutureDir.size < proposedReplicaAssignment.size) {
-          replicasAssignedToFutureDir ++= alterReplicaLogDirsIgnoreReplicaNotAvailable(
-            proposedReplicaAssignment.filter { case (replica, _) => !replicasAssignedToFutureDir.contains(replica) },
-            adminClientOpt.get, remainingTimeMs)
-          Thread.sleep(100)
-          remainingTimeMs = startTimeMs + timeoutMs - System.currentTimeMillis()
-        }
-        replicasAssignedToFutureDir.size == proposedReplicaAssignment.size
-      }
-    } catch {
-      case _: NodeExistsException =>
-        val partitionsBeingReassigned = zkClient.getPartitionReassignment()
-        throw new AdminCommandFailedException("Partition reassignment currently in " +
-          "progress for %s. Aborting operation".format(partitionsBeingReassigned))
-    }
-  }
-
-  def validatePartition(zkClient: KafkaZkClient, topic: String, topicPartitionReplicas: Map[TopicPartition, Seq[Int]])
-  :Map[TopicPartition, Seq[Int]] = {
-    // check if partition exists
-    val partitionsOpt = zkClient.getPartitionsForTopics(immutable.Set(topic)).get(topic)
-    topicPartitionReplicas.filter { case (topicPartition, _) =>
-      partitionsOpt match {
-        case Some(partitions) =>
-          if (partitions.contains(topicPartition.partition())) {
-            true
-          } else {
-            error("Skipping reassignment of partition [%s,%d] ".format(topic, topicPartition.partition()) +
-              "since it doesn't exist")
-            false
-          }
-        case None => error("Skipping reassignment of partition " +
-          "[%s,%d] since topic %s doesn't exist".format(topic, topicPartition.partition(), topic))
-          false
-      }
-    }
-  }
-}
-
-sealed trait ReassignmentStatus { def status: Int }
-case object ReassignmentCompleted extends ReassignmentStatus { val status = 1 }
-case object ReassignmentInProgress extends ReassignmentStatus { val status = 0 }
-case object ReassignmentFailed extends ReassignmentStatus { val status = -1 }
-
diff --git a/core/src/test/scala/integration/kafka/admin/ReassignPartitionsIntegrationTest.scala b/core/src/test/scala/integration/kafka/admin/ReassignPartitionsIntegrationTest.scala
index 5d33552..0993bfe 100644
--- a/core/src/test/scala/integration/kafka/admin/ReassignPartitionsIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/admin/ReassignPartitionsIntegrationTest.scala
@@ -1,175 +1,602 @@
-/**
-  * 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.
-  */
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package kafka.admin
 
-import java.util.Optional
+import java.io.Closeable
+import java.util.{Collections, HashMap, List}
 
-import kafka.admin.TopicCommand.ZookeeperTopicService
+import kafka.admin.ReassignPartitionsCommand._
 import kafka.server.{KafkaConfig, KafkaServer}
 import kafka.utils.TestUtils
-import kafka.zk.ZooKeeperTestHarness
-import org.apache.kafka.clients.admin.{AdminClientConfig, NewPartitionReassignment, NewTopic, AdminClient => JAdminClient}
-import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.errors.InvalidReplicaAssignmentException
+import kafka.zk.{KafkaZkClient, ZooKeeperTestHarness}
+import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, AlterConfigOp, ConfigEntry, DescribeLogDirsResult, NewTopic}
+import org.apache.kafka.clients.producer.ProducerRecord
+import org.apache.kafka.common.{TopicPartition, TopicPartitionReplica}
+import org.apache.kafka.common.config.ConfigResource
 import org.apache.kafka.common.network.ListenerName
-import org.apache.kafka.test.{TestUtils => JTestUtils}
-import org.junit.{After, Before, Test}
+import org.apache.kafka.common.security.auth.SecurityProtocol
+import org.apache.kafka.common.utils.Utils
+import org.junit.rules.Timeout
+import org.junit.Assert.{assertEquals, assertFalse, assertTrue}
+import org.junit.{After, Rule, Test}
 
+import scala.collection.Map
 import scala.collection.JavaConverters._
-import scala.collection.Seq
-
-class ReassignPartitionsIntegrationTest extends ZooKeeperTestHarness with RackAwareTest {
-  import ReassignPartitionsIntegrationTest._
-
-  var servers: Seq[KafkaServer] = Seq()
-  val broker1 = 0
-  val broker2 = 1
-  val broker3 = 2
-  val broker4 = 3
-  val broker5 = 4
-  val broker6 = 5
-  val rack = Map(
-    broker1 -> "rack1",
-    broker2 -> "rack2",
-    broker3 -> "rack2",
-    broker4 -> "rack1",
-    broker5 -> "rack3",
-    broker6 -> "rack3"
-  )
-
-  @Before
-  override def setUp(): Unit = {
-    super.setUp()
-
-    val brokerConfigs = TestUtils.createBrokerConfigs(6, zkConnect, enableControlledShutdown = true)
-    servers = brokerConfigs.map { config =>
-      config.setProperty(KafkaConfig.RackProp, rack(config.getProperty(KafkaConfig.BrokerIdProp).toInt))
-      config.setProperty(KafkaConfig.AutoLeaderRebalanceEnableProp, "false")
-      config.setProperty(KafkaConfig.ControlledShutdownMaxRetriesProp, "1")
-      config.setProperty(KafkaConfig.ControlledShutdownRetryBackoffMsProp, "1000")
-      config.setProperty(KafkaConfig.ReplicaLagTimeMaxMsProp, "1000")
-      TestUtils.createServer(KafkaConfig.fromProps(config))
-    }
+import scala.collection.{Seq, mutable}
+
+class ReassignPartitionsIntegrationTest extends ZooKeeperTestHarness {
+  @Rule
+  def globalTimeout: Timeout = Timeout.millis(300000)
+
+  var cluster: ReassignPartitionsTestCluster = null
+
+  def generateConfigs: Seq[KafkaConfig] = {
+    TestUtils.createBrokerConfigs(5, zkConnect).map(KafkaConfig.fromProps)
   }
 
   @After
   override def tearDown(): Unit = {
-    TestUtils.shutdownServers(servers)
+    Utils.closeQuietly(cluster, "ReassignPartitionsTestCluster")
     super.tearDown()
   }
 
+  val unthrottledBrokerConfigs =
+    0.to(4).map {
+      case brokerId => (brokerId, brokerLevelThrottles.map {
+        case throttleName => (throttleName, -1L)
+      }.toMap)
+    }.toMap
+
+  /**
+   * Test running a quick reassignment.
+   */
   @Test
-  def testRackAwareReassign(): Unit = {
-    val numPartitions = 18
-    val replicationFactor = 3
-
-    // create a non rack aware assignment topic first
-    val createOpts = new kafka.admin.TopicCommand.TopicCommandOptions(Array(
-      "--partitions", numPartitions.toString,
-      "--replication-factor", replicationFactor.toString,
-      "--disable-rack-aware",
-      "--topic", "foo"))
-    new ZookeeperTopicService(zkClient).createTopic(createOpts)
-
-    val topicJson = """{"topics": [{"topic": "foo"}], "version":1}"""
-    val (proposedAssignment, currentAssignment) = ReassignPartitionsCommand.generateAssignment(zkClient,
-      rack.keys.toSeq.sorted, topicJson, disableRackAware = false)
-
-    val assignment = proposedAssignment map { case (topicPartition, replicas) =>
-      (topicPartition.partition, replicas)
-    }
-    checkReplicaDistribution(assignment, rack, rack.size, numPartitions, replicationFactor)
+  def testReassignment(): Unit = {
+    cluster = new ReassignPartitionsTestCluster(zkConnect)
+    cluster.setup()
+    val assignment = """{"version":1,"partitions":""" +
+      """[{"topic":"foo","partition":0,"replicas":[0,1,3],"log_dirs":["any","any","any"]},""" +
+      """{"topic":"bar","partition":0,"replicas":[3,2,0],"log_dirs":["any","any","any"]}""" +
+      """]}"""
+
+    // Check that the assignment has not yet been started yet.
+    val initialAssignment = Map(
+      new TopicPartition("foo", 0) ->
+        PartitionReassignmentState(Seq(0, 1, 2), Seq(0, 1, 3), true),
+      new TopicPartition("bar", 0) ->
+        PartitionReassignmentState(Seq(3, 2, 1), Seq(3, 2, 0), true)
+    )
+    waitForVerifyAssignment(cluster.adminClient, assignment, false,
+      VerifyAssignmentResult(initialAssignment))
+    waitForVerifyAssignment(zkClient, assignment, false,
+      VerifyAssignmentResult(initialAssignment))
+
+    // Execute the assignment
+    runExecuteAssignment(cluster.adminClient, false, assignment, -1L, -1L)
+    assertEquals(unthrottledBrokerConfigs,
+      describeBrokerLevelThrottles(unthrottledBrokerConfigs.keySet.toSeq))
+    val finalAssignment = Map(
+      new TopicPartition("foo", 0) ->
+        PartitionReassignmentState(Seq(0, 1, 3), Seq(0, 1, 3), true),
+      new TopicPartition("bar", 0) ->
+        PartitionReassignmentState(Seq(3, 2, 0), Seq(3, 2, 0), true)
+    )
+
+    // When using --zookeeper, we aren't able to see the new-style assignment
+    assertFalse(runVerifyAssignment(zkClient, assignment, false).movesOngoing)
+
+    // Wait for the assignment to complete
+    waitForVerifyAssignment(zkClient, assignment, false,
+      VerifyAssignmentResult(finalAssignment))
+
+    assertEquals(unthrottledBrokerConfigs,
+      describeBrokerLevelThrottles(unthrottledBrokerConfigs.keySet.toSeq))
   }
 
+  /**
+   * Test running a quick reassignment with the --zookeeper option.
+   */
   @Test
-  def testReassignPartition(): Unit = {
-    TestUtils.resource(JAdminClient.create(createConfig(servers).asJava)) { client =>
-      val topic = "test-topic"
-      val partition = 0: Integer
+  def testLegacyReassignment(): Unit = {
+    cluster = new ReassignPartitionsTestCluster(zkConnect)
+    cluster.setup()
+    val assignment = """{"version":1,"partitions":""" +
+      """[{"topic":"foo","partition":0,"replicas":[0,1,3],"log_dirs":["any","any","any"]},""" +
+      """{"topic":"bar","partition":0,"replicas":[3,2,0],"log_dirs":["any","any","any"]}""" +
+      """]}"""
+    // Execute the assignment
+    runExecuteAssignment(zkClient, assignment, -1L)
+    val finalAssignment = Map(
+      new TopicPartition("foo", 0) ->
+        PartitionReassignmentState(Seq(0, 1, 3), Seq(0, 1, 3), true),
+      new TopicPartition("bar", 0) ->
+        PartitionReassignmentState(Seq(3, 2, 0), Seq(3, 2, 0), true)
+    )
+    // Wait for the assignment to complete
+    waitForVerifyAssignment(cluster.adminClient, assignment, false,
+      VerifyAssignmentResult(finalAssignment))
+    waitForVerifyAssignment(zkClient, assignment, false,
+      VerifyAssignmentResult(finalAssignment))
+  }
 
-      val partitionAssignment = Map(partition -> Seq(broker1: Integer, broker2:Integer).asJava).asJava
-      val newTopic = new NewTopic(topic, partitionAssignment)
-      client.createTopics(Seq(newTopic).asJava).all().get()
+  @Test
+  def testHighWaterMarkAfterPartitionReassignment(): Unit = {
+    cluster = new ReassignPartitionsTestCluster(zkConnect)
+    cluster.setup()
+    val assignment = """{"version":1,"partitions":""" +
+      """[{"topic":"foo","partition":0,"replicas":[3,1,2],"log_dirs":["any","any","any"]}""" +
+      """]}"""
 
-      val topicPartition = new TopicPartition(topic, partition)
+    // Set the high water mark of foo-0 to 123 on its leader.
+    val part = new TopicPartition("foo", 0)
+    cluster.servers(0).replicaManager.logManager.truncateFullyAndStartAt(part, 123L, false)
 
-      // All sync replicas are in the ISR
-      TestUtils.waitForBrokersInIsr(client, topicPartition, Set(broker1, broker2))
+    // Execute the assignment
+    runExecuteAssignment(cluster.adminClient, false, assignment, -1L, -1L)
+    val finalAssignment = Map(part ->
+        PartitionReassignmentState(Seq(3, 1, 2), Seq(3, 1, 2), true))
 
-      // Reassign replicas to different brokers
-      client.alterPartitionReassignments(
-        Map(topicPartition -> reassignmentEntry(Seq(broker3, broker4))).asJava
-      ).all().get()
+    // Wait for the assignment to complete
+    waitForVerifyAssignment(cluster.adminClient, assignment, false,
+      VerifyAssignmentResult(finalAssignment))
 
-      waitForAllReassignmentsToComplete(client)
+    TestUtils.waitUntilTrue(() => {
+      cluster.servers(3).replicaManager.nonOfflinePartition(part).
+        flatMap(_.leaderLogIfLocal).isDefined
+      }, "broker 3 should be the new leader", pause = 10L)
+    assertEquals(s"Expected broker 3 to have the correct high water mark for the " +
+      "partition.", 123L, cluster.servers(3).replicaManager.
+      localLogOrException(part).highWatermark)
+  }
 
-      // Metadata info is eventually consistent wait for update
-      TestUtils.waitForReplicasAssigned(client, topicPartition, Seq(broker3, broker4))
-      // All sync replicas are in the ISR
-      TestUtils.waitForBrokersInIsr(client, topicPartition, Set(broker3, broker4))
-    }
+  /**
+   * Test running a reassignment with the interBrokerThrottle set.
+   */
+  @Test
+  def testThrottledReassignment(): Unit = {
+    cluster = new ReassignPartitionsTestCluster(zkConnect)
+    cluster.setup()
+    cluster.produceMessages("foo", 0, 50)
+    cluster.produceMessages("baz", 2, 60)
+    val assignment = """{"version":1,"partitions":""" +
+      """[{"topic":"foo","partition":0,"replicas":[0,3,2],"log_dirs":["any","any","any"]},""" +
+      """{"topic":"baz","partition":2,"replicas":[3,2,1],"log_dirs":["any","any","any"]}""" +
+      """]}"""
+
+    // Check that the assignment has not yet been started yet.
+    val initialAssignment = Map(
+      new TopicPartition("foo", 0) ->
+        PartitionReassignmentState(Seq(0, 1, 2), Seq(0, 3, 2), true),
+      new TopicPartition("baz", 2) ->
+        PartitionReassignmentState(Seq(0, 2, 1), Seq(3, 2, 1), true))
+    assertEquals(VerifyAssignmentResult(initialAssignment),
+      runVerifyAssignment(cluster.adminClient, assignment, false))
+    assertEquals(VerifyAssignmentResult(initialAssignment),
+      runVerifyAssignment(zkClient, assignment, false))
+    assertEquals(unthrottledBrokerConfigs,
+      describeBrokerLevelThrottles(unthrottledBrokerConfigs.keySet.toSeq))
+
+    // Execute the assignment
+    val interBrokerThrottle = 300000L
+    runExecuteAssignment(cluster.adminClient, false, assignment, interBrokerThrottle, -1L)
+
+    val throttledConfigMap = Map[String, Long](
+      brokerLevelLeaderThrottle -> interBrokerThrottle,
+      brokerLevelFollowerThrottle -> interBrokerThrottle,
+      brokerLevelLogDirThrottle -> -1L
+    )
+    val throttledBrokerConfigs = Map[Int, Map[String, Long]](
+      0 -> throttledConfigMap,
+      1 -> throttledConfigMap,
+      2 -> throttledConfigMap,
+      3 -> throttledConfigMap,
+      4 -> unthrottledBrokerConfigs(4)
+    )
+    waitForBrokerLevelThrottles(throttledBrokerConfigs)
+    val finalAssignment = Map(
+      new TopicPartition("foo", 0) ->
+        PartitionReassignmentState(Seq(0, 3, 2), Seq(0, 3, 2), true),
+      new TopicPartition("baz", 2) ->
+        PartitionReassignmentState(Seq(3, 2, 1), Seq(3, 2, 1), true))
+
+    // Wait for the assignment to complete
+    TestUtils.waitUntilTrue(
+      () => {
+        // Check the reassignment status.
+        val result = runVerifyAssignment(cluster.adminClient, assignment, true)
+        if (!result.partsOngoing) {
+          true
+        } else {
+          assertTrue("Expected at least one partition reassignment to be ongoing when " +
+            s"result = ${result}", !result.partStates.forall(_._2.done))
+          assertEquals(Seq(0, 3, 2),
+            result.partStates(new TopicPartition("foo", 0)).targetReplicas)
+          assertEquals(Seq(3, 2, 1),
+            result.partStates(new TopicPartition("baz", 2)).targetReplicas)
+          logger.info(s"Current result: ${result}")
+          waitForBrokerLevelThrottles(throttledBrokerConfigs)
+          false
+        }
+      }, "Expected reassignment to complete.")
+    waitForVerifyAssignment(cluster.adminClient, assignment, true,
+      VerifyAssignmentResult(finalAssignment))
+    waitForVerifyAssignment(zkClient, assignment, true,
+      VerifyAssignmentResult(finalAssignment))
+    // The throttles should still have been preserved, since we ran with --preserve-throttles
+    waitForBrokerLevelThrottles(throttledBrokerConfigs)
+    // Now remove the throttles.
+    waitForVerifyAssignment(cluster.adminClient, assignment, false,
+      VerifyAssignmentResult(finalAssignment))
+    waitForBrokerLevelThrottles(unthrottledBrokerConfigs)
   }
 
   @Test
-  def testInvalidReplicaIds(): Unit = {
-    TestUtils.resource(JAdminClient.create(createConfig(servers).asJava)) { client =>
-      val topic = "test-topic"
-      val partition = 0: Integer
-
-      val partitionAssignment = Map(partition -> Seq(broker1: Integer, broker2: Integer).asJava).asJava
-      val newTopic = new NewTopic(topic, partitionAssignment)
-      client.createTopics(Seq(newTopic).asJava).all().get()
-
-      val topicPartition = new TopicPartition(topic, partition)
-
-      // All sync replicas are in the ISR
-      TestUtils.waitForBrokersInIsr(client, topicPartition, Set(broker1, broker2))
-
-      // Test reassignment with duplicate broker ids
-      var future = client.alterPartitionReassignments(
-        Map(topicPartition -> reassignmentEntry(Seq(broker4, broker5, broker5))).asJava
-      ).all()
-      JTestUtils.assertFutureThrows(future, classOf[InvalidReplicaAssignmentException])
-
-      // Test reassignment with invalid broker ids
-      future = client.alterPartitionReassignments(
-        Map(topicPartition -> reassignmentEntry(Seq(-1, broker3))).asJava
-      ).all()
-      JTestUtils.assertFutureThrows(future, classOf[InvalidReplicaAssignmentException])
-
-      // Test reassignment with extra broker ids
-      future = client.alterPartitionReassignments(
-        Map(topicPartition -> reassignmentEntry(Seq(6, broker2, broker3))).asJava
-      ).all()
-      JTestUtils.assertFutureThrows(future, classOf[InvalidReplicaAssignmentException])
+  def testProduceAndConsumeWithReassignmentInProgress(): Unit = {
+    cluster = new ReassignPartitionsTestCluster(zkConnect)
+    cluster.setup()
+    cluster.produceMessages("baz", 2, 60)
+    val assignment = """{"version":1,"partitions":""" +
+      """[{"topic":"baz","partition":2,"replicas":[3,2,1],"log_dirs":["any","any","any"]}""" +
+      """]}"""
+    runExecuteAssignment(cluster.adminClient, false, assignment, 300L, -1L)
+    cluster.produceMessages("baz", 2, 100)
+    val consumer = TestUtils.createConsumer(cluster.brokerList)
+    val part = new TopicPartition("baz", 2)
+    try {
+      consumer.assign(Seq(part).asJava)
+      TestUtils.pollUntilAtLeastNumRecords(consumer, numRecords = 100)
+    } finally {
+      consumer.close()
     }
+    TestUtils.removeReplicationThrottleForPartitions(cluster.adminClient, Seq(0,1,2,3), Set(part))
+    val finalAssignment = Map(part ->
+      PartitionReassignmentState(Seq(3, 2, 1), Seq(3, 2, 1), true))
+    waitForVerifyAssignment(cluster.adminClient, assignment, false,
+      VerifyAssignmentResult(finalAssignment))
   }
-}
 
-object ReassignPartitionsIntegrationTest {
-  def createConfig(servers: Seq[KafkaServer]): Map[String, Object] = {
-    Map(
-      AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG -> TestUtils.bootstrapServers(servers, new ListenerName("PLAINTEXT")),
-      AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG -> "20000"
+  /**
+   * Test running a reassignment and then cancelling it.
+   */
+  @Test
+  def testCancellation(): Unit = {
+    cluster = new ReassignPartitionsTestCluster(zkConnect)
+    cluster.setup()
+    cluster.produceMessages("foo", 0, 60)
+    cluster.produceMessages("baz", 1, 80)
+    val assignment = """{"version":1,"partitions":""" +
+      """[{"topic":"foo","partition":0,"replicas":[0,1,3],"log_dirs":["any","any","any"]},""" +
+      """{"topic":"baz","partition":1,"replicas":[0,2,3],"log_dirs":["any","any","any"]}""" +
+      """]}"""
+    assertEquals(unthrottledBrokerConfigs,
+      describeBrokerLevelThrottles(unthrottledBrokerConfigs.keySet.toSeq))
+    val interBrokerThrottle = 100L
+    runExecuteAssignment(cluster.adminClient, false, assignment, interBrokerThrottle, -1L)
+    val throttledConfigMap = Map[String, Long](
+      brokerLevelLeaderThrottle -> interBrokerThrottle,
+      brokerLevelFollowerThrottle -> interBrokerThrottle,
+      brokerLevelLogDirThrottle -> -1L
+    )
+    val throttledBrokerConfigs = Map[Int, Map[String, Long]](
+      0 -> throttledConfigMap,
+      1 -> throttledConfigMap,
+      2 -> throttledConfigMap,
+      3 -> throttledConfigMap,
+      4 -> unthrottledBrokerConfigs(4)
+    )
+    waitForBrokerLevelThrottles(throttledBrokerConfigs)
+    // Verify that the reassignment is running.  The very low throttle should keep it
+    // from completing before this runs.
+    waitForVerifyAssignment(cluster.adminClient, assignment, true,
+      VerifyAssignmentResult(Map(new TopicPartition("foo", 0) ->
+        PartitionReassignmentState(Seq(0, 1, 3, 2), Seq(0, 1, 3), false),
+        new TopicPartition("baz", 1) ->
+          PartitionReassignmentState(Seq(0, 2, 3, 1), Seq(0, 2, 3), false)),
+      true, Map(), false))
+    // Cancel the reassignment.
+    assertEquals((Set(
+        new TopicPartition("foo", 0),
+        new TopicPartition("baz", 1)
+      ), Set()), runCancelAssignment(cluster.adminClient, assignment, true))
+    // Broker throttles are still active because we passed --preserve-throttles
+    waitForBrokerLevelThrottles(throttledBrokerConfigs)
+    // Cancelling the reassignment again should reveal nothing to cancel.
+    assertEquals((Set(), Set()), runCancelAssignment(cluster.adminClient, assignment, false))
+    // This time, the broker throttles were removed.
+    waitForBrokerLevelThrottles(unthrottledBrokerConfigs)
+    // Verify that there are no ongoing reassignments.
+    assertFalse(runVerifyAssignment(cluster.adminClient, assignment, false).partsOngoing)
+  }
+
+  private def waitForBrokerLevelThrottles(targetThrottles: Map[Int, Map[String, Long]]): Unit = {
+    var curThrottles: Map[Int, Map[String, Long]] = Map.empty
+    TestUtils.waitUntilTrue(() => {
+      curThrottles = describeBrokerLevelThrottles(targetThrottles.keySet.toSeq)
+      targetThrottles.equals(curThrottles)
+    }, s"timed out waiting for broker throttle to become ${targetThrottles}.  " +
+      s"Latest throttles were ${curThrottles}", pause = 25)
+  }
+
+  /**
+   * Describe the broker-level throttles in the cluster.
+   *
+   * @return                A map whose keys are broker IDs and whose values are throttle
+   *                        information.  The nested maps are keyed on throttle name.
+   */
+  private def describeBrokerLevelThrottles(brokerIds: Seq[Int]): Map[Int, Map[String, Long]] = {
+    brokerIds.map {
+      case brokerId =>
+        val props = zkClient.getEntityConfigs("brokers", brokerId.toString)
+        (brokerId, brokerLevelThrottles.map {
+          case throttleName => (throttleName,
+            props.getOrDefault(throttleName, "-1").asInstanceOf[String].toLong)
+        }.toMap)
+    }.toMap
+  }
+
+  /**
+   * Test moving partitions between directories.
+   */
+  @Test
+  def testReplicaDirectoryMoves(): Unit = {
+    cluster = new ReassignPartitionsTestCluster(zkConnect)
+    cluster.setup()
+    cluster.produceMessages("foo", 0, 7000)
+    cluster.produceMessages("baz", 1, 6000)
+
+    val result0 = cluster.adminClient.describeLogDirs(
+        0.to(4).map(_.asInstanceOf[Integer]).asJavaCollection)
+    val info0 = new BrokerDirs(result0, 0)
+    assertTrue(info0.futureLogDirs.isEmpty)
+    assertEquals(Set(new TopicPartition("foo", 0),
+        new TopicPartition("baz", 0),
+        new TopicPartition("baz", 1),
+        new TopicPartition("baz", 2)),
+      info0.curLogDirs.keySet)
+    val curFoo1Dir = info0.curLogDirs.getOrElse(new TopicPartition("foo", 0), "")
+    assertFalse(curFoo1Dir.equals(""))
+    val newFoo1Dir = info0.logDirs.find(!_.equals(curFoo1Dir)).get
+    val assignment = """{"version":1,"partitions":""" +
+        """[{"topic":"foo","partition":0,"replicas":[0,1,2],""" +
+          "\"log_dirs\":[\"%s\",\"any\",\"any\"]}".format(newFoo1Dir) +
+            "]}"
+    // Start the replica move, but throttle it to be very slow so that it can't complete
+    // before our next checks happen.
+    runExecuteAssignment(cluster.adminClient, false, assignment, -1L, 1L)
+
+    // Check the output of --verify
+    waitForVerifyAssignment(cluster.adminClient, assignment, true,
+      VerifyAssignmentResult(Map(
+          new TopicPartition("foo", 0) -> PartitionReassignmentState(Seq(0, 1, 2), Seq(0, 1, 2), true)
+        ), false, Map(
+          new TopicPartitionReplica("foo", 0, 0) -> ActiveMoveState(curFoo1Dir, newFoo1Dir, newFoo1Dir)
+        ), true))
+
+    // Check that the appropriate broker throttle is in place.
+    val throttledConfigMap = Map[String, Long](
+      brokerLevelLeaderThrottle -> -1,
+      brokerLevelFollowerThrottle -> -1,
+      brokerLevelLogDirThrottle -> 1L
+    )
+    val throttledBrokerConfigs = Map[Int, Map[String, Long]](
+      0 -> throttledConfigMap,
+      1 -> unthrottledBrokerConfigs(1),
+      2 -> unthrottledBrokerConfigs(2),
+      3 -> unthrottledBrokerConfigs(3),
+      4 -> unthrottledBrokerConfigs(4)
     )
+    waitForBrokerLevelThrottles(throttledBrokerConfigs)
+
+    // Remove the throttle
+    cluster.adminClient.incrementalAlterConfigs(Collections.singletonMap(
+      new ConfigResource(ConfigResource.Type.BROKER, "0"),
+      Collections.singletonList(new AlterConfigOp(
+        new ConfigEntry(brokerLevelLogDirThrottle, ""), AlterConfigOp.OpType.DELETE)))).
+          all().get()
+    waitForBrokerLevelThrottles(unthrottledBrokerConfigs)
+
+    // Wait for the directory movement to complete.
+    waitForVerifyAssignment(cluster.adminClient, assignment, true,
+        VerifyAssignmentResult(Map(
+          new TopicPartition("foo", 0) -> PartitionReassignmentState(Seq(0, 1, 2), Seq(0, 1, 2), true)
+        ), false, Map(
+          new TopicPartitionReplica("foo", 0, 0) -> CompletedMoveState(newFoo1Dir)
+        ), false))
+
+    val info1 = new BrokerDirs(cluster.adminClient.describeLogDirs(0.to(4).
+        map(_.asInstanceOf[Integer]).asJavaCollection), 0)
+    assertEquals(newFoo1Dir,
+      info1.curLogDirs.getOrElse(new TopicPartition("foo", 0), ""))
+  }
+
+  private def runVerifyAssignment(adminClient: Admin, jsonString: String,
+                                  preserveThrottles: Boolean) = {
+    println(s"==> verifyAssignment(adminClient, jsonString=${jsonString})")
+    verifyAssignment(adminClient, jsonString, preserveThrottles)
+  }
+
+  private def waitForVerifyAssignment(adminClient: Admin, jsonString: String,
+                                      preserveThrottles: Boolean,
+                                      expectedResult: VerifyAssignmentResult): Unit = {
+    var latestResult: VerifyAssignmentResult = null
+    TestUtils.waitUntilTrue(
+      () => {
+        latestResult = runVerifyAssignment(adminClient, jsonString, preserveThrottles)
+        expectedResult.equals(latestResult)
+      }, s"Timed out waiting for verifyAssignment result ${expectedResult}.  " +
+        s"The latest result was ${latestResult}", pause = 10L)
+  }
+
+  private def runVerifyAssignment(zkClient: KafkaZkClient, jsonString: String,
+                                  preserveThrottles: Boolean) = {
+    println(s"==> verifyAssignment(zkClient, jsonString=${jsonString})")
+    verifyAssignment(zkClient, jsonString, preserveThrottles)
+  }
+
+  private def waitForVerifyAssignment(zkClient: KafkaZkClient, jsonString: String,
+                                      preserveThrottles: Boolean,
+                                      expectedResult: VerifyAssignmentResult): Unit = {
+    var latestResult: VerifyAssignmentResult = null
+    TestUtils.waitUntilTrue(
+      () => {
+        println(s"==> verifyAssignment(zkClient, jsonString=${jsonString}, " +
+          s"preserveThrottles=${preserveThrottles})")
+        latestResult = verifyAssignment(zkClient, jsonString, preserveThrottles)
+        expectedResult.equals(latestResult)
+      }, s"Timed out waiting for verifyAssignment result ${expectedResult}.  " +
+        s"The latest result was ${latestResult}", pause = 10L)
+  }
+
+  private def runExecuteAssignment(adminClient: Admin,
+                        additional: Boolean,
+                        reassignmentJson: String,
+                        interBrokerThrottle: Long,
+                        replicaAlterLogDirsThrottle: Long) = {
+    println(s"==> executeAssignment(adminClient, additional=${additional}, " +
+      s"reassignmentJson=${reassignmentJson}, " +
+      s"interBrokerThrottle=${interBrokerThrottle}, " +
+      s"replicaAlterLogDirsThrottle=${replicaAlterLogDirsThrottle}))")
+    executeAssignment(adminClient, additional, reassignmentJson,
+      interBrokerThrottle, replicaAlterLogDirsThrottle)
+  }
+
+  private def runExecuteAssignment(zkClient: KafkaZkClient,
+                                   reassignmentJson: String,
+                                   interBrokerThrottle: Long) = {
+    println(s"==> executeAssignment(adminClient, " +
+      s"reassignmentJson=${reassignmentJson}, " +
+      s"interBrokerThrottle=${interBrokerThrottle})")
+    executeAssignment(zkClient, reassignmentJson, interBrokerThrottle)
+  }
+
+  private def runCancelAssignment(adminClient: Admin, jsonString: String,
+                                  preserveThrottles: Boolean) = {
+    println(s"==> cancelAssignment(adminClient, jsonString=${jsonString})")
+    cancelAssignment(adminClient, jsonString, preserveThrottles)
   }
 
-  def reassignmentEntry(replicas: Seq[Int]): Optional[NewPartitionReassignment] = {
-    Optional.of(new NewPartitionReassignment(replicas.map(r => r: Integer).asJava))
+  class BrokerDirs(result: DescribeLogDirsResult, val brokerId: Int) {
+    val logDirs = new mutable.HashSet[String]
+    val curLogDirs = new mutable.HashMap[TopicPartition, String]
+    val futureLogDirs = new mutable.HashMap[TopicPartition, String]
+    result.values().get(brokerId).get().asScala.foreach {
+      case (logDirName, logDirInfo) => {
+        logDirs.add(logDirName)
+        logDirInfo.replicaInfos.asScala.foreach {
+          case (part, info) =>
+            if (info.isFuture) {
+              futureLogDirs.put(part, logDirName)
+            } else {
+              curLogDirs.put(part, logDirName)
+            }
+        }
+      }
+    }
   }
 
-  def waitForAllReassignmentsToComplete(client: JAdminClient): Unit = {
-    TestUtils.waitUntilTrue(() => client.listPartitionReassignments().reassignments().get().isEmpty,
-      s"There still are ongoing reassignments", pause = 100L)
+  class ReassignPartitionsTestCluster(val zkConnect: String) extends Closeable {
+    val brokers = Map(
+      0 -> "rack0",
+      1 -> "rack0",
+      2 -> "rack1",
+      3 -> "rack1",
+      4 -> "rack1"
+    )
+
+    val topics = Map(
+      "foo" -> Seq(Seq(0, 1, 2), Seq(1, 2, 3)),
+      "bar" -> Seq(Seq(3, 2, 1)),
+      "baz" -> Seq(Seq(1, 0, 2), Seq(2, 0, 1), Seq(0, 2, 1))
+    )
+
+    val brokerConfigs = brokers.map {
+      case (brokerId, rack) =>
+        val config = TestUtils.createBrokerConfig(
+          nodeId = brokerId,
+          zkConnect = zkConnect,
+          rack = Some(rack),
+          enableControlledShutdown = false, // shorten test time
+          logDirCount = 3)
+        // shorter backoff to reduce test durations when no active partitions are eligible for fetching due to throttling
+        config.setProperty(KafkaConfig.ReplicaFetchBackoffMsProp, "100")
+        // Don't move partition leaders automatically.
+        config.setProperty(KafkaConfig.AutoLeaderRebalanceEnableProp, "false")
+        config.setProperty(KafkaConfig.ReplicaLagTimeMaxMsProp, "1000")
+        config
+    }.toBuffer
+
+    var servers = new mutable.ArrayBuffer[KafkaServer]
+
+    var brokerList: String = null
+
+    var adminClient: Admin = null
+
+    def setup(): Unit = {
+      createServers()
+      createTopics()
+    }
+
+    def createServers(): Unit = {
+      brokers.keySet.foreach {
+        case brokerId =>
+          servers += TestUtils.createServer(KafkaConfig(brokerConfigs(brokerId)))
+      }
+    }
+
+    def createTopics(): Unit = {
+      TestUtils.waitUntilBrokerMetadataIsPropagated(servers)
+      brokerList = TestUtils.bootstrapServers(servers,
+        ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT))
+      adminClient = Admin.create(Map[String, Object](
+        AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG -> brokerList
+      ).asJava)
+      adminClient.createTopics(topics.map {
+        case (topicName, parts) =>
+          val partMap = new HashMap[Integer, List[Integer]]()
+          parts.zipWithIndex.foreach {
+            case (part, index) => partMap.put(index, part.map(Integer.valueOf(_)).asJava)
+          }
+          new NewTopic(topicName, partMap)
+      }.toList.asJava).all().get()
+      topics.foreach {
+        case (topicName, parts) =>
+          parts.indices.foreach {
+            index => TestUtils.waitUntilMetadataIsPropagated(servers, topicName, index)
+          }
+      }
+    }
+
+    def produceMessages(topic: String, partition: Int, numMessages: Int): Unit = {
+      val records = (0 until numMessages).map(_ =>
+        new ProducerRecord[Array[Byte], Array[Byte]](topic, partition,
+          null, new Array[Byte](10000)))
+      TestUtils.produceMessages(servers, records, -1)
+    }
+
+    override def close(): Unit = {
+      brokerList = null
+      Utils.closeQuietly(adminClient, "adminClient")
+      adminClient = null
+      try {
+        TestUtils.shutdownServers(servers)
+      } finally {
+        servers.clear()
+      }
+    }
   }
 }
diff --git a/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala b/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala
index 5f31818..b579537 100644
--- a/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala
+++ b/core/src/test/scala/other/kafka/ReplicationQuotasTestRig.scala
@@ -20,16 +20,19 @@ package kafka
 import java.io.{File, PrintWriter}
 import java.nio.charset.StandardCharsets
 import java.nio.file.{Files, StandardOpenOption}
-import javax.imageio.ImageIO
 
+import javax.imageio.ImageIO
 import kafka.admin.ReassignPartitionsCommand
-import kafka.admin.ReassignPartitionsCommand.Throttle
 import kafka.server.{KafkaConfig, KafkaServer, QuotaType}
 import kafka.utils.TestUtils._
 import kafka.utils.{Exit, Logging, TestUtils}
 import kafka.zk.{ReassignPartitionsZNode, ZooKeeperTestHarness}
+import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
 import org.apache.kafka.clients.producer.ProducerRecord
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.network.ListenerName
+import org.apache.kafka.common.security.auth.SecurityProtocol
+import org.apache.kafka.common.utils.Utils
 import org.jfree.chart.plot.PlotOrientation
 import org.jfree.chart.{ChartFactory, ChartFrame, JFreeChart}
 import org.jfree.data.xy.{XYSeries, XYSeriesCollection}
@@ -100,14 +103,23 @@ object ReplicationQuotasTestRig {
     var servers: Seq[KafkaServer] = null
     val leaderRates = mutable.Map[Int, Array[Double]]()
     val followerRates = mutable.Map[Int, Array[Double]]()
+    var adminClient: Admin = null
 
     def startBrokers(brokerIds: Seq[Int]): Unit = {
       println("Starting Brokers")
       servers = brokerIds.map(i => createBrokerConfig(i, zkConnect))
         .map(c => createServer(KafkaConfig.fromProps(c)))
+
+      TestUtils.waitUntilBrokerMetadataIsPropagated(servers)
+      val brokerList = TestUtils.bootstrapServers(servers,
+        ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT))
+      adminClient = Admin.create(Map[String, Object](
+        AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG -> brokerList
+      ).asJava)
     }
 
     override def tearDown(): Unit = {
+      Utils.closeQuietly(adminClient, "adminClient")
       TestUtils.shutdownServers(servers)
       super.tearDown()
     }
@@ -135,12 +147,15 @@ object ReplicationQuotasTestRig {
         }
       }
 
-      println("Starting Reassignment")
-      val newAssignment = ReassignPartitionsCommand.generateAssignment(zkClient, brokers, json(topicName), true)._1
+      println("Generating Reassignment")
+      val (newAssignment, _) = ReassignPartitionsCommand.generateAssignment(adminClient,
+        json(topicName), brokers.mkString(","), true)
 
+      println("Starting Reassignment")
       val start = System.currentTimeMillis()
-      ReassignPartitionsCommand.executeAssignment(zkClient, None,
-        new String(ReassignPartitionsZNode.encode(newAssignment), StandardCharsets.UTF_8), Throttle(config.throttle))
+      ReassignPartitionsCommand.executeAssignment(adminClient, false,
+        new String(ReassignPartitionsZNode.encode(newAssignment), StandardCharsets.UTF_8),
+        config.throttle)
 
       //Await completion
       waitForReassignmentToComplete()
@@ -190,8 +205,8 @@ object ReplicationQuotasTestRig {
     def waitForReassignmentToComplete(): Unit = {
       waitUntilTrue(() => {
         printRateMetrics()
-        !zkClient.reassignPartitionsInProgress()
-      }, s"Znode ${ReassignPartitionsZNode.path} wasn't deleted", 60 * 60 * 1000, pause = 1000L)
+        adminClient.listPartitionReassignments().reassignments().get().isEmpty
+      }, s"Partition reassignments didn't complete.", 60 * 60 * 1000, pause = 1000L)
     }
 
     def renderChart(data: mutable.Map[Int, Array[Double]], name: String, journal: Journal, displayChartsOnScreen: Boolean): Unit = {
diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
index eb61dd2..dd1ea68 100644
--- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
@@ -16,20 +16,21 @@
  */
 package kafka.admin
 
-import java.util.Properties
+import java.util
+import java.util.concurrent.ExecutionException
+import java.util.{Collections, Optional, Properties}
 
 import scala.collection.Seq
-
 import kafka.log.Log
 import kafka.zk.{TopicPartitionZNode, ZooKeeperTestHarness}
 import kafka.utils.TestUtils
 import kafka.server.{KafkaConfig, KafkaServer}
 import org.junit.Assert._
 import org.junit.{After, Test}
-
 import kafka.admin.TopicCommand.ZookeeperTopicService
 import kafka.common.TopicAlreadyMarkedForDeletionException
 import kafka.controller.{OfflineReplica, PartitionAndReplica, ReplicaAssignment, ReplicaDeletionSuccessful}
+import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, NewPartitionReassignment}
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.errors.UnknownTopicOrPartitionException
 import org.scalatest.Assertions.fail
@@ -127,32 +128,46 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
     follower.shutdown()
     // start topic deletion
     adminZkClient.deleteTopic(topic)
-    // start partition reassignment at the same time right after delete topic. In this case, reassignment will fail since
-    // the topic is being deleted
-    // reassign partition 0
-    val oldAssignedReplicas = zkClient.getReplicasForPartition(new TopicPartition(topic, 0))
-    val newReplicas = Seq(1, 2, 3)
-    val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, None,
-      Map(topicPartition -> newReplicas),  adminZkClient = adminZkClient)
-    assertTrue("Partition reassignment should fail for [test,0]", reassignPartitionsCommand.reassignPartitions())
-    // wait until reassignment is completed
-    TestUtils.waitUntilTrue(() => {
-      ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, Map(topicPartition -> newReplicas))
-        .getOrElse(topicPartition, fail(s"Failed to get reassignment status for $topicPartition")) == ReassignmentFailed
-    }, "Partition reassignment shouldn't complete.")
-    val controllerId = zkClient.getControllerId.getOrElse(fail("Controller doesn't exist"))
-    val controller = servers.filter(s => s.config.brokerId == controllerId).head
-
-    // partitionsBeingReassigned is updated after re-assignment znode is removed, so wait again
-    TestUtils.waitUntilTrue(() => {
-      !controller.kafkaController.controllerContext.partitionsBeingReassigned.contains(topicPartition)
-    }, "Partition should be removed from partitionsBeingReassigned.")
-    val assignedReplicas = zkClient.getReplicasForPartition(new TopicPartition(topic, 0))
-    assertEquals("Partition should not be reassigned to 0, 1, 2", oldAssignedReplicas, assignedReplicas)
+    // verify that a partition from the topic cannot be reassigned
+    val props = new Properties()
+    props.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers))
+    val adminClient = Admin.create(props)
+    try {
+      waitUntilTopicGone(adminClient, "test")
+      verifyReassignmentFailsForMissing(adminClient, new TopicPartition(topic, 0),
+        new NewPartitionReassignment(util.Arrays.asList(1, 2, 3)))
+    } finally {
+      adminClient.close()
+    }
     follower.startup()
     TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers)
   }
 
+  private def waitUntilTopicGone(adminClient: Admin, topicName: String): Unit = {
+    TestUtils.waitUntilTrue(() => {
+      try {
+        adminClient.describeTopics(util.Collections.singletonList(topicName)).all().get()
+        false
+      } catch {
+        case e: ExecutionException =>
+          classOf[UnknownTopicOrPartitionException].equals(e.getCause.getClass)
+      }
+    }, s"Topic ${topicName} should be deleted.")
+  }
+
+  private def verifyReassignmentFailsForMissing(adminClient: Admin,
+                                                partition: TopicPartition,
+                                                reassignment: NewPartitionReassignment): Unit = {
+    try {
+      adminClient.alterPartitionReassignments(Collections.singletonMap(partition,
+        Optional.of(new NewPartitionReassignment(util.Arrays.asList(1, 2, 3))))).all().get()
+      fail("expected partition reassignment to fail for [test,0]")
+    } catch {
+      case e: ExecutionException =>
+        assertEquals(classOf[UnknownTopicOrPartitionException], e.getCause.getClass)
+    }
+  }
+
   private def getController() : (KafkaServer, Int) = {
     val controllerId = zkClient.getControllerId.getOrElse(fail("Controller doesn't exist"))
     val controller = servers.find(s => s.config.brokerId == controllerId).get
diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala
deleted file mode 100644
index 6dccdd0..0000000
--- a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala
+++ /dev/null
@@ -1,1304 +0,0 @@
-/**
-  * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
-  * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
-  * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
-  * License. You may obtain a copy of the License at
-  *
-  * http://www.apache.org/licenses/LICENSE-2.0
-  *
-  * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
-  * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
-  * specific language governing permissions and limitations under the License.
-  */
-package kafka.admin
-
-import kafka.admin.ReassignPartitionsCommand._
-import kafka.common.AdminCommandFailedException
-import kafka.controller.ReplicaAssignment
-import kafka.server.{KafkaConfig, KafkaServer}
-import kafka.utils.TestUtils._
-import kafka.utils.{Logging, TestUtils}
-import kafka.zk.{ReassignPartitionsZNode, ZkVersion, ZooKeeperTestHarness}
-import org.junit.Assert.{assertEquals, assertFalse, assertTrue}
-import org.junit.{After, Before, Test}
-import kafka.admin.ReplicationQuotaUtils._
-import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, NewPartitionReassignment, NewPartitions, PartitionReassignment}
-import org.apache.kafka.common.{TopicPartition, TopicPartitionReplica}
-
-import scala.collection.JavaConverters._
-import scala.collection.{Map, Seq}
-import scala.util.Random
-import java.io.File
-import java.util.{Collections, Optional, Properties}
-import java.util.concurrent.ExecutionException
-
-import org.apache.kafka.clients.producer.ProducerRecord
-import org.apache.kafka.common.errors.{NoReassignmentInProgressException, ReassignmentInProgressException}
-import org.scalatest.Assertions.intercept
-
-
-class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
-  var servers: Seq[KafkaServer] = null
-  var brokerIds: Seq[Int] = null
-  val topicName = "my-topic"
-  val tp0 = new TopicPartition(topicName, 0)
-  val tp1 = new TopicPartition(topicName, 1)
-  val delayMs = 1000
-  var adminClient: Admin = null
-
-  def zkUpdateDelay(): Unit = Thread.sleep(delayMs)
-
-  @Before
-  override def setUp(): Unit = {
-    super.setUp()
-  }
-
-  def startBrokers(ids: Seq[Int]): Unit = {
-    brokerIds = ids
-    servers = ids.map { i =>
-      val props = createBrokerConfig(i, zkConnect, enableControlledShutdown = false, logDirCount = 3)
-      // shorter backoff to reduce test durations when no active partitions are eligible for fetching due to throttling
-      props.put(KafkaConfig.ReplicaFetchBackoffMsProp, "100")
-      props
-    }.map(c => createServer(KafkaConfig.fromProps(c)))
-  }
-
-  def createAdminClient(servers: Seq[KafkaServer]): Admin = {
-    val props = new Properties()
-    props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers))
-    props.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, "10000")
-    props.put(AdminClientConfig.DEFAULT_API_TIMEOUT_MS_CONFIG, "15000")
-    Admin.create(props)
-  }
-
-  def getRandomLogDirAssignment(brokerId: Int, excluded: Option[String] = None): String = {
-    val server = servers.find(_.config.brokerId == brokerId).get
-    val logDirs = server.config.logDirs.filterNot(excluded.contains)
-    new File(logDirs(Random.nextInt(logDirs.size))).getAbsolutePath
-  }
-
-  @After
-  override def tearDown(): Unit = {
-    if (adminClient != null) {
-      adminClient.close()
-      adminClient = null
-    }
-    TestUtils.shutdownServers(servers)
-    super.tearDown()
-  }
-
-  @Test
-  def testHwAfterPartitionReassignment(): Unit = {
-    //Given a single replica on server 100
-    startBrokers(Seq(100, 101, 102))
-    adminClient = createAdminClient(servers)
-    createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
-
-    val leaderServer = servers.find(_.config.brokerId == 100).get
-    leaderServer.replicaManager.logManager.truncateFullyAndStartAt(tp0, 100L, false)
-
-    val topicJson = executeAssignmentJson(Seq(
-      PartitionAssignmentJson(tp0, replicas=Seq(101, 102))
-    ))
-    ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, NoThrottle)
-
-    val newLeaderServer = servers.find(_.config.brokerId == 101).get
-
-    waitUntilTrue (
-      () => newLeaderServer.replicaManager.nonOfflinePartition(tp0).flatMap(_.leaderLogIfLocal).isDefined,
-      "broker 101 should be the new leader", pause = 1L
-    )
-
-    assertEquals(100, newLeaderServer.replicaManager.localLogOrException(tp0)
-      .highWatermark)
-    val newFollowerServer = servers.find(_.config.brokerId == 102).get
-    waitUntilTrue(() => newFollowerServer.replicaManager.localLogOrException(tp0)
-      .highWatermark == 100,
-      "partition follower's highWatermark should be 100")
-  }
-
-  @Test
-  def shouldMoveSinglePartition(): Unit = {
-    //Given a single replica on server 100
-    startBrokers(Seq(100, 101))
-    adminClient = createAdminClient(servers)
-    // Get a random log directory on broker 101
-    val expectedLogDir = getRandomLogDirAssignment(101)
-    createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
-
-    //When we move the replica on 100 to broker 101
-    val topicJson = executeAssignmentJson(Seq(
-      PartitionAssignmentJson(tp0, replicas = Seq(101), logDirectories = Some(Seq(expectedLogDir)))
-    ))
-    ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, NoThrottle)
-    waitForZkReassignmentToComplete()
-
-    //Then the replica should be on 101
-    val partitionAssignment = zkClient.getPartitionAssignmentForTopics(Set(topicName)).get(topicName).get(tp0.partition())
-    assertMoveForPartitionOccurred(Seq(101), partitionAssignment)
-    // The replica should be in the expected log directory on broker 101
-    val replica = new TopicPartitionReplica(topicName, 0, 101)
-    assertEquals(expectedLogDir, adminClient.describeReplicaLogDirs(Collections.singleton(replica)).all().get.get(replica).getCurrentReplicaLogDir)
-  }
-
-  @Test
-  def testReassignmentMatchesCurrentAssignment(): Unit = {
-    // Given a single replica on server 100
-    startBrokers(Seq(100))
-    adminClient = createAdminClient(servers)
-    createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
-
-    // Execute no-op reassignment
-    val topicJson = executeAssignmentJson(Seq(
-      PartitionAssignmentJson(tp0, replicas = Seq(100), logDirectories = None)
-    ))
-    ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, NoThrottle)
-    waitForZkReassignmentToComplete()
-
-    // The replica should remain on 100
-    val partitionAssignment = zkClient.getPartitionAssignmentForTopics(Set(topicName))(topicName)(tp0.partition)
-    assertMoveForPartitionOccurred(Seq(100), partitionAssignment)
-  }
-
-  @Test
-  def shouldMoveSinglePartitionToSameFolderWithinBroker(): Unit = shouldMoveSinglePartitionWithinBroker(true)
-
-  @Test
-  def shouldMoveSinglePartitionToDifferentFolderWithinBroker(): Unit = shouldMoveSinglePartitionWithinBroker(false)
-
-  private[this] def shouldMoveSinglePartitionWithinBroker(moveToSameFolder: Boolean): Unit = {
-    // Given a single replica on server 100
-    startBrokers(Seq(100, 101))
-    adminClient = createAdminClient(servers)
-    createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
-
-    val replica = new TopicPartitionReplica(topicName, 0, 100)
-    val currentLogDir = adminClient.describeReplicaLogDirs(java.util.Collections.singleton(replica))
-      .all()
-      .get()
-      .get(replica)
-      .getCurrentReplicaLogDir
-
-    val expectedLogDir = if (moveToSameFolder) currentLogDir else getRandomLogDirAssignment(100, excluded = Some(currentLogDir))
-
-    // When we execute an assignment that moves an existing replica to another log directory on the same broker
-    val topicJson = executeAssignmentJson(Seq(
-      PartitionAssignmentJson(tp0, replicas = Seq(100), logDirectories = Some(Seq(expectedLogDir)))
-    ))
-    ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, NoThrottle)
-    waitUntilTrue(() => {
-      expectedLogDir == adminClient.describeReplicaLogDirs(Collections.singleton(replica)).all().get.get(replica).getCurrentReplicaLogDir
-    }, "Partition should have been moved to the expected log directory", 1000)
-  }
-
-  @Test
-  def shouldExpandCluster(): Unit = {
-    val brokers = Array(100, 101, 102)
-    startBrokers(brokers)
-    adminClient = createAdminClient(servers)
-    createTopic(zkClient, topicName, Map(
-      0 -> Seq(100, 101),
-      1 -> Seq(100, 101),
-      2 -> Seq(100, 101)
-    ), servers = servers)
-
-    //When rebalancing
-    val newAssignment = generateAssignment(zkClient, brokers, generateAssignmentJson(topicName), true)._1
-    // Find a partition in the new assignment on broker 102 and a random log directory on broker 102,
-    // which currently does not have any partition for this topic
-    val partition1 = newAssignment.find { case (_, brokerIds) => brokerIds.contains(102) }.get._1.partition
-    val replica1 = new TopicPartitionReplica(topicName, partition1, 102)
-    val expectedLogDir1 = getRandomLogDirAssignment(102)
-    // Find a partition in the new assignment on broker 100 and a random log directory on broker 100,
-    // which currently has partition for this topic
-    val partition2 = newAssignment.find { case (_, brokerIds) => brokerIds.contains(100) }.get._1.partition
-    val replica2 = new TopicPartitionReplica(topicName, partition2, 100)
-    val expectedLogDir2 = getRandomLogDirAssignment(100)
-    // Generate a replica assignment to reassign replicas on broker 100 and 102 respectively to a random log directory on the same broker.
-    // Before this reassignment, the replica already exists on broker 100 but does not exist on broker 102
-    val newReplicaAssignment = Map(replica1 -> expectedLogDir1, replica2 -> expectedLogDir2)
-    ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient),
-      ReassignPartitionsCommand.formatAsReassignmentJson(newAssignment, newReplicaAssignment), NoThrottle)
-    waitForZkReassignmentToComplete()
-
-    // Then the replicas should span all three brokers
-    val actual = zkClient.getPartitionAssignmentForTopics(Set(topicName))(topicName)
-    assertMoveForTopicOccurred(Seq(100, 101, 102), actual)
-    // The replica should be in the expected log directory on broker 102 and 100
-    waitUntilTrue(() => {
-      expectedLogDir1 == adminClient.describeReplicaLogDirs(Collections.singleton(replica1)).all().get.get(replica1).getCurrentReplicaLogDir
-    }, "Partition should have been moved to the expected log directory on broker 102", 1000)
-    waitUntilTrue(() => {
-      expectedLogDir2 == adminClient.describeReplicaLogDirs(Collections.singleton(replica2)).all().get.get(replica2).getCurrentReplicaLogDir
-    }, "Partition should have been moved to the expected log directory on broker 100", 1000)
-  }
-
-  @Test
-  def shouldShrinkCluster(): Unit = {
-    //Given partitions on 3 of 3 brokers
-    val brokers = Array(100, 101, 102)
-    startBrokers(brokers)
-    createTopic(zkClient, topicName, Map(
-      0 -> Seq(100, 101),
-      1 -> Seq(101, 102),
-      2 -> Seq(102, 100)
-    ), servers = servers)
-
-    //When rebalancing
-    val newAssignment = generateAssignment(zkClient, Array(100, 101), generateAssignmentJson(topicName), true)._1
-    ReassignPartitionsCommand.executeAssignment(zkClient, None,
-      ReassignPartitionsCommand.formatAsReassignmentJson(newAssignment, Map.empty), NoThrottle)
-    waitForZkReassignmentToComplete()
-
-    //Then replicas should only span the first two brokers
-    val actual = zkClient.getPartitionAssignmentForTopics(Set(topicName))(topicName)
-    assertMoveForTopicOccurred(Seq(100, 101), actual)
-  }
-
-  @Test
-  def shouldMoveSubsetOfPartitions(): Unit = {
-    //Given partitions on 3 of 3 brokers
-    val brokers = Array(100, 101, 102)
-    startBrokers(brokers)
-    adminClient = createAdminClient(servers)
-    createTopic(zkClient, "topic1", Map(
-      0 -> Seq(100, 101),
-      1 -> Seq(101, 102),
-      2 -> Seq(102, 100)
-    ), servers = servers)
-    createTopic(zkClient, "topic2", Map(
-      0 -> Seq(100, 101),
-      1 -> Seq(101, 102),
-      2 -> Seq(102, 100)
-    ), servers = servers)
-
-    val proposed: Map[TopicPartition, Seq[Int]] = Map(
-      new TopicPartition("topic1", 0) -> Seq(100, 102),
-      new TopicPartition("topic1", 2) -> Seq(100, 102),
-      new TopicPartition("topic2", 1) -> Seq(101, 100),
-      new TopicPartition("topic2", 2) -> Seq(100, 102)
-    )
-
-    val replica1 = new TopicPartitionReplica("topic1", 0, 102)
-    val replica2 = new TopicPartitionReplica("topic2", 1, 100)
-    val proposedReplicaAssignment: Map[TopicPartitionReplica, String] = Map(
-      replica1 -> getRandomLogDirAssignment(102),
-      replica2 -> getRandomLogDirAssignment(100)
-    )
-
-    //When rebalancing
-    ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient),
-      ReassignPartitionsCommand.formatAsReassignmentJson(proposed, proposedReplicaAssignment), NoThrottle)
-    waitForZkReassignmentToComplete()
-
-    //Then the proposed changes should have been made
-    val actual = zkClient.getPartitionAssignmentForTopics(Set("topic1", "topic2"))
-    assertMoveForPartitionOccurred(Seq(100, 102), actual("topic1")(0)) //changed
-    assertMoveForPartitionOccurred(Seq(101, 102), actual("topic1")(1))
-    assertMoveForPartitionOccurred(Seq(100, 102), actual("topic1")(2)) //changed
-    assertMoveForPartitionOccurred(Seq(100, 101), actual("topic2")(0))
-    assertMoveForPartitionOccurred(Seq(101, 100), actual("topic2")(1)) //changed
-    assertMoveForPartitionOccurred(Seq(100, 102), actual("topic2")(2)) //changed
-
-    // The replicas should be in the expected log directories
-    val replicaDirs = adminClient.describeReplicaLogDirs(List(replica1, replica2).asJava).all().get()
-    assertEquals(proposedReplicaAssignment(replica1), replicaDirs.get(replica1).getCurrentReplicaLogDir)
-    assertEquals(proposedReplicaAssignment(replica2), replicaDirs.get(replica2).getCurrentReplicaLogDir)
-  }
-
-  @Test
-  def shouldExecuteThrottledReassignment(): Unit = {
-
-    //Given partitions on 3 of 3 brokers
-    val brokers = Array(100, 101, 102)
-    startBrokers(brokers)
-    createTopic(zkClient, topicName, Map(
-      0 -> Seq(100, 101)
-    ), servers = servers)
-
-    //Given throttle set so replication will take a certain number of secs
-    val initialThrottle = Throttle(10 * 1000 * 1000, -1, () => zkUpdateDelay)
-    val expectedDurationSecs = 5
-    val numMessages = 500
-    val msgSize = 100 * 1000
-    produceMessages(topicName, numMessages, acks = 0, msgSize)
-    assertEquals(expectedDurationSecs, numMessages * msgSize / initialThrottle.interBrokerLimit)
-
-    //Start rebalance which will move replica on 100 -> replica on 102
-    val newAssignment = generateAssignment(zkClient, Array(101, 102), generateAssignmentJson(topicName), true)._1
-
-    val start = System.currentTimeMillis()
-    ReassignPartitionsCommand.executeAssignment(zkClient, None,
-      ReassignPartitionsCommand.formatAsReassignmentJson(newAssignment, Map.empty), initialThrottle)
-
-    //Check throttle config. Should be throttling replica 0 on 100 and 102 only.
-    checkThrottleConfigAddedToZK(adminZkClient, initialThrottle.interBrokerLimit, servers, topicName, Set("0:100","0:101"), Set("0:102"))
-
-    //Await completion
-    waitForZkReassignmentToComplete()
-    val took = System.currentTimeMillis() - start - delayMs
-
-    //Check move occurred
-    val actual = zkClient.getPartitionAssignmentForTopics(Set(topicName))(topicName)
-    assertMoveForTopicOccurred(Seq(101, 102), actual)
-
-    //Then command should have taken longer than the throttle rate
-    assertTrue(s"Expected replication to be > ${expectedDurationSecs * 0.9 * 1000} but was $took",
-      took > expectedDurationSecs * 0.9 * 1000)
-    assertTrue(s"Expected replication to be < ${expectedDurationSecs * 2 * 1000} but was $took",
-      took < expectedDurationSecs * 2 * 1000)
-  }
-
-
-  @Test
-  def shouldOnlyThrottleMovingReplicas(): Unit = {
-    //Given 6 brokers, two topics
-    val brokers = Array(100, 101, 102, 103, 104, 105)
-    startBrokers(brokers)
-    createTopic(zkClient, "topic1", Map(
-      0 -> Seq(100, 101),
-      1 -> Seq(100, 101),
-      2 -> Seq(103, 104) //will leave in place
-    ), servers = servers)
-
-    createTopic(zkClient, "topic2", Map(
-      0 -> Seq(104, 105),
-      1 -> Seq(104, 105),
-      2 -> Seq(103, 104)//will leave in place
-    ), servers = servers)
-
-    //Given throttle set so replication will take a while
-    val throttle: Long = 1000 * 1000
-    produceMessages("topic1", 100, acks = 0, 100 * 1000)
-    produceMessages("topic2", 100, acks = 0, 100 * 1000)
-
-    //Start rebalance
-    val newAssignment = Map(
-      new TopicPartition("topic1", 0) -> Seq(100, 102),//moved 101=>102
-      new TopicPartition("topic1", 1) -> Seq(100, 102),//moved 101=>102
-      new TopicPartition("topic2", 0) -> Seq(103, 105),//moved 104=>103
-      new TopicPartition("topic2", 1) -> Seq(103, 105),//moved 104=>103
-      new TopicPartition("topic1", 2) -> Seq(103, 104), //didn't move
-      new TopicPartition("topic2", 2) -> Seq(103, 104)  //didn't move
-    )
-    ReassignPartitionsCommand.executeAssignment(zkClient, None,
-      ReassignPartitionsCommand.formatAsReassignmentJson(newAssignment, Map.empty), Throttle(throttle))
-
-    //Check throttle config. Should be throttling specific replicas for each topic.
-    checkThrottleConfigAddedToZK(adminZkClient, throttle, servers, "topic1",
-      Set("1:100","1:101","0:100","0:101"), //All replicas for moving partitions should be leader-throttled
-      Set("1:102","0:102") //Move destinations should be follower throttled.
-    )
-    checkThrottleConfigAddedToZK(adminZkClient, throttle, servers, "topic2",
-      Set("1:104","1:105","0:104","0:105"), //All replicas for moving partitions should be leader-throttled
-      Set("1:103","0:103") //Move destinations should be follower throttled.
-    )
-  }
-
-  @Test
-  def shouldChangeThrottleOnRerunAndRemoveOnVerify(): Unit = {
-    //Given partitions on 3 of 3 brokers
-    val brokers = Array(100, 101, 102)
-    startBrokers(brokers)
-    createTopic(zkClient, topicName, Map(
-      0 -> Seq(100, 101)
-    ), servers = servers)
-
-    //Given throttle set so replication will take at least 20 sec (we won't wait this long)
-    val initialThrottle: Long = 1000 * 1000
-    produceMessages(topicName, numMessages = 200, acks = 0, valueLength = 100 * 1000)
-
-    //Start rebalance
-    val newAssignment = generateAssignment(zkClient, Array(101, 102), generateAssignmentJson(topicName), true)._1
-
-    ReassignPartitionsCommand.executeAssignment(zkClient, None,
-      ReassignPartitionsCommand.formatAsReassignmentJson(newAssignment, Map.empty), Throttle(initialThrottle))
-
-    //Check throttle config
-    checkThrottleConfigAddedToZK(adminZkClient, initialThrottle, servers, topicName, Set("0:100","0:101"), Set("0:102"))
-
-    //Ensure that running Verify, whilst the command is executing, should have no effect
-    verifyAssignment(zkClient, None, ReassignPartitionsCommand.formatAsReassignmentJson(newAssignment, Map.empty))
-
-    //Check throttle config again
-    checkThrottleConfigAddedToZK(adminZkClient, initialThrottle, servers, topicName, Set("0:100","0:101"), Set("0:102"))
-
-    //Now re-run the same assignment with a larger throttle, which should only act to increase the throttle and make progress
-    val newThrottle = initialThrottle * 1000
-
-    ReassignPartitionsCommand.executeAssignment(zkClient, None,
-      ReassignPartitionsCommand.formatAsReassignmentJson(newAssignment, Map.empty), Throttle(newThrottle))
-
-    //Check throttle was changed
-    checkThrottleConfigAddedToZK(adminZkClient, newThrottle, servers, topicName, Set("0:100","0:101"), Set("0:102"))
-
-    //Await completion
-    waitForZkReassignmentToComplete()
-
-    //Verify should remove the throttle
-    verifyAssignment(zkClient, None, ReassignPartitionsCommand.formatAsReassignmentJson(newAssignment, Map.empty))
-
-    //Check removed
-    checkThrottleConfigRemovedFromZK(adminZkClient, topicName, servers)
-
-    //Check move occurred
-    val actual = zkClient.getPartitionAssignmentForTopics(Set(topicName))(topicName)
-    assertMoveForTopicOccurred(Seq(101, 102), actual)
-  }
-
-  @Test(expected = classOf[AdminCommandFailedException])
-  def shouldFailIfProposedDoesNotMatchExisting(): Unit = {
-    //Given a single replica on server 100
-    startBrokers(Seq(100, 101))
-    createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
-
-    //When we execute an assignment that includes an invalid partition (1:101 in this case)
-    val topicJson = executeAssignmentJson(Seq(PartitionAssignmentJson(tp1, Seq(101))))
-    ReassignPartitionsCommand.executeAssignment(zkClient, None, topicJson, NoThrottle)
-  }
-
-  @Test(expected = classOf[AdminCommandFailedException])
-  def shouldFailIfProposedHasEmptyReplicaList(): Unit = {
-    //Given a single replica on server 100
-    startBrokers(Seq(100, 101))
-    createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
-
-    //When we execute an assignment that specifies an empty replica list (0: empty list in this case)
-    val topicJson = executeAssignmentJson(Seq(PartitionAssignmentJson(tp0, Seq())))
-    ReassignPartitionsCommand.executeAssignment(zkClient, None, topicJson, NoThrottle)
-  }
-
-  @Test(expected = classOf[AdminCommandFailedException])
-  def shouldFailIfProposedHasInvalidBrokerID(): Unit = {
-    //Given a single replica on server 100
-    startBrokers(Seq(100, 101))
-    createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
-
-    //When we execute an assignment that specifies an invalid brokerID (102: invalid broker ID in this case)
-    val topicJson = executeAssignmentJson(Seq(PartitionAssignmentJson(tp0, Seq(101, 102))))
-    ReassignPartitionsCommand.executeAssignment(zkClient, None, topicJson, NoThrottle)
-  }
-
-  @Test(expected = classOf[AdminCommandFailedException])
-  def shouldFailIfProposedHasInvalidLogDir(): Unit = {
-    // Given a single replica on server 100
-    startBrokers(Seq(100, 101))
-    adminClient = createAdminClient(servers)
-    createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
-
-    // When we execute an assignment that specifies an invalid log directory
-    val topicJson = executeAssignmentJson(Seq(PartitionAssignmentJson(tp0, Seq(101), logDirectories = Some(Seq("invalidDir")))))
-    ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, NoThrottle)
-  }
-
-  @Test(expected = classOf[AdminCommandFailedException])
-  def shouldFailIfProposedHasInconsistentReplicasAndLogDirs(): Unit = {
-    // Given a single replica on server 100
-    startBrokers(Seq(100, 101))
-    adminClient = createAdminClient(servers)
-    val logDir = getRandomLogDirAssignment(100)
-    createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
-
-    // When we execute an assignment whose length of replicas doesn't match that of log dirs
-    val topicJson = executeAssignmentJson(Seq(PartitionAssignmentJson(tp0, Seq(101), logDirectories = Some(Seq(logDir, logDir)))))
-    ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, NoThrottle)
-  }
-
-  @Test
-  def shouldPerformThrottledReassignmentOverVariousTopics(): Unit = {
-    val throttle = Throttle(1000L)
-
-    startBrokers(Seq(0, 1, 2, 3))
-
-    //With up several small topics
-    createTopic(zkClient, "orders", Map(0 -> List(0, 1, 2), 1 -> List(0, 1, 2)), servers)
-    createTopic(zkClient, "payments", Map(0 -> List(0, 1), 1 -> List(0, 1)), servers)
-    createTopic(zkClient, "deliveries", Map(0 -> List(0)), servers)
-    createTopic(zkClient, "customers", Map(0 -> List(0), 1 -> List(1), 2 -> List(2), 3 -> List(3)), servers)
-
-    //Define a move for some of them
-    val move = Map(
-      new TopicPartition("orders", 0) -> Seq(0, 2, 3),//moves
-      new TopicPartition("orders", 1) -> Seq(0, 1, 2),//stays
-      new TopicPartition("payments", 1) -> Seq(1, 2), //only define one partition as moving
-      new TopicPartition("deliveries", 0) -> Seq(1, 2) //increase replication factor
-    )
-
-    //When we run a throttled reassignment
-    new ReassignPartitionsCommand(zkClient, None, move, adminZkClient = adminZkClient).reassignPartitions(throttle)
-
-    waitForZkReassignmentToComplete()
-
-    //Check moved replicas did move
-    assertEquals(Seq(0, 2, 3), zkClient.getReplicasForPartition(new TopicPartition("orders", 0)))
-    assertEquals(Seq(0, 1, 2), zkClient.getReplicasForPartition(new TopicPartition("orders", 1)))
-    assertEquals(Seq(1, 2), zkClient.getReplicasForPartition(new TopicPartition("payments", 1)))
-    assertEquals(Seq(1, 2), zkClient.getReplicasForPartition(new TopicPartition("deliveries", 0)))
-
-    //Check untouched replicas are still there
-    assertEquals(Seq(0, 1), zkClient.getReplicasForPartition(new TopicPartition("payments", 0)))
-    assertEquals(Seq(0), zkClient.getReplicasForPartition(new TopicPartition("customers", 0)))
-    assertEquals(Seq(1), zkClient.getReplicasForPartition(new TopicPartition("customers", 1)))
-    assertEquals(Seq(2), zkClient.getReplicasForPartition(new TopicPartition("customers", 2)))
-    assertEquals(Seq(3), zkClient.getReplicasForPartition(new TopicPartition("customers", 3)))
-  }
-
-  /**
-   * Verifies that the Controller sets a watcher for the reassignment znode after reassignment completion.
-   * This includes the case where the znode is set immediately after it's deleted (i.e. before the watch is set).
-   * This case relies on the scheduling of the operations, so it won't necessarily fail every time, but it fails
-   * often enough to detect a regression.
-   */
-  @Test
-  def shouldPerformMultipleReassignmentOperationsOverVariousTopics(): Unit = {
-    startBrokers(Seq(0, 1, 2, 3))
-
-    createTopic(zkClient, "orders", Map(0 -> List(0, 1, 2), 1 -> List(0, 1, 2)), servers)
-    createTopic(zkClient, "payments", Map(0 -> List(0, 1), 1 -> List(0, 1)), servers)
-    createTopic(zkClient, "deliveries", Map(0 -> List(0)), servers)
-    createTopic(zkClient, "customers", Map(0 -> List(0), 1 -> List(1), 2 -> List(2), 3 -> List(3)), servers)
-
-    val firstMove = Map(
-      new TopicPartition("orders", 0) -> Seq(0, 2, 3), //moves
-      new TopicPartition("orders", 1) -> Seq(0, 1, 2), //stays
-      new TopicPartition("payments", 1) -> Seq(1, 2), //only define one partition as moving
-      new TopicPartition("deliveries", 0) -> Seq(1, 2) //increase replication factor
-    )
-
-    new ReassignPartitionsCommand(zkClient, None, firstMove, adminZkClient = adminZkClient).reassignPartitions()
-    // Low pause to detect deletion of the reassign_partitions znode before the reassignment is complete
-    waitForZkReassignmentToComplete(pause = 1L)
-
-    // Check moved replicas did move
-    assertEquals(Seq(0, 2, 3), zkClient.getReplicasForPartition(new TopicPartition("orders", 0)))
-    assertEquals(Seq(0, 1, 2), zkClient.getReplicasForPartition(new TopicPartition("orders", 1)))
-    assertEquals(Seq(1, 2), zkClient.getReplicasForPartition(new TopicPartition("payments", 1)))
-    assertEquals(Seq(1, 2), zkClient.getReplicasForPartition(new TopicPartition("deliveries", 0)))
-
-    // Check untouched replicas are still there
-    assertEquals(Seq(0, 1), zkClient.getReplicasForPartition(new TopicPartition("payments", 0)))
-    assertEquals(Seq(0), zkClient.getReplicasForPartition(new TopicPartition("customers", 0)))
-    assertEquals(Seq(1), zkClient.getReplicasForPartition(new TopicPartition("customers", 1)))
-    assertEquals(Seq(2), zkClient.getReplicasForPartition(new TopicPartition("customers", 2)))
-    assertEquals(Seq(3), zkClient.getReplicasForPartition(new TopicPartition("customers", 3)))
-
-    // Define a move for some of them
-    val secondMove = Map(
-      new TopicPartition("orders", 0) -> Seq(0, 2, 3), // stays
-      new TopicPartition("orders", 1) -> Seq(3, 1, 2), // moves
-      new TopicPartition("payments", 1) -> Seq(2, 1), // changed preferred leader
-      new TopicPartition("deliveries", 0) -> Seq(1, 2, 3) //increase replication factor
-    )
-
-    new ReassignPartitionsCommand(zkClient, None, secondMove, adminZkClient = adminZkClient).reassignPartitions()
-    // Low pause to detect deletion of the reassign_partitions znode before the reassignment is complete
-    waitForZkReassignmentToComplete(pause = 1L)
-
-    // Check moved replicas did move
-    assertEquals(Seq(0, 2, 3), zkClient.getReplicasForPartition(new TopicPartition("orders", 0)))
-    assertEquals(Seq(3, 1, 2), zkClient.getReplicasForPartition(new TopicPartition("orders", 1)))
-    assertEquals(Seq(2, 1), zkClient.getReplicasForPartition(new TopicPartition("payments", 1)))
-    assertEquals(Seq(1, 2, 3), zkClient.getReplicasForPartition(new TopicPartition("deliveries", 0)))
-
-    //Check untouched replicas are still there
-    assertEquals(Seq(0, 1), zkClient.getReplicasForPartition(new TopicPartition("payments", 0)))
-    assertEquals(Seq(0), zkClient.getReplicasForPartition(new TopicPartition("customers", 0)))
-    assertEquals(Seq(1), zkClient.getReplicasForPartition(new TopicPartition("customers", 1)))
-    assertEquals(Seq(2), zkClient.getReplicasForPartition(new TopicPartition("customers", 2)))
-    assertEquals(Seq(3), zkClient.getReplicasForPartition(new TopicPartition("customers", 3)))
-
-    // We set the znode and then continuously attempt to set it again to exercise the case where the znode is set
-    // immediately after deletion (i.e. before we set the watcher again)
-
-    val thirdMove = Map(new TopicPartition("orders", 0) -> Seq(1, 2, 3))
-
-    new ReassignPartitionsCommand(zkClient, None, thirdMove, adminZkClient = adminZkClient).reassignPartitions()
-
-    val fourthMove = Map(new TopicPartition("payments", 1) -> Seq(2, 3))
-
-    // Continuously attempt to set the reassignment znode with `fourthMove` until it succeeds. It will only succeed
-    // after `thirdMove` completes.
-    Iterator.continually {
-      try new ReassignPartitionsCommand(zkClient, None, fourthMove, adminZkClient = adminZkClient).reassignPartitions()
-      catch {
-        case _: AdminCommandFailedException => false
-      }
-    }.exists(identity)
-
-    // Low pause to detect deletion of the reassign_partitions znode before the reassignment is complete
-    waitForZkReassignmentToComplete(pause = 1L)
-
-    // Check moved replicas for thirdMove and fourthMove
-    assertEquals(Seq(1, 2, 3), zkClient.getReplicasForPartition(new TopicPartition("orders", 0)))
-    assertEquals(Seq(2, 3), zkClient.getReplicasForPartition(new TopicPartition("payments", 1)))
-
-    //Check untouched replicas are still there
-    assertEquals(Seq(3, 1, 2), zkClient.getReplicasForPartition(new TopicPartition("orders", 1)))
-    assertEquals(Seq(1, 2, 3), zkClient.getReplicasForPartition(new TopicPartition("deliveries", 0)))
-    assertEquals(Seq(0, 1), zkClient.getReplicasForPartition(new TopicPartition("payments", 0)))
-    assertEquals(Seq(0), zkClient.getReplicasForPartition(new TopicPartition("customers", 0)))
-    assertEquals(Seq(1), zkClient.getReplicasForPartition(new TopicPartition("customers", 1)))
-    assertEquals(Seq(2), zkClient.getReplicasForPartition(new TopicPartition("customers", 2)))
-    assertEquals(Seq(3), zkClient.getReplicasForPartition(new TopicPartition("customers", 3)))
-  }
-
-  /**
-   * Set the `reassign_partitions` znode while the brokers are down and verify that the reassignment is triggered by
-   * the Controller during start-up.
-   */
-  @Test
-  def shouldTriggerReassignmentOnControllerStartup(): Unit = {
-    startBrokers(Seq(0, 1, 2))
-    createTopic(zkClient, "orders", Map(0 -> List(0, 1), 1 -> List(1, 2)), servers)
-    servers.foreach(_.shutdown())
-
-    val firstMove = Map(
-      new TopicPartition("orders", 0) -> Seq(2, 1), // moves
-      new TopicPartition("orders", 1) -> Seq(1, 2), // stays
-      new TopicPartition("customers", 0) -> Seq(1, 2) // non-existent topic, triggers topic deleted path
-    )
-
-    // Set znode directly to avoid non-existent topic validation
-    zkClient.setOrCreatePartitionReassignment(firstMove, ZkVersion.MatchAnyVersion)
-
-    servers.foreach(_.startup())
-    waitForZkReassignmentToComplete()
-
-    assertEquals(Seq(2, 1), zkClient.getReplicasForPartition(new TopicPartition("orders", 0)))
-    assertEquals(Seq(1, 2), zkClient.getReplicasForPartition(new TopicPartition("orders", 1)))
-    assertEquals(Seq.empty, zkClient.getReplicasForPartition(new TopicPartition("customers", 0)))
-  }
-
-  /**
-    * Set a reassignment through the `/topics/<topic>` znode and set the `reassign_partitions` znode while the brokers are down.
-    * Verify that the reassignment is triggered by the Controller during start-up with the `reassign_partitions` znode taking precedence
-    */
-  @Test
-  def shouldTriggerReassignmentWithZnodePrecedenceOnControllerStartup(): Unit = {
-    startBrokers(Seq(0, 1, 2))
-    adminClient = createAdminClient(servers)
-    createTopic(zkClient, "orders", Map(0 -> List(0, 1), 1 -> List(1, 2), 2 -> List(0, 1), 3 -> List(0, 1)), servers)
-    val sameMoveTp = new TopicPartition("orders", 2)
-
-    // Throttle to ensure we minimize race conditions and test flakiness
-    throttle(Seq("orders"), throttleSettingForSeconds(10), Set(sameMoveTp))
-
-    servers.foreach(_.shutdown())
-    adminClient.close()
-
-    zkClient.setTopicAssignment("orders", Map(
-      new TopicPartition("orders", 0) -> ReplicaAssignment(List(0, 1), List(2), List(0)), // should be overwritten
-      new TopicPartition("orders", 1) -> ReplicaAssignment(List(1, 2), List(3), List(1)), // should be overwritten
-      // should be overwritten (so we know to remove it from ZK) even though we do the exact same move
-      sameMoveTp -> ReplicaAssignment(List(0, 1, 2), List(2), List(0)),
-      new TopicPartition("orders", 3) -> ReplicaAssignment(List(0, 1, 2), List(2), List(0)) // moves
-    ))
-    val move = Map(
-      new TopicPartition("orders", 0) -> Seq(2, 1), // moves
-      new TopicPartition("orders", 1) -> Seq(1, 2), // stays
-      sameMoveTp -> Seq(1, 2), // same reassignment
-      // orders-3 intentionally left for API
-      new TopicPartition("customers", 0) -> Seq(1, 2) // non-existent topic, triggers topic deleted path
-    )
-
-    // Set znode directly to avoid non-existent topic validation
-    zkClient.setOrCreatePartitionReassignment(move, ZkVersion.MatchAnyVersion)
-
-    servers.foreach(_.startup())
-    TestUtils.waitUntilBrokerMetadataIsPropagated(servers)
-    adminClient = createAdminClient(servers)
-    TestUtils.resetBrokersThrottle(adminClient, brokerIds)
-
-    waitForZkReassignmentToComplete()
-
-    assertEquals(Seq(2, 1), zkClient.getReplicasForPartition(new TopicPartition("orders", 0)))
-    assertEquals(Seq(1, 2), zkClient.getReplicasForPartition(new TopicPartition("orders", 1)))
-    assertEquals(Seq(1, 2), zkClient.getReplicasForPartition(sameMoveTp))
-    assertEquals(Seq(1, 2), zkClient.getReplicasForPartition(new TopicPartition("orders", 3)))
-    assertEquals(Seq.empty, zkClient.getReplicasForPartition(new TopicPartition("customers", 0)))
-  }
-
-  @Test
-  def shouldListReassignmentsTriggeredByZk(): Unit = {
-    // Given a single replica on server 100
-    startBrokers(Seq(100, 101))
-    adminClient = createAdminClient(servers)
-    // Get a random log directory on broker 101
-    val expectedLogDir = getRandomLogDirAssignment(101)
-    createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
-    // Given throttle set so replication will take at least 2 sec (to ensure we don't minimize race condition and test flakiness
-    val throttle: Long = 1000 * 1000
-    produceMessages(topicName, numMessages = 20, acks = 0, valueLength = 100 * 1000)
-
-    // When we move the replica on 100 to broker 101
-    val topicJson = executeAssignmentJson(Seq(
-      PartitionAssignmentJson(tp0, replicas = Seq(101), Some(Seq(expectedLogDir)))))
-    ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, Throttle(throttle))
-    // Then the replica should be removing
-    val reassigningPartitionsResult = adminClient.listPartitionReassignments(Set(tp0).asJava).reassignments().get().get(tp0)
-    assertIsReassigning(from = Seq(100), to = Seq(101), reassigningPartitionsResult)
-
-    waitForZkReassignmentToComplete()
-
-    // Then the replica should be on 101
-    val partitionAssignment = zkClient.getPartitionAssignmentForTopics(Set(topicName)).get(topicName).get(tp0.partition())
-    assertMoveForPartitionOccurred(Seq(101), partitionAssignment)
-  }
-
-  @Test
-  def shouldReassignThroughApi(): Unit = {
-    startBrokers(Seq(100, 101))
-    adminClient = createAdminClient(servers)
-    createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
-
-    assertTrue(adminClient.listPartitionReassignments(Set(tp0).asJava).reassignments().get().isEmpty)
-    assertEquals(Seq(100), zkClient.getReplicasForPartition(tp0))
-    adminClient.alterPartitionReassignments(
-      Map(reassignmentEntry(tp0, Seq(101))).asJava
-    ).all().get()
-
-    waitForAllReassignmentsToComplete()
-    assertEquals(Seq(101), zkClient.getReplicasForPartition(tp0))
-  }
-
-  @Test
-  def testProduceAndConsumeWithReassignmentInProgress(): Unit = {
-    startBrokers(Seq(100, 101))
-    adminClient = createAdminClient(servers)
-    createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100)), servers = servers)
-
-    produceMessages(tp0.topic, 500, acks = -1, valueLength = 100 * 1000)
-
-    TestUtils.setReplicationThrottleForPartitions(adminClient, Seq(101), Set(tp0), throttleBytes = 1)
-
-    adminClient.alterPartitionReassignments(
-      Map(reassignmentEntry(tp0, Seq(100, 101))).asJava
-    ).all().get()
-
-    awaitReassignmentInProgress(tp0)
-
-    produceMessages(tp0.topic, 500, acks = -1, valueLength = 64)
-    val consumer = TestUtils.createConsumer(TestUtils.getBrokerListStrFromServers(servers))
-    try {
-      consumer.assign(Seq(tp0).asJava)
-      pollUntilAtLeastNumRecords(consumer, numRecords = 1000)
-    } finally {
-      consumer.close()
-    }
-
-    assertTrue(isAssignmentInProgress(tp0))
-
-    TestUtils.removeReplicationThrottleForPartitions(adminClient, brokerIds, Set(tp0))
-
-    waitForAllReassignmentsToComplete()
-    assertEquals(Seq(100, 101), zkClient.getReplicasForPartition(tp0))
-  }
-
-  @Test
-  def shouldListMovingPartitionsThroughApi(): Unit = {
-    startBrokers(Seq(100, 101))
-    adminClient = createAdminClient(servers)
-    val topic2 = "topic2"
-    val tp2 = new TopicPartition(topic2, 0)
-
-    createTopic(zkClient, topicName,
-      Map(tp0.partition() -> Seq(100),
-          tp1.partition() -> Seq(101)),
-      servers = servers)
-    createTopic(zkClient, topic2,
-      Map(tp2.partition() -> Seq(100)),
-      servers = servers)
-    assertTrue(adminClient.listPartitionReassignments().reassignments().get().isEmpty)
-
-    // Throttle to ensure we minimize race conditions and test flakiness
-    throttle(Seq(topicName), throttleSettingForSeconds(10), Set(tp0, tp2))
-
-    adminClient.alterPartitionReassignments(
-      Map(reassignmentEntry(tp0, Seq(101)),
-          reassignmentEntry(tp2, Seq(101))).asJava
-    ).all().get()
-
-    val reassignmentsInProgress = adminClient.listPartitionReassignments(Set(tp0, tp1, tp2).asJava).reassignments().get()
-    assertFalse(reassignmentsInProgress.containsKey(tp1)) // tp1 is not reassigning
-    assertIsReassigning(from = Seq(100), to = Seq(101), reassignmentsInProgress.get(tp0))
-    assertIsReassigning(from = Seq(100), to = Seq(101), reassignmentsInProgress.get(tp2))
-
-    TestUtils.resetBrokersThrottle(adminClient, brokerIds)
-    waitForAllReassignmentsToComplete()
-    assertEquals(Seq(101), zkClient.getReplicasForPartition(tp0))
-    assertEquals(Seq(101), zkClient.getReplicasForPartition(tp2))
-  }
-
-  @Test
-  def shouldUseLatestOrderingIfTwoConsecutiveReassignmentsHaveSameSetButDifferentOrdering(): Unit = {
-    startBrokers(Seq(100, 101, 102))
-    adminClient = createAdminClient(servers)
-    createTopic(zkClient, topicName,
-      Map(tp0.partition() -> Seq(100, 101),
-          tp1.partition() -> Seq(100, 101)),
-      servers = servers)
-
-    // Throttle to ensure we minimize race conditions and test flakiness
-    throttle(Seq(topicName), throttleSettingForSeconds(10), Set(tp0, tp1))
-
-    adminClient.alterPartitionReassignments(
-      Map(reassignmentEntry(tp0, Seq(100, 101, 102)),
-          reassignmentEntry(tp1, Seq(100, 101, 102))).asJava
-    ).all().get()
-    val apiReassignmentsInProgress = adminClient.listPartitionReassignments(Set(tp0, tp1).asJava).reassignments().get()
-    assertIsReassigning(from = Seq(100, 101), to = Seq(100, 101, 102), apiReassignmentsInProgress.get(tp0))
-    assertIsReassigning(from = Seq(100, 101), to = Seq(100, 101, 102), apiReassignmentsInProgress.get(tp1))
-
-    // API reassignment to the same replicas but a different order
-    adminClient.alterPartitionReassignments(
-      Map(reassignmentEntry(tp0, Seq(102, 101, 100)),
-          reassignmentEntry(tp1, Seq(102, 101, 100))).asJava
-    ).all().get()
-    val apiReassignmentsInProgress2 = adminClient.listPartitionReassignments(Set(tp0, tp1).asJava).reassignments().get()
-    // assert same replicas, ignoring ordering
-    assertIsReassigning(from = Seq(100, 101), to = Seq(100, 101, 102), apiReassignmentsInProgress2.get(tp0))
-    assertIsReassigning(from = Seq(100, 101), to = Seq(100, 101, 102), apiReassignmentsInProgress2.get(tp1))
-
-    TestUtils.resetBrokersThrottle(adminClient, brokerIds)
-    waitForAllReassignmentsToComplete()
-
-    //Check move occurred
-    val actual = zkClient.getPartitionAssignmentForTopics(Set(topicName))(topicName)
-    assertMoveForPartitionOccurred(Seq(102, 101, 100), actual(tp0.partition()))
-    assertMoveForPartitionOccurred(Seq(102, 101, 100), actual(tp1.partition()))
-  }
-
-  /**
-    * 1. Trigger API reassignment for partitions
-    * 2. Trigger ZK reassignment for partitions
-    * Ensure ZK reassignment overrides API reassignment and znode is deleted
-    */
-  @Test
-  def znodeReassignmentShouldOverrideApiTriggeredReassignment(): Unit = {
-    startBrokers(Seq(100, 101, 102))
-    adminClient = createAdminClient(servers)
-    createTopic(zkClient, topicName,
-      Map(tp0.partition() -> Seq(100),
-          tp1.partition() -> Seq(100)),
-      servers = servers)
-
-    // Throttle to avoid race conditions
-    val throttleSetting = throttleSettingForSeconds(10)
-    throttle(Seq(topicName), throttleSetting, Set(tp0, tp1))
-
-    // API reassignment to 101 for both partitions
-    adminClient.alterPartitionReassignments(
-      Map(reassignmentEntry(tp0, Seq(101)),
-          reassignmentEntry(tp1, Seq(101))).asJava
-    ).all().get()
-    val apiReassignmentsInProgress = adminClient.listPartitionReassignments(Set(tp0, tp1).asJava).reassignments().get()
-    assertIsReassigning(from = Seq(100), to = Seq(101), apiReassignmentsInProgress.get(tp0))
-    assertIsReassigning(from = Seq(100), to = Seq(101), apiReassignmentsInProgress.get(tp1))
-
-    // znode reassignment to 102 for both partitions
-    val topicJson = executeAssignmentJson(Seq(
-      PartitionAssignmentJson(tp0, Seq(102)),
-      PartitionAssignmentJson(tp1, Seq(102))
-    ))
-    ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, Throttle(throttleSetting.throttleBytes.toLong))
-    waitUntilTrue(() => {
-      !adminClient.listPartitionReassignments().reassignments().get().isEmpty
-    }, "Controller should have picked up on znode creation", 1000)
-
-    val zkReassignmentsInProgress = adminClient.listPartitionReassignments(Set(tp0, tp1).asJava).reassignments().get()
-    assertIsReassigning(from = Seq(100), to = Seq(102), zkReassignmentsInProgress.get(tp0))
-    assertIsReassigning(from = Seq(100), to = Seq(102), zkReassignmentsInProgress.get(tp1))
-
-    TestUtils.resetBrokersThrottle(adminClient, brokerIds)
-    waitForZkReassignmentToComplete()
-    assertTrue(adminClient.listPartitionReassignments(Set(tp0, tp1).asJava).reassignments().get().isEmpty)
-    assertEquals(Seq(102), zkClient.getReplicasForPartition(tp0))
-    assertEquals(Seq(102), zkClient.getReplicasForPartition(tp1))
-  }
-
-  /**
-    * 1. Trigger ZK reassignment for TP A-0, A-1
-    * 2. Trigger API reassignment for partitions TP A-1, B-0
-    * 3. Unthrottle A-0, A-1 so the ZK reassignment finishes quickly
-    * 4. Ensure ZK node is emptied out after the API reassignment of 1 finishes
-    */
-  @Test
-  def shouldDeleteReassignmentZnodeAfterApiReassignmentForPartitionCompletes(): Unit = {
-    startBrokers(Seq(100, 101, 102))
-    adminClient = createAdminClient(servers)
-    val tpA0 = new TopicPartition("A", 0)
-    val tpA1 = new TopicPartition("A", 1)
-    val tpB0 = new TopicPartition("B", 0)
-
-    createTopic(zkClient, "A",
-      Map(tpA0.partition() -> Seq(100),
-          tpA1.partition() -> Seq(100)),
-      servers = servers)
-    createTopic(zkClient, "B",
-      Map(tpB0.partition() -> Seq(100)),
-      servers = servers)
-
-    // Throttle to avoid race conditions
-    throttle(Seq("A", "B"), throttleSettingForSeconds(10), Set(tpA0, tpA1, tpB0))
-
-    // 1. znode reassignment to 101 for TP A-0, A-1
-    val topicJson = executeAssignmentJson(Seq(
-      PartitionAssignmentJson(tpA0, replicas=Seq(101)),
-      PartitionAssignmentJson(tpA1, replicas=Seq(101))
-    ))
-    ReassignPartitionsCommand.executeAssignment(zkClient, Some(adminClient), topicJson, NoThrottle)
-    waitUntilTrue(() => {
-      !adminClient.listPartitionReassignments().reassignments().get().isEmpty
-    }, "Controller should have picked up on znode creation", 1000)
-    val zkReassignmentsInProgress = adminClient.listPartitionReassignments(Set(tpA0, tpA1).asJava).reassignments().get()
-    assertIsReassigning(from = Seq(100), to = Seq(101), zkReassignmentsInProgress.get(tpA0))
-    assertIsReassigning(from = Seq(100), to = Seq(101), zkReassignmentsInProgress.get(tpA1))
-
-    // 2. API reassignment to 102 for TP A-1, B-0
-    adminClient.alterPartitionReassignments(
-      Map(reassignmentEntry(tpA1, Seq(102)), reassignmentEntry(tpB0, Seq(102))).asJava
-    ).all().get()
-    val apiReassignmentsInProgress = adminClient.listPartitionReassignments(Set(tpA1, tpB0).asJava).reassignments().get()
-    assertIsReassigning(from = Seq(100), to = Seq(102), apiReassignmentsInProgress.get(tpA1))
-    assertIsReassigning(from = Seq(100), to = Seq(102), apiReassignmentsInProgress.get(tpB0))
-
-    // 3. Unthrottle topic A
-    TestUtils.removePartitionReplicaThrottles(adminClient, Set(tpA0, tpA1))
-    waitForZkReassignmentToComplete()
-    // 4. Ensure the API reassignment not part of the znode is still in progress
-    val leftoverReassignments = adminClient.listPartitionReassignments(Set(tpA0, tpA1, tpB0).asJava).reassignments().get()
-    assertTrue(leftoverReassignments.keySet().asScala.subsetOf(Set(tpA1, tpB0)))
-
-    TestUtils.resetBrokersThrottle(adminClient, brokerIds)
-    waitForAllReassignmentsToComplete()
-    assertEquals(Seq(101), zkClient.getReplicasForPartition(tpA0))
-    assertEquals(Seq(102), zkClient.getReplicasForPartition(tpA1))
-    assertEquals(Seq(102), zkClient.getReplicasForPartition(tpB0))
-  }
-
-  @Test
-  def shouldBeAbleToCancelThroughApi(): Unit = {
-    startBrokers(Seq(100, 101, 102))
-    adminClient = createAdminClient(servers)
-    createTopic(zkClient, topicName, Map(tp0.partition() -> Seq(100, 101)), servers = servers)
-    // Throttle to ensure we minimize race conditions and test flakiness
-    throttle(Seq(topicName), throttleSettingForSeconds(10), Set(tp0))
-
-    // move to [102, 101]
-    adminClient.alterPartitionReassignments(
-      Map(reassignmentEntry(tp0, Seq(102, 101))).asJava
-    ).all().get()
-    val apiReassignmentsInProgress = adminClient.listPartitionReassignments().reassignments().get()
-    val tpReassignment = apiReassignmentsInProgress.get(tp0)
-    assertIsReassigning(from = Seq(100, 101), to = Seq(101, 102), tpReassignment)
-
-    adminClient.alterPartitionReassignments(
-      Map(cancelReassignmentEntry(tp0)).asJava
-    ).all().get()
-
-    TestUtils.resetBrokersThrottle(adminClient, brokerIds)
-    waitForAllReassignmentsToComplete()
-    assertEquals(Seq(100, 101), zkClient.getReplicasForPartition(tp0).sorted) // revert ordering is not guaranteed
-  }
-
-  @Test
-  def shouldBeAbleToCancelZkTriggeredReassignmentThroughApi(): Unit = {
-    startBrokers(Seq(100, 101))
-    adminClient = createAdminClient(servers)
-    createTopic(zkClient, topicName,
-      Map(tp0.partition() -> Seq(100),
-          tp1.partition() -> Seq(100)),
-      servers = servers)
-
-    // Throttle to avoid race conditions
-    throttle(Seq(topicName), throttleSettingForSeconds(10), Set(tp0, tp1))
-
-    val move = Map(
-      tp0 -> Seq(101),
-      tp1 -> Seq(101)
-    )
-    zkClient.setOrCreatePartitionReassignment(move, ZkVersion.MatchAnyVersion)
-    waitUntilTrue(() => {
-      !adminClient.listPartitionReassignments().reassignments().get().isEmpty
-    }, "Controller should have picked up on znode creation", 1000)
-    var reassignmentIsOngoing = adminClient.listPartitionReassignments().reassignments().get().size() > 0
-    assertTrue(reassignmentIsOngoing)
-
-    adminClient.alterPartitionReassignments(
-      Map(cancelReassignmentEntry(tp0), cancelReassignmentEntry(tp1)).asJava
-    ).all().get()
-
-    TestUtils.resetBrokersThrottle(adminClient, brokerIds)
-    waitForZkReassignmentToComplete()
-    reassignmentIsOngoing = adminClient.listPartitionReassignments().reassignments().get().size() > 0
-    assertFalse(reassignmentIsOngoing)
-    assertEquals(Seq(100), zkClient.getReplicasForPartition(tp0))
-    assertEquals(Seq(100), zkClient.getReplicasForPartition(tp1))
-  }
-
-  /**
-    * Cancel and set reassignments in the same API call.
-    * Even though one cancellation is invalid, ensure the other entries in the request pass
-    */
-  @Test
-  def testCancelAndSetSomeReassignments(): Unit = {
-    startBrokers(Seq(100, 101, 102))
-    adminClient = createAdminClient(servers)
-    val tp2 = new TopicPartition(topicName, 2)
-    val tp3 = new TopicPartition(topicName, 3)
-
-    createTopic(zkClient, topicName,
-      Map(tp0.partition() -> Seq(100), tp1.partition() -> Seq(100), tp2.partition() -> Seq(100), tp3.partition() -> Seq(100)),
-      servers = servers)
-
-    // Throttle to avoid race conditions
-    throttle(Seq(topicName), throttleSettingForSeconds(10), Set(tp0, tp1, tp2, tp3))
-
-    // API reassignment to 101 for tp0 and tp1
-    adminClient.alterPartitionReassignments(
-      Map(reassignmentEntry(tp0, Seq(101)), reassignmentEntry(tp1, Seq(101))).asJava
-    ).all().get()
-
-    // cancel tp0, reassign tp1 to 102 (override), assign tp2 to 101 (new reassignment) and cancel tp3 (it is not moving)
-    val alterResults = adminClient.alterPartitionReassignments(
-      Map(cancelReassignmentEntry(tp0), reassignmentEntry(tp1, Seq(102)),
-          reassignmentEntry(tp2, Seq(101)), cancelReassignmentEntry(tp3)).asJava
-    ).values()
-    alterResults.get(tp0).get()
-    alterResults.get(tp1).get()
-    alterResults.get(tp2).get()
-    try {
-      alterResults.get(tp3).get()
-    } catch {
-      case exception: Exception =>
-        assertEquals(exception.getCause.getClass, classOf[NoReassignmentInProgressException])
-    }
-
-    TestUtils.resetBrokersThrottle(adminClient, brokerIds)
-    waitForAllReassignmentsToComplete()
-    assertEquals(Seq(100), zkClient.getReplicasForPartition(tp0))
-    assertEquals(Seq(102), zkClient.getReplicasForPartition(tp1))
-    assertEquals(Seq(101), zkClient.getReplicasForPartition(tp2))
-    assertEquals(Seq(100), zkClient.getReplicasForPartition(tp3))
-  }
-
-  /**
-    * Three different Alter Reassignment calls should all create reassignments
-    */
-  @Test
-  def shouldBeAbleToIncrementallyStackDifferentReassignments(): Unit = {
-    startBrokers(Seq(100, 101))
-    adminClient = createAdminClient(servers)
-    val tpA0 = new TopicPartition("A", 0)
-    val tpA1 = new TopicPartition("A", 1)
-    val tpB0 = new TopicPartition("B", 0)
-
-    createTopic(zkClient, "A",
-      Map(tpA0.partition() -> Seq(100),
-          tpA1.partition() -> Seq(100)),
-      servers = servers)
-    createTopic(zkClient, "B",
-      Map(tpB0.partition() -> Seq(100)),
-      servers = servers)
-
-    // Throttle to avoid race conditions
-    throttle(Seq("A", "B"), throttleSettingForSeconds(10), Set(tpA0, tpA1, tpB0))
-
-    adminClient.alterPartitionReassignments(Map(reassignmentEntry(tpA0, Seq(101))).asJava).all().get()
-    val apiReassignmentsInProgress1 = adminClient.listPartitionReassignments().reassignments().get()
-    assertEquals(1, apiReassignmentsInProgress1.size())
-    assertIsReassigning(
-      from = Seq(100), to = Seq(101),
-      apiReassignmentsInProgress1.get(tpA0)
-    )
-
-    adminClient.alterPartitionReassignments(Map(reassignmentEntry(tpA1, Seq(101))).asJava).all().get()
-    val apiReassignmentsInProgress2 = adminClient.listPartitionReassignments().reassignments().get()
-    assertEquals(2, apiReassignmentsInProgress2.size())
-    assertIsReassigning(from = Seq(100), to = Seq(101), apiReassignmentsInProgress2.get(tpA0))
-    assertIsReassigning(
-      from = Seq(100), to = Seq(101),
-      apiReassignmentsInProgress2.get(tpA1)
-    )
-
-    adminClient.alterPartitionReassignments(Map(reassignmentEntry(tpB0, Seq(101))).asJava).all().get()
-    val apiReassignmentsInProgress3 = adminClient.listPartitionReassignments().reassignments().get()
-    assertEquals(s"${apiReassignmentsInProgress3}", 3, apiReassignmentsInProgress3.size())
-    assertIsReassigning(from = Seq(100), to = Seq(101), apiReassignmentsInProgress3.get(tpA0))
-    assertIsReassigning(from = Seq(100), to = Seq(101), apiReassignmentsInProgress3.get(tpA1))
-    assertIsReassigning(
-      from = Seq(100), to = Seq(101),
-      apiReassignmentsInProgress3.get(tpB0)
-    )
-
-    TestUtils.resetBrokersThrottle(adminClient, brokerIds)
-    waitForAllReassignmentsToComplete()
-    assertEquals(Seq(101), zkClient.getReplicasForPartition(tpA0))
-    assertEquals(Seq(101), zkClient.getReplicasForPartition(tpA1))
-    assertEquals(Seq(101), zkClient.getReplicasForPartition(tpB0))
-  }
-
-  /**
-   * Verifies that partitions can be created for topics not in reassignment and for the topics that are in reassignment
-   * an ReassignmentInProgressException should be thrown. The test creates two topics `topicName` and `otherTopicName`,
-   * the `topicName` topic undergoes partition reassignment and the test validates that during reassignment createPartitions
-   * call throws ReassignmentInProgressException `topicName` topic and for topic `otherTopicName` which is not being reassigned
-   * successfully creates partitions. Further validates that after the reassignment is complete for topic `topicName`
-   * createPartition is successful for that topic.
-   */
-  @Test
-  def shouldCreatePartitionsForTopicNotInReassignment(): Unit = {
-    startBrokers(Seq(100, 101))
-    val otherTopicName = "anyTopic"
-    val otp0 = new TopicPartition(otherTopicName, 0)
-    val otp1 = new TopicPartition(otherTopicName, 1)
-    adminClient = createAdminClient(servers)
-    createTopic(zkClient, topicName,
-      Map(otp0.partition() -> Seq(100),
-          otp1.partition() -> Seq(100)),
-      servers = servers)
-    createTopic(zkClient, otherTopicName,
-      Map(tp0.partition() -> Seq(100),
-          tp1.partition() -> Seq(100)),
-      servers = servers)
-
-    // Throttle to avoid race conditions
-    throttle(Seq(topicName), throttleSettingForSeconds(10), Set(tp0, tp1))
-
-    // Alter `topicName` partition reassignment
-    adminClient.alterPartitionReassignments(
-      Map(reassignmentEntry(tp0, Seq(101)),
-        reassignmentEntry(tp1, Seq(101))).asJava
-    ).all().get()
-    waitUntilTrue(() => {
-      !adminClient.listPartitionReassignments().reassignments().get().isEmpty
-    }, "Controller should have picked up reassignment", 1000)
-
-    def testCreatePartitions(topicName: String, isTopicBeingReassigned: Boolean): Unit = {
-      if (isTopicBeingReassigned)
-        assertTrue("createPartitions for topic under reassignment should throw an exception", intercept[ExecutionException](
-          adminClient.createPartitions(Map(topicName -> NewPartitions.increaseTo(4)).asJava).values.get(topicName).get()).
-          getCause.isInstanceOf[ReassignmentInProgressException])
-      else
-        adminClient.createPartitions(Map(topicName -> NewPartitions.increaseTo(4)).asJava).values.get(topicName).get()
-    }
-
-    // Test case: createPartitions throws ReassignmentInProgressException Topics with partitions in reassignment.
-    testCreatePartitions(topicName, true)
-    // Test case: createPartitions is successful for Topics with partitions NOT in reassignment.
-    testCreatePartitions(otherTopicName, false)
-
-    // complete reassignment
-    TestUtils.resetBrokersThrottle(adminClient, brokerIds)
-    waitForAllReassignmentsToComplete()
-
-    // Test case: createPartitions is successful for Topics with partitions after reassignment has completed.
-    testCreatePartitions(topicName, false)
-  }
-
-  /**
-    * Asserts that a replica is being reassigned from the given replicas to the target replicas
-    */
-  def assertIsReassigning(from: Seq[Int], to: Seq[Int], reassignment: PartitionReassignment): Unit = {
-    assertReplicas((from ++ to).distinct, reassignment.replicas())
-    assertReplicas(to.filterNot(from.contains(_)), reassignment.addingReplicas())
-    assertReplicas(from.filterNot(to.contains(_)), reassignment.removingReplicas())
-  }
-
-  /**
-   * Asserts that a topic's reassignments completed and span across the expected replicas
-   */
-  def assertMoveForTopicOccurred(expectedReplicas: Seq[Int],
-                                 partitionAssignments: Map[Int, ReplicaAssignment]): Unit = {
-    assertEquals(expectedReplicas, partitionAssignments.values.flatMap(_.replicas).toSeq.distinct.sorted)
-    assertTrue(partitionAssignments.values.flatMap(_.addingReplicas).isEmpty)
-    assertTrue(partitionAssignments.values.flatMap(_.removingReplicas).isEmpty)
-  }
-
-  /**
-   * Asserts that a partition moved to the exact expected replicas in the specific order
-   */
-  def assertMoveForPartitionOccurred(expectedReplicas: Seq[Int],
-                                     partitionAssignment: ReplicaAssignment): Unit = {
-    assertEquals(expectedReplicas, partitionAssignment.replicas)
-    assertTrue(partitionAssignment.addingReplicas.isEmpty)
-    assertTrue(partitionAssignment.removingReplicas.isEmpty)
-  }
-
-  /**
-   * Asserts that two replica sets are equal, ignoring ordering
-   */
-  def assertReplicas(expectedReplicas: Seq[Int], receivedReplicas: java.util.List[Integer]): Unit = {
-    assertEquals(expectedReplicas.sorted, receivedReplicas.asScala.map(_.toInt).sorted)
-  }
-
-  def reassignmentEntry(tp: TopicPartition, replicas: Seq[Int]): (TopicPartition, java.util.Optional[NewPartitionReassignment]) =
-    tp -> Optional.of(new NewPartitionReassignment((replicas.map(_.asInstanceOf[Integer]).asJava)))
-
-  def cancelReassignmentEntry(tp: TopicPartition): (TopicPartition, java.util.Optional[NewPartitionReassignment]) =
-    tp -> java.util.Optional.empty()
-
-  def waitForZkReassignmentToComplete(pause: Long = 100L): Unit = {
-    waitUntilTrue(() => !zkClient.reassignPartitionsInProgress,
-      s"Znode ${ReassignPartitionsZNode.path} wasn't deleted", pause = pause)
-  }
-
-  def awaitReassignmentInProgress(topicPartition: TopicPartition): Unit = {
-    waitUntilTrue(() => isAssignmentInProgress(topicPartition),
-      "Timed out waiting for expected reassignment to begin")
-  }
-
-  def isAssignmentInProgress(topicPartition: TopicPartition): Boolean = {
-    val reassignments = adminClient.listPartitionReassignments().reassignments().get()
-    reassignments.asScala.get(topicPartition).isDefined
-  }
-
-  def waitForAllReassignmentsToComplete(pause: Long = 100L): Unit = {
-    waitUntilTrue(() => adminClient.listPartitionReassignments().reassignments().get().isEmpty,
-      s"There still are ongoing reassignments", pause = pause)
-  }
-
-  def generateAssignmentJson(topic: String*): String = {
-    val topicStr = topic.map { t => "{\"topic\": \"" + t + "\"}" }.mkString(",")
-    s"""{"topics": [$topicStr],"version":1}"""
-  }
-
-  def executeAssignmentJson(partitions: Seq[PartitionAssignmentJson]): String =
-    s"""{"version":1,"partitions":[${partitions.map(_.toJson).mkString(",")}]}"""
-
-  case class PartitionAssignmentJson(topicPartition: TopicPartition, replicas: Seq[Int],
-                                     logDirectories: Option[Seq[String]] = None) {
-    def toJson: String = {
-      val logDirsSuffix = logDirectories match {
-        case Some(dirs) => s""","log_dirs":[${dirs.map("\"" + _ + "\"").mkString(",")}]"""
-        case None => ""
-      }
-      s"""{"topic":"${topicPartition.topic()}","partition":${topicPartition.partition()}""" +
-        s""","replicas":[${replicas.mkString(",")}]""" +
-        s"$logDirsSuffix}"
-    }
-  }
-
-  case class ThrottleSetting(throttleBytes: String, numMessages: Int, messageSizeBytes: Int)
-
-  def throttleSettingForSeconds(secondsDuration: Int): ThrottleSetting = {
-    val throttle = 1000 * 1000 // 1 MB/s throttle
-    val messageSize = 100 * 100 // 0.01 MB message size
-    val messagesPerSecond = throttle / messageSize
-    ThrottleSetting(throttle.toString, messagesPerSecond * secondsDuration, messageSize)
-  }
-
-  def throttle(topics: Seq[String], throttle: ThrottleSetting, partitions: Set[TopicPartition]): Unit = {
-    val messagesPerTopic = throttle.numMessages / topics.size
-    for (topic <- topics) {
-      produceMessages(topic, numMessages = messagesPerTopic, acks = 0, valueLength = throttle.messageSizeBytes)
-    }
-    TestUtils.setReplicationThrottleForPartitions(adminClient, brokerIds, partitions, throttle.throttleBytes.toInt)
-  }
-
-  private def produceMessages(topic: String, numMessages: Int, acks: Int, valueLength: Int): Unit = {
-    val records = (0 until numMessages).map(_ => new ProducerRecord[Array[Byte], Array[Byte]](topic,
-      new Array[Byte](valueLength)))
-    TestUtils.produceMessages(servers, records, acks)
-  }
-}
diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandArgsTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandArgsTest.scala
index 7037f29..d15ad82 100644
--- a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandArgsTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandArgsTest.scala
@@ -18,9 +18,12 @@ package kafka.admin
 
 import kafka.utils.Exit
 import org.junit.Assert._
-import org.junit.{After, Before, Test}
+import org.junit.rules.Timeout
+import org.junit.{After, Before, Rule, Test}
 
 class ReassignPartitionsCommandArgsTest {
+  @Rule
+  def globalTimeout: Timeout = Timeout.millis(60000)
 
   @Before
   def setUp(): Unit = {
@@ -32,14 +35,11 @@ class ReassignPartitionsCommandArgsTest {
     Exit.resetExitProcedure()
   }
 
-  /**
-    * HAPPY PATH
-    */
-
+  ///// Test valid argument parsing
   @Test
   def shouldCorrectlyParseValidMinimumGenerateOptions(): Unit = {
     val args = Array(
-      "--zookeeper", "localhost:1234",
+      "--bootstrap-server", "localhost:1234",
       "--generate",
       "--broker-list", "101,102",
       "--topics-to-move-json-file", "myfile.json")
@@ -49,6 +49,15 @@ class ReassignPartitionsCommandArgsTest {
   @Test
   def shouldCorrectlyParseValidMinimumExecuteOptions(): Unit = {
     val args = Array(
+      "--bootstrap-server", "localhost:1234",
+      "--execute",
+      "--reassignment-json-file", "myfile.json")
+    ReassignPartitionsCommand.validateAndParseArgs(args)
+  }
+
+  @Test
+  def shouldCorrectlyParseValidMinimumLegacyExecuteOptions(): Unit = {
+    val args = Array(
       "--zookeeper", "localhost:1234",
       "--execute",
       "--reassignment-json-file", "myfile.json")
@@ -58,6 +67,15 @@ class ReassignPartitionsCommandArgsTest {
   @Test
   def shouldCorrectlyParseValidMinimumVerifyOptions(): Unit = {
     val args = Array(
+      "--bootstrap-server", "localhost:1234",
+      "--verify",
+      "--reassignment-json-file", "myfile.json")
+    ReassignPartitionsCommand.validateAndParseArgs(args)
+  }
+
+  @Test
+  def shouldCorrectlyParseValidMinimumLegacyVerifyOptions(): Unit = {
+    val args = Array(
       "--zookeeper", "localhost:1234",
       "--verify",
       "--reassignment-json-file", "myfile.json")
@@ -67,7 +85,7 @@ class ReassignPartitionsCommandArgsTest {
   @Test
   def shouldAllowThrottleOptionOnExecute(): Unit = {
     val args = Array(
-      "--zookeeper", "localhost:1234",
+      "--bootstrap-server", "localhost:1234",
       "--execute",
       "--throttle", "100",
       "--reassignment-json-file", "myfile.json")
@@ -77,7 +95,7 @@ class ReassignPartitionsCommandArgsTest {
   @Test
   def shouldUseDefaultsIfEnabled(): Unit = {
     val args = Array(
-      "--zookeeper", "localhost:1234",
+      "--bootstrap-server", "localhost:1234",
       "--execute",
       "--reassignment-json-file", "myfile.json")
     val opts = ReassignPartitionsCommand.validateAndParseArgs(args)
@@ -85,11 +103,26 @@ class ReassignPartitionsCommandArgsTest {
     assertEquals(-1L, opts.options.valueOf(opts.interBrokerThrottleOpt))
   }
 
-  /**
-    * NO ARGS
-    */
+  @Test
+  def testList(): Unit = {
+    val args = Array(
+      "--list",
+      "--bootstrap-server", "localhost:1234")
+    ReassignPartitionsCommand.validateAndParseArgs(args)
+  }
 
   @Test
+  def testCancelWithPreserveThrottlesOption(): Unit = {
+    val args = Array(
+      "--cancel",
+      "--bootstrap-server", "localhost:1234",
+      "--reassignment-json-file", "myfile.json",
+      "--preserve-throttles")
+    ReassignPartitionsCommand.validateAndParseArgs(args)
+  }
+
+  ///// Test handling missing or invalid actions
+  @Test
   def shouldFailIfNoArgs(): Unit = {
     val args: Array[String]= Array()
     shouldFailWith(ReassignPartitionsCommand.helpText, args)
@@ -98,104 +131,125 @@ class ReassignPartitionsCommandArgsTest {
   @Test
   def shouldFailIfBlankArg(): Unit = {
     val args = Array(" ")
-    shouldFailWith("Command must include exactly one action: --generate, --execute or --verify", args)
+    shouldFailWith("Command must include exactly one action", args)
   }
 
-  /**
-    * UNHAPPY PATH: EXECUTE ACTION
-    */
+  @Test
+  def shouldFailIfMultipleActions(): Unit = {
+    val args = Array(
+      "--bootstrap-server", "localhost:1234",
+      "--execute",
+      "--verify",
+      "--reassignment-json-file", "myfile.json"
+    )
+    shouldFailWith("Command must include exactly one action", args)
+  }
 
+  ///// Test --execute
   @Test
   def shouldNotAllowExecuteWithTopicsOption(): Unit = {
     val args = Array(
-      "--zookeeper", "localhost:1234",
+      "--bootstrap-server", "localhost:1234",
       "--execute",
       "--reassignment-json-file", "myfile.json",
       "--topics-to-move-json-file", "myfile.json")
-    shouldFailWith("Option \"[execute]\" can't be used with option \"[topics-to-move-json-file]\"", args)
+    shouldFailWith("Option \"[topics-to-move-json-file]\" can't be used with action \"[execute]\"", args)
   }
 
   @Test
-  def shouldNotAllowExecuteWithBrokers(): Unit = {
+  def shouldNotAllowExecuteWithBrokerList(): Unit = {
     val args = Array(
-      "--zookeeper", "localhost:1234",
+      "--bootstrap-server", "localhost:1234",
       "--execute",
       "--reassignment-json-file", "myfile.json",
       "--broker-list", "101,102"
     )
-    shouldFailWith("Option \"[execute]\" can't be used with option \"[broker-list]\"", args)
+    shouldFailWith("Option \"[broker-list]\" can't be used with action \"[execute]\"", args)
   }
 
   @Test
   def shouldNotAllowExecuteWithoutReassignmentOption(): Unit = {
     val args = Array(
-      "--zookeeper", "localhost:1234",
+      "--bootstrap-server", "localhost:1234",
       "--execute")
-    shouldFailWith("If --execute option is used, command must include --reassignment-json-file that was output during the --generate option", args)
+    shouldFailWith("Missing required argument \"[reassignment-json-file]\"", args)
   }
 
-  /**
-    * UNHAPPY PATH: GENERATE ACTION
-    */
+  @Test
+  def testMissingBootstrapServerArgumentForExecute(): Unit = {
+    val args = Array(
+      "--execute")
+    shouldFailWith("Please specify --bootstrap-server", args)
+  }
 
+  ///// Test --generate
   @Test
   def shouldNotAllowGenerateWithoutBrokersAndTopicsOptions(): Unit = {
     val args = Array(
-      "--zookeeper", "localhost:1234",
+      "--bootstrap-server", "localhost:1234",
       "--generate")
-    shouldFailWith("If --generate option is used, command must include both --topics-to-move-json-file and --broker-list options", args)
+    shouldFailWith("Missing required argument \"[topics-to-move-json-file]\"", args)
   }
 
   @Test
   def shouldNotAllowGenerateWithoutBrokersOption(): Unit = {
     val args = Array(
-      "--zookeeper", "localhost:1234",
+      "--bootstrap-server", "localhost:1234",
       "--topics-to-move-json-file", "myfile.json",
       "--generate")
-    shouldFailWith("If --generate option is used, command must include both --topics-to-move-json-file and --broker-list options", args)
+    shouldFailWith("Missing required argument \"[broker-list]\"", args)
   }
 
   @Test
   def shouldNotAllowGenerateWithoutTopicsOption(): Unit = {
     val args = Array(
-      "--zookeeper", "localhost:1234",
+      "--bootstrap-server", "localhost:1234",
       "--broker-list", "101,102",
       "--generate")
-    shouldFailWith("If --generate option is used, command must include both --topics-to-move-json-file and --broker-list options", args)
+    shouldFailWith("Missing required argument \"[topics-to-move-json-file]\"", args)
   }
 
   @Test
   def shouldNotAllowGenerateWithThrottleOption(): Unit = {
     val args = Array(
-      "--zookeeper", "localhost:1234",
+      "--bootstrap-server", "localhost:1234",
       "--generate",
       "--broker-list", "101,102",
       "--throttle", "100",
       "--topics-to-move-json-file", "myfile.json")
-    shouldFailWith("Option \"[generate]\" can't be used with option \"[throttle]\"", args)
+    shouldFailWith("Option \"[throttle]\" can't be used with action \"[generate]\"", args)
   }
 
   @Test
   def shouldNotAllowGenerateWithReassignmentOption(): Unit = {
     val args = Array(
-      "--zookeeper", "localhost:1234",
+      "--bootstrap-server", "localhost:1234",
       "--generate",
       "--broker-list", "101,102",
       "--topics-to-move-json-file", "myfile.json",
       "--reassignment-json-file", "myfile.json")
-    shouldFailWith("Option \"[generate]\" can't be used with option \"[reassignment-json-file]\"", args)
+    shouldFailWith("Option \"[reassignment-json-file]\" can't be used with action \"[generate]\"", args)
   }
 
-  /**
-    * UNHAPPY PATH: VERIFY ACTION
-    */
+  @Test
+  def testInvalidCommandConfigArgumentForLegacyGenerate(): Unit = {
+    val args = Array(
+      "--zookeeper", "localhost:1234",
+      "--generate",
+      "--broker-list", "101,102",
+      "--topics-to-move-json-file", "myfile.json",
+      "--command-config", "/tmp/command-config.properties"
+    )
+    shouldFailWith("You must specify --bootstrap-server when using \"[command-config]\"", args)
+  }
 
+  ///// Test --verify
   @Test
   def shouldNotAllowVerifyWithoutReassignmentOption(): Unit = {
     val args = Array(
-      "--zookeeper", "localhost:1234",
+      "--bootstrap-server", "localhost:1234",
       "--verify")
-    shouldFailWith("If --verify option is used, command must include --reassignment-json-file that was used during the --execute option", args)
+    shouldFailWith("Missing required argument \"[reassignment-json-file]\"", args)
   }
 
   @Test
@@ -205,7 +259,7 @@ class ReassignPartitionsCommandArgsTest {
       "--verify",
       "--broker-list", "100,101",
       "--reassignment-json-file", "myfile.json")
-    shouldFailWith("Option \"[verify]\" can't be used with option \"[broker-list]\"", args)
+    shouldFailWith("Option \"[broker-list]\" can't be used with action \"[verify]\"", args)
   }
 
   @Test
@@ -215,7 +269,7 @@ class ReassignPartitionsCommandArgsTest {
       "--verify",
       "--throttle", "100",
       "--reassignment-json-file", "myfile.json")
-    shouldFailWith("Option \"[verify]\" can't be used with option \"[throttle]\"", args)
+    shouldFailWith("Option \"[throttle]\" can't be used with action \"[verify]\"", args)
   }
 
   @Test
@@ -225,7 +279,7 @@ class ReassignPartitionsCommandArgsTest {
       "--verify",
       "--reassignment-json-file", "myfile.json",
       "--topics-to-move-json-file", "myfile.json")
-    shouldFailWith("Option \"[verify]\" can't be used with option \"[topics-to-move-json-file]\"", args)
+    shouldFailWith("Option \"[topics-to-move-json-file]\" can't be used with action \"[verify]\"", args)
   }
 
   def shouldFailWith(msg: String, args: Array[String]): Unit = {
@@ -236,4 +290,30 @@ class ReassignPartitionsCommandArgsTest {
       case e: Exception => assertTrue(s"Expected exception with message:\n[$msg]\nbut was\n[${e.getMessage}]", e.getMessage.startsWith(msg))
     }
   }
+
+  ///// Test --cancel
+  @Test
+  def shouldNotAllowCancelWithoutBootstrapServerOption(): Unit = {
+    val args = Array(
+      "--cancel")
+    shouldFailWith("Please specify --bootstrap-server", args)
+  }
+
+  @Test
+  def shouldNotAllowCancelWithoutReassignmentJsonFile(): Unit = {
+    val args = Array(
+      "--cancel",
+      "--bootstrap-server", "localhost:1234",
+      "--preserve-throttles")
+    shouldFailWith("Missing required argument \"[reassignment-json-file]\"", args)
+  }
+
+  ///// Test --list
+  @Test
+  def shouldNotAllowZooKeeperWithListOption(): Unit = {
+    val args = Array(
+      "--list",
+      "--zookeeper", "localhost:1234")
+    shouldFailWith("Option \"[zookeeper]\" can't be used with action \"[list]\"", args)
+  }
 }
diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala
deleted file mode 100644
index 790e3d8..0000000
--- a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala
+++ /dev/null
@@ -1,582 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package kafka.admin
-
-import java.io.File
-import java.util.Properties
-
-import kafka.admin.ReassignPartitionsCommand.Throttle
-import kafka.log.LogConfig
-import kafka.log.LogConfig._
-import kafka.server.{ConfigType, DynamicConfig, KafkaConfig, KafkaServer}
-import kafka.utils.CoreUtils._
-import kafka.utils.TestUtils._
-import kafka.utils.{CoreUtils, Logging, TestUtils}
-import kafka.zk.{AdminZkClient, KafkaZkClient, ZooKeeperTestHarness}
-import org.easymock.EasyMock._
-import org.easymock.{Capture, CaptureType, EasyMock}
-import org.junit.{After, Before, Test}
-import org.junit.Assert.{assertEquals, assertFalse, assertNull, assertTrue}
-import org.scalatest.Assertions.fail
-
-import scala.collection.JavaConverters._
-import org.apache.kafka.common.TopicPartition
-
-import scala.collection.mutable
-import scala.collection.Seq
-
-class ReassignPartitionsCommandTest extends ZooKeeperTestHarness with Logging {
-  var servers: Seq[KafkaServer] = Seq()
-  var calls = 0
-
-  @After
-  override def tearDown(): Unit = {
-    TestUtils.shutdownServers(servers)
-    super.tearDown()
-  }
-
-  @Test
-  def shouldFindMovingReplicas(): Unit = {
-    val control = new TopicPartition("topic1", 1) -> Seq(100, 102)
-    val assigner = new ReassignPartitionsCommand(null, null, null, null, null)
-
-    //Given partition 0 moves from broker 100 -> 102. Partition 1 does not move.
-    val existing = Map(new TopicPartition("topic1", 0) -> Seq(100, 101), control)
-    val proposed = Map(new TopicPartition("topic1", 0) -> Seq(101, 102), control)
-
-    class TestAdminZkClient(val zkClient: KafkaZkClient) extends AdminZkClient(zkClient) {
-      override def changeTopicConfig(topic: String, configChange: Properties): Unit = {
-        assertEquals(Set("0:102"), toReplicaSet(configChange.get(FollowerReplicationThrottledReplicasProp))) //Should only be follower-throttle the moving replica
-        assertEquals(Set("0:100","0:101"), toReplicaSet(configChange.get(LeaderReplicationThrottledReplicasProp))) //Should leader-throttle all existing (pre move) replicas
-        calls += 1
-      }
-      override def fetchEntityConfig(entityType: String, entityName: String): Properties = {new Properties}
-    }
-
-    val admin = new TestAdminZkClient(zkClient)
-    assigner.assignThrottledReplicas(existing, proposed, admin)
-    assertEquals(1, calls)
-  }
-
-  @Test
-  def shouldFindMovingReplicasWhenProposedIsSubsetOfExisting(): Unit = {
-    val assigner = new ReassignPartitionsCommand(null, null, null, null, null)
-
-    //Given we have more existing partitions than we are proposing
-    val existingSuperset = Map(
-      new TopicPartition("topic1", 0) -> Seq(100, 101),
-      new TopicPartition("topic1", 1) -> Seq(100, 102),
-      new TopicPartition("topic1", 2) -> Seq(100, 101),
-      new TopicPartition("topic2", 0) -> Seq(100, 101, 102),
-      new TopicPartition("topic3", 0) -> Seq(100, 101, 102)
-    )
-    val proposedSubset = Map(
-      new TopicPartition("topic1", 0) -> Seq(101, 102),
-      new TopicPartition("topic1", 1) -> Seq(102),
-      new TopicPartition("topic1", 2) -> Seq(100, 101, 102)
-    )
-
-    class TestAdminZkClient(val zkClient: KafkaZkClient) extends AdminZkClient(zkClient) {
-      override def changeTopicConfig(topic: String, configChange: Properties): Unit = {
-        assertEquals(Set("0:102","2:102"), toReplicaSet(configChange.get(FollowerReplicationThrottledReplicasProp)))
-        assertEquals(Set("0:100","0:101","2:100","2:101"), toReplicaSet(configChange.get(LeaderReplicationThrottledReplicasProp)))
-        assertEquals("topic1", topic)
-        calls += 1
-      }
-
-      override def fetchEntityConfig(entityType: String, entityName: String): Properties = {new Properties}
-    }
-
-    val admin = new TestAdminZkClient(zkClient)
-    //Then replicas should assign correctly (based on the proposed map)
-    assigner.assignThrottledReplicas(existingSuperset, proposedSubset, admin)
-    assertEquals(1, calls)
-  }
-
-  @Test
-  def shouldFindMovingReplicasMultiplePartitions(): Unit = {
-    val control = new TopicPartition("topic1", 2) -> Seq(100, 102)
-    val assigner = new ReassignPartitionsCommand(null, null, null, null, null)
-
-    //Given partitions 0 & 1 moves from broker 100 -> 102. Partition 2 does not move.
-    val existing = Map(new TopicPartition("topic1", 0) -> Seq(100, 101), new TopicPartition("topic1", 1) -> Seq(100, 101), control)
-    val proposed = Map(new TopicPartition("topic1", 0) -> Seq(101, 102), new TopicPartition("topic1", 1) -> Seq(101, 102), control)
-
-    class TestAdminZkClient(val zkClient: KafkaZkClient) extends AdminZkClient(zkClient) {
-      override def changeTopicConfig(topic: String, configChange: Properties): Unit = {
-        assertEquals(Set("0:102","1:102"), toReplicaSet(configChange.get(FollowerReplicationThrottledReplicasProp))) //Should only be follower-throttle the moving replica
-        assertEquals(Set("0:100","0:101","1:100","1:101"), toReplicaSet(configChange.get(LeaderReplicationThrottledReplicasProp))) //Should leader-throttle all existing (pre move) replicas
-        calls += 1
-      }
-
-      override def fetchEntityConfig(entityType: String, entityName: String): Properties = {new Properties}
-    }
-
-    val admin = new TestAdminZkClient(zkClient)
-    //When
-    assigner.assignThrottledReplicas(existing, proposed, admin)
-    assertEquals(1, calls)
-  }
-
-  @Test
-  def shouldFindMovingReplicasMultipleTopics(): Unit = {
-    val control = new TopicPartition("topic1", 1) -> Seq(100, 102)
-    val assigner = new ReassignPartitionsCommand(null, null, null, null, null)
-
-    //Given topics 1 -> move from broker 100 -> 102, topics 2 -> move from broker 101 -> 100
-    val existing = Map(new TopicPartition("topic1", 0) -> Seq(100, 101), new TopicPartition("topic2", 0) -> Seq(101, 102), control)
-    val proposed = Map(new TopicPartition("topic1", 0) -> Seq(101, 102), new TopicPartition("topic2", 0) -> Seq(100, 102), control)
-
-    //Then
-    class TestAdminZkClient(val zkClient: KafkaZkClient) extends AdminZkClient(zkClient) {
-      override def changeTopicConfig(topic: String, configChange: Properties): Unit = {
-        topic match {
-          case "topic1" =>
-            assertEquals(Set("0:100","0:101"), toReplicaSet(configChange.get(LeaderReplicationThrottledReplicasProp)))
-            assertEquals(Set("0:102"), toReplicaSet(configChange.get(FollowerReplicationThrottledReplicasProp)))
-          case "topic2" =>
-            assertEquals(Set("0:101","0:102"), toReplicaSet(configChange.get(LeaderReplicationThrottledReplicasProp)))
-            assertEquals(Set("0:100"), toReplicaSet(configChange.get(FollowerReplicationThrottledReplicasProp)))
-          case _ => fail(s"Unexpected topic $topic")
-        }
-        calls += 1
-      }
-      override def fetchEntityConfig(entityType: String, entityName: String): Properties = {new Properties}
-    }
-
-    val admin = new TestAdminZkClient(zkClient)
-    //When
-    assigner.assignThrottledReplicas(existing, proposed, admin)
-    assertEquals(2, calls)
-  }
-
-  @Test
-  def shouldFindMovingReplicasMultipleTopicsAndPartitions(): Unit = {
-    val assigner = new ReassignPartitionsCommand(null, null, null, null, null)
-
-    //Given
-    val existing = Map(
-      new TopicPartition("topic1", 0) -> Seq(100, 101),
-      new TopicPartition("topic1", 1) -> Seq(100, 101),
-      new TopicPartition("topic2", 0) -> Seq(101, 102),
-      new TopicPartition("topic2", 1) -> Seq(101, 102)
-    )
-    val proposed = Map(
-      new TopicPartition("topic1", 0) -> Seq(101, 102), //moves to 102
-      new TopicPartition("topic1", 1) -> Seq(101, 102), //moves to 102
-      new TopicPartition("topic2", 0) -> Seq(100, 102), //moves to 100
-      new TopicPartition("topic2", 1) -> Seq(101, 100)  //moves to 100
-    )
-
-    //Then
-    class TestAdminZkClient(val zkClient: KafkaZkClient) extends AdminZkClient(zkClient) {
-      override def changeTopicConfig(topic: String, configChange: Properties): Unit = {
-        topic match {
-          case "topic1" =>
-            assertEquals(Set("0:102","1:102"), toReplicaSet(configChange.get(FollowerReplicationThrottledReplicasProp)))
-            assertEquals(Set("0:100","0:101","1:100","1:101"), toReplicaSet(configChange.get(LeaderReplicationThrottledReplicasProp)))
-          case "topic2" =>
-            assertEquals(Set("0:100","1:100"), toReplicaSet(configChange.get(FollowerReplicationThrottledReplicasProp)))
-            assertEquals(Set("0:101","0:102","1:101","1:102"), toReplicaSet(configChange.get(LeaderReplicationThrottledReplicasProp)))
-          case _ => fail(s"Unexpected topic $topic")
-        }
-        calls += 1
-      }
-
-      override def fetchEntityConfig(entityType: String, entityName: String): Properties = {new Properties}
-    }
-
-    val admin = new TestAdminZkClient(zkClient)
-
-    //When
-    assigner.assignThrottledReplicas(existing, proposed, admin)
-    assertEquals(2, calls)
-  }
-
-  @Test
-  def shouldFindTwoMovingReplicasInSamePartition(): Unit = {
-    val control = new TopicPartition("topic1", 1) -> Seq(100, 102)
-    val assigner = new ReassignPartitionsCommand(null, null, null, null, null)
-
-    //Given partition 0 has 2 moves from broker 102 -> 104 & 103 -> 105
-    val existing = Map(new TopicPartition("topic1", 0) -> Seq(100, 101, 102, 103), control)
-    val proposed = Map(new TopicPartition("topic1", 0) -> Seq(100, 101, 104, 105), control)
-
-    // Then
-    class TestAdminZkClient(val zkClient: KafkaZkClient) extends AdminZkClient(zkClient) {
-      override def changeTopicConfig(topic: String, configChange: Properties) = {
-        assertEquals(Set("0:104","0:105"), toReplicaSet(configChange.get(FollowerReplicationThrottledReplicasProp))) //Should only be follower-throttle the moving replicas
-        assertEquals(Set("0:100","0:101","0:102","0:103"), toReplicaSet(configChange.get(LeaderReplicationThrottledReplicasProp))) //Should leader-throttle all existing (pre move) replicas
-        calls += 1
-      }
-
-      override def fetchEntityConfig(entityType: String, entityName: String): Properties = {new Properties}
-    }
-
-    val admin = new TestAdminZkClient(zkClient)
-    //When
-    assigner.assignThrottledReplicas(existing, proposed, admin)
-    assertEquals(1, calls)
-  }
-
-  @Test
-  def shouldNotOverwriteEntityConfigsWhenUpdatingThrottledReplicas(): Unit = {
-    val control = new TopicPartition("topic1", 1) -> Seq(100, 102)
-    val assigner = new ReassignPartitionsCommand(null, null, null, null, null)
-    val existing = Map(new TopicPartition("topic1", 0) -> Seq(100, 101), control)
-    val proposed = Map(new TopicPartition("topic1", 0) -> Seq(101, 102), control)
-
-    //Given partition there are existing properties
-    val existingProperties = propsWith("some-key", "some-value")
-
-    //Then the dummy property should still be there
-    class TestAdminZkClient(val zkClient: KafkaZkClient) extends AdminZkClient(zkClient) {
-      override def changeTopicConfig(topic: String, configChange: Properties): Unit = {
-        assertEquals("some-value", configChange.getProperty("some-key"))
-        calls += 1
-      }
-
-      override def fetchEntityConfig(entityType: String, entityName: String): Properties = {
-        existingProperties
-      }
-    }
-
-    val admin = new TestAdminZkClient(zkClient)
-
-    //When
-    assigner.assignThrottledReplicas(existing, proposed, admin)
-    assertEquals(1, calls)
-  }
-
-  @Test
-  def shouldSetQuotaLimit(): Unit = {
-    //Given
-    val existing = Map(new TopicPartition("topic1", 0) -> Seq(100, 101))
-    val proposed = mutable.Map(new TopicPartition("topic1", 0) -> Seq(101, 102))
-
-    //Setup
-    val zk = stubZKClient(existing)
-    val admin: AdminZkClient = createMock(classOf[AdminZkClient])
-    val propsCapture: Capture[Properties] = newCapture(CaptureType.ALL)
-    val assigner = new ReassignPartitionsCommand(zk, None, proposed, Map.empty, admin)
-    expect(admin.fetchEntityConfig(anyString(), anyString())).andStubReturn(new Properties)
-    expect(admin.changeBrokerConfig(anyObject().asInstanceOf[List[Int]], capture(propsCapture))).anyTimes()
-    replay(admin)
-
-    //When
-    assigner.maybeLimit(Throttle(1000))
-
-    //Then
-    for (actual <- propsCapture.getValues.asScala) {
-      assertEquals("1000", actual.getProperty(DynamicConfig.Broker.LeaderReplicationThrottledRateProp))
-      assertEquals("1000", actual.getProperty(DynamicConfig.Broker.FollowerReplicationThrottledRateProp))
-    }
-    assertEquals(3, propsCapture.getValues.size) //3 brokers
-  }
-
-  @Test
-  def shouldUpdateQuotaLimit(): Unit = {
-    //Given
-    val existing = Map(new TopicPartition("topic1", 0) -> Seq(100, 101))
-    val proposed = mutable.Map(new TopicPartition("topic1", 0) -> Seq(101, 102))
-
-    //Setup
-    val zk = stubZKClient(existing)
-    val admin: AdminZkClient = createMock(classOf[AdminZkClient])
-    val propsCapture: Capture[Properties] = newCapture(CaptureType.ALL)
-    val assigner = new ReassignPartitionsCommand(zk, None, proposed, Map.empty, admin)
-    expect(admin.changeBrokerConfig(anyObject().asInstanceOf[List[Int]], capture(propsCapture))).anyTimes()
-
-    //Expect the existing broker config to be changed from 10/100 to 1000
-    val existingConfigs = CoreUtils.propsWith(
-      (DynamicConfig.Broker.FollowerReplicationThrottledRateProp, "10"),
-      (DynamicConfig.Broker.LeaderReplicationThrottledRateProp, "100")
-    )
-    expect(admin.fetchEntityConfig(is(ConfigType.Broker), is("100"))).andReturn(copyOf(existingConfigs))
-    expect(admin.fetchEntityConfig(is(ConfigType.Broker), is("101"))).andReturn(copyOf(existingConfigs))
-    expect(admin.fetchEntityConfig(is(ConfigType.Broker), is("102"))).andReturn(copyOf(existingConfigs))
-    replay(admin)
-
-    //When
-    assigner.maybeLimit(Throttle(1000))
-
-    //Then
-    for (actual <- propsCapture.getValues.asScala) {
-      assertEquals("1000", actual.getProperty(DynamicConfig.Broker.LeaderReplicationThrottledRateProp))
-      assertEquals("1000", actual.getProperty(DynamicConfig.Broker.FollowerReplicationThrottledRateProp))
-    }
-    assertEquals(3, propsCapture.getValues.size) //three brokers
-  }
-
-  @Test
-  def shouldNotOverwriteExistingPropertiesWhenLimitIsAdded(): Unit = {
-    //Given
-    val existing = Map(new TopicPartition("topic1", 0) -> Seq(100, 101))
-    val proposed = mutable.Map(new TopicPartition("topic1", 0) -> Seq(101, 102))
-
-    //Setup
-    val zk = stubZKClient(existing)
-    val admin: AdminZkClient = createMock(classOf[AdminZkClient])
-    val propsCapture: Capture[Properties] = newCapture(CaptureType.ALL)
-    val assigner = new ReassignPartitionsCommand(zk, None, proposed, Map.empty, admin)
-    expect(admin.changeBrokerConfig(anyObject().asInstanceOf[List[Int]], capture(propsCapture))).anyTimes()
-
-    //Given there is some existing config
-    expect(admin.fetchEntityConfig(is(ConfigType.Broker), anyString())).andReturn(
-      propsWith("useful.key", "useful.value")).atLeastOnce()
-
-    replay(admin)
-
-    //When
-    assigner.maybeLimit(Throttle(1000))
-
-    //Then other property remains
-    for (actual <- propsCapture.getValues.asScala) {
-      assertEquals("useful.value", actual.getProperty("useful.key"))
-      assertEquals("1000", actual.getProperty(DynamicConfig.Broker.LeaderReplicationThrottledRateProp))
-      assertEquals("1000", actual.getProperty(DynamicConfig.Broker.FollowerReplicationThrottledRateProp))
-    }
-    assertEquals(3, propsCapture.getValues.size) //3 brokers
-  }
-
-  @Test
-  def shouldRemoveThrottleLimitFromAllBrokers(): Unit = {
-    //Given 3 brokers, but with assignment only covering 2 of them
-    val brokers = Seq(100, 101, 102)
-    val status = mutable.Map(new TopicPartition("topic1", 0) -> ReassignmentCompleted)
-    val existingBrokerConfigs = propsWith(
-      (DynamicConfig.Broker.FollowerReplicationThrottledRateProp, "10"),
-      (DynamicConfig.Broker.LeaderReplicationThrottledRateProp, "100"),
-      ("useful.key", "value")
-    )
-
-    //Setup
-    val zk = stubZKClient(brokers = brokers)
-    val admin: AdminZkClient = createMock(classOf[AdminZkClient])
-    val propsCapture: Capture[Properties] = newCapture(CaptureType.ALL)
-    expect(admin.fetchEntityConfig(is(ConfigType.Topic), anyString())).andStubReturn(new Properties)
-    expect(admin.changeBrokerConfig(anyObject().asInstanceOf[Seq[Int]], capture(propsCapture))).anyTimes()
-    //Stub each invocation as EasyMock caches the return value which can be mutated
-    expect(admin.fetchEntityConfig(is(ConfigType.Broker), is("100"))).andReturn(copyOf(existingBrokerConfigs))
-    expect(admin.fetchEntityConfig(is(ConfigType.Broker), is("101"))).andReturn(copyOf(existingBrokerConfigs))
-    expect(admin.fetchEntityConfig(is(ConfigType.Broker), is("102"))).andReturn(copyOf(existingBrokerConfigs))
-    replay(admin)
-
-    //When
-    ReassignPartitionsCommand.removeThrottle(zk, status, Map.empty, admin)
-
-    //Then props should have gone (dummy remains)
-    for (capture <- propsCapture.getValues.asScala) {
-      assertEquals("value", capture.get("useful.key"))
-      assertNull(capture.get(DynamicConfig.Broker.FollowerReplicationThrottledRateProp))
-      assertNull(capture.get(DynamicConfig.Broker.LeaderReplicationThrottledRateProp))
-    }
-    assertEquals(3, propsCapture.getValues.size) //3 brokers
-  }
-
-  @Test
-  def shouldRemoveThrottleReplicaListBasedOnProposedAssignment(): Unit = {
-    //Given two topics with existing config
-    val status = mutable.Map(new TopicPartition("topic1", 0) -> ReassignmentCompleted,
-                             new TopicPartition("topic2", 0) -> ReassignmentCompleted)
-    val existingConfigs = CoreUtils.propsWith(
-      (LogConfig.LeaderReplicationThrottledReplicasProp, "1:100:2:100"),
-      (LogConfig.FollowerReplicationThrottledReplicasProp, "1:101,2:101"),
-      ("useful.key", "value")
-    )
-
-    //Setup
-    val zk = stubZKClient(brokers = Seq(100, 101))
-    val admin: AdminZkClient = createMock(classOf[AdminZkClient])
-    val propsCapture: Capture[Properties] = newCapture(CaptureType.ALL)
-    expect(admin.fetchEntityConfig(is(ConfigType.Broker), anyString())).andStubReturn(new Properties)
-    expect(admin.fetchEntityConfig(is(ConfigType.Topic), is("topic1"))).andStubReturn(copyOf(existingConfigs))
-    expect(admin.fetchEntityConfig(is(ConfigType.Topic), is("topic2"))).andStubReturn(copyOf(existingConfigs))
-
-    //Should change both topics
-    expect(admin.changeTopicConfig(is("topic1"), capture(propsCapture)))
-    expect(admin.changeTopicConfig(is("topic2"), capture(propsCapture)))
-
-    replay(admin)
-
-    //When
-    ReassignPartitionsCommand.removeThrottle(zk, status, Map.empty, admin)
-
-    //Then props should have gone (dummy remains)
-    for (actual <- propsCapture.getValues.asScala) {
-      assertEquals("value", actual.getProperty("useful.key"))
-      assertNull(actual.getProperty(LogConfig.LeaderReplicationThrottledReplicasProp))
-      assertNull(actual.getProperty(LogConfig.FollowerReplicationThrottledReplicasProp))
-    }
-    assertEquals(2, propsCapture.getValues.size) //2 topics
-  }
-
-  @Test
-  def testPartitionReassignmentWithLeaderInNewReplicas(): Unit = {
-    val expectedReplicaAssignment = Map(0  -> List(0, 1, 2))
-    val topic = "test"
-    // create brokers
-    servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b)))
-    // create the topic
-    TestUtils.createTopic(zkClient, topic, expectedReplicaAssignment, servers)
-    // reassign partition 0
-    val newReplicas = Seq(0, 2, 3)
-    val partitionToBeReassigned = 0
-    val topicAndPartition = new TopicPartition(topic, partitionToBeReassigned)
-    val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, None, Map(topicAndPartition -> newReplicas), adminZkClient = adminZkClient)
-    assertTrue("Partition reassignment attempt failed for [test, 0]", reassignPartitionsCommand.reassignPartitions())
-    // wait until reassignment is completed
-    TestUtils.waitUntilTrue(() => {
-        ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, Map(topicAndPartition -> newReplicas))
-          .getOrElse(topicAndPartition, fail(s"Failed to get reassignment status for $topicAndPartition")) == ReassignmentCompleted
-      },
-      "Partition reassignment should complete")
-    val assignedReplicas = zkClient.getReplicasForPartition(new TopicPartition(topic, partitionToBeReassigned))
-    // in sync replicas should not have any replica that is not in the new assigned replicas
-    checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas)
-    assertEquals("Partition should have been reassigned to 0, 2, 3", newReplicas, assignedReplicas)
-    ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers)
-    TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet,
-                            "New replicas should exist on brokers")
-  }
-
-  @Test
-  def testPartitionReassignmentWithLeaderNotInNewReplicas(): Unit = {
-    val expectedReplicaAssignment = Map(0  -> List(0, 1, 2))
-    val topic = "test"
-    // create brokers
-    servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b)))
-    // create the topic
-    TestUtils.createTopic(zkClient, topic, expectedReplicaAssignment, servers)
-    // reassign partition 0
-    val newReplicas = Seq(1, 2, 3)
-    val partitionToBeReassigned = 0
-    val topicAndPartition = new TopicPartition(topic, partitionToBeReassigned)
-    val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, None, Map(topicAndPartition -> newReplicas), adminZkClient = adminZkClient)
-    assertTrue("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions())
-    // wait until reassignment is completed
-    TestUtils.waitUntilTrue(() => {
-        ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, Map(topicAndPartition -> newReplicas))
-          .getOrElse(topicAndPartition, fail(s"Failed to get reassignment status for $topicAndPartition")) == ReassignmentCompleted
-      },
-      "Partition reassignment should complete")
-    val assignedReplicas = zkClient.getReplicasForPartition(new TopicPartition(topic, partitionToBeReassigned))
-    assertEquals("Partition should have been reassigned to 0, 2, 3", newReplicas, assignedReplicas)
-    checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas)
-    ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers)
-    TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet,
-                            "New replicas should exist on brokers")
-  }
-
-  @Test
-  def testPartitionReassignmentNonOverlappingReplicas(): Unit = {
-    val expectedReplicaAssignment = Map(0  -> List(0, 1))
-    val topic = "test"
-    // create brokers
-    servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b)))
-    // create the topic
-    TestUtils.createTopic(zkClient, topic, expectedReplicaAssignment, servers)
-    // reassign partition 0
-    val newReplicas = Seq(2, 3)
-    val partitionToBeReassigned = 0
-    val topicAndPartition = new TopicPartition(topic, partitionToBeReassigned)
-    val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, None, Map(topicAndPartition -> newReplicas),  adminZkClient = adminZkClient)
-    assertTrue("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions())
-    // wait until reassignment is completed
-    TestUtils.waitUntilTrue(() => {
-        ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, Map(topicAndPartition -> newReplicas))
-          .getOrElse(topicAndPartition, fail(s"Failed to get reassignment status for $topicAndPartition")) == ReassignmentCompleted
-      },
-      "Partition reassignment should complete")
-    val assignedReplicas = zkClient.getReplicasForPartition(new TopicPartition(topic, partitionToBeReassigned))
-    assertEquals("Partition should have been reassigned to 2, 3", newReplicas, assignedReplicas)
-    checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas)
-    ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers)
-    TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet,
-                            "New replicas should exist on brokers")
-  }
-
-  @Test
-  def testReassigningNonExistingPartition(): Unit = {
-    val topic = "test"
-    // create brokers
-    servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b)))
-    // reassign partition 0
-    val newReplicas = Seq(2, 3)
-    val partitionToBeReassigned = 0
-    val topicAndPartition = new TopicPartition(topic, partitionToBeReassigned)
-    val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, None, Map(topicAndPartition -> newReplicas), adminZkClient = adminZkClient)
-    assertFalse("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions())
-    val reassignedPartitions = zkClient.getPartitionReassignment
-    assertFalse("Partition should not be reassigned", reassignedPartitions.contains(topicAndPartition))
-  }
-
-  @Test
-  def testResumePartitionReassignmentThatWasCompleted(): Unit = {
-    val initialAssignment = Map(0  -> List(0, 2))
-    val topic = "test"
-    // create the topic
-    adminZkClient.createTopicWithAssignment(topic, config = new Properties, initialAssignment)
-    // put the partition in the reassigned path as well
-    // reassign partition 0
-    val newReplicas = Seq(0, 1)
-    val partitionToBeReassigned = 0
-    val topicAndPartition = new TopicPartition(topic, partitionToBeReassigned)
-    val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, None, Map(topicAndPartition -> newReplicas), adminZkClient = adminZkClient)
-    reassignPartitionsCommand.reassignPartitions()
-    // create brokers
-    servers = TestUtils.createBrokerConfigs(2, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b)))
-
-    // wait until reassignment completes
-    TestUtils.waitUntilTrue(() => !zkClient.reassignPartitionsInProgress(),
-                            "Partition reassignment should complete")
-    val assignedReplicas = zkClient.getReplicasForPartition(new TopicPartition(topic, partitionToBeReassigned))
-    assertEquals("Partition should have been reassigned to 0, 1", newReplicas, assignedReplicas)
-    checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas)
-    // ensure that there are no under replicated partitions
-    ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers)
-    TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet,
-                            "New replicas should exist on brokers")
-  }
-
-  private def getBrokersWithPartitionDir(servers: Iterable[KafkaServer], topic: String, partitionId: Int): Set[Int] = {
-    servers.filter(server => new File(server.config.logDirs.head, topic + "-" + partitionId).exists)
-           .map(_.config.brokerId)
-           .toSet
-  }
-
-  //Override eq as is for brevity
-  def is[T](v: T): T = EasyMock.eq(v)
-
-  @Before
-  def setup(): Unit = {
-    calls = 0
-  }
-
-  def stubZKClient(existingAssignment: Map[TopicPartition, Seq[Int]] = Map[TopicPartition, Seq[Int]](),
-                   brokers: Seq[Int] = Seq[Int]()): KafkaZkClient = {
-    val zkClient: KafkaZkClient = createMock(classOf[KafkaZkClient])
-    expect(zkClient.getReplicaAssignmentForTopics(anyObject().asInstanceOf[Set[String]])).andStubReturn(existingAssignment)
-    expect(zkClient.getAllBrokersInCluster).andStubReturn(brokers.map(TestUtils.createBroker(_, "", 1)))
-    replay(zkClient)
-    zkClient
-  }
-
-  def toReplicaSet(throttledReplicasString: Any): Set[String] = {
-    throttledReplicasString.toString.split(",").toSet
-  }
-}
diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsUnitTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsUnitTest.scala
new file mode 100644
index 0000000..6dbca36
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsUnitTest.scala
@@ -0,0 +1,643 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.admin
+
+import java.util.concurrent.ExecutionException
+import java.util.{Arrays, Collections}
+
+import kafka.admin.ReassignPartitionsCommand._
+import kafka.common.AdminCommandFailedException
+import kafka.utils.Exit
+import org.apache.kafka.clients.admin.{Config, MockAdminClient, PartitionReassignment}
+import org.apache.kafka.common.config.ConfigResource
+import org.apache.kafka.common.errors.{InvalidReplicationFactorException, UnknownTopicOrPartitionException}
+import org.apache.kafka.common.{Node, TopicPartition, TopicPartitionInfo, TopicPartitionReplica}
+import org.junit.Assert.{assertEquals, assertFalse, assertThrows, assertTrue}
+import org.junit.function.ThrowingRunnable
+import org.junit.rules.Timeout
+import org.junit.{After, Assert, Before, Rule, Test}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+class ReassignPartitionsUnitTest {
+  @Rule
+  def globalTimeout: Timeout = Timeout.millis(60000)
+
+  @Before
+  def setUp(): Unit = {
+    Exit.setExitProcedure((_, message) => throw new IllegalArgumentException(message.orNull))
+  }
+
+  @After
+  def tearDown(): Unit = {
+    Exit.resetExitProcedure()
+  }
+
+  @Test
+  def testCompareTopicPartitions(): Unit = {
+    assertTrue(compareTopicPartitions(new TopicPartition("abc", 0),
+      new TopicPartition("abc", 1)))
+    assertFalse(compareTopicPartitions(new TopicPartition("def", 0),
+      new TopicPartition("abc", 1)))
+  }
+
+  @Test
+  def testCompareTopicPartitionReplicas(): Unit = {
+    assertTrue(compareTopicPartitionReplicas(new TopicPartitionReplica("def", 0, 0),
+      new TopicPartitionReplica("abc", 0, 1)))
+    assertFalse(compareTopicPartitionReplicas(new TopicPartitionReplica("def", 0, 0),
+      new TopicPartitionReplica("cde", 0, 0)))
+  }
+
+  @Test
+  def testPartitionReassignStatesToString(): Unit = {
+    assertEquals(Seq(
+      "Status of partition reassignment:",
+      "Reassignment of partition bar-0 is still in progress.",
+      "Reassignment of partition foo-0 is complete.",
+      "Reassignment of partition foo-1 is still in progress.").
+        mkString(System.lineSeparator()),
+      partitionReassignmentStatesToString(Map(
+        new TopicPartition("foo", 0) ->
+          new PartitionReassignmentState(Seq(1, 2, 3), Seq(1, 2, 3), true),
+        new TopicPartition("foo", 1) ->
+          new PartitionReassignmentState(Seq(1, 2, 3), Seq(1, 2, 4), false),
+        new TopicPartition("bar", 0) ->
+          new PartitionReassignmentState(Seq(1, 2, 3), Seq(1, 2, 4), false),
+      )))
+  }
+
+  private def addTopics(adminClient: MockAdminClient): Unit = {
+    val b = adminClient.brokers()
+    adminClient.addTopic(false, "foo", Arrays.asList(
+      new TopicPartitionInfo(0, b.get(0),
+        Arrays.asList(b.get(0), b.get(1), b.get(2)),
+        Arrays.asList(b.get(0), b.get(1))),
+      new TopicPartitionInfo(1, b.get(1),
+        Arrays.asList(b.get(1), b.get(2), b.get(3)),
+        Arrays.asList(b.get(1), b.get(2), b.get(3)))
+    ), Collections.emptyMap())
+    adminClient.addTopic(false, "bar", Arrays.asList(
+      new TopicPartitionInfo(0, b.get(2),
+        Arrays.asList(b.get(2), b.get(3), b.get(0)),
+        Arrays.asList(b.get(2), b.get(3), b.get(0)))
+    ), Collections.emptyMap())
+  }
+
+  @Test
+  def testFindPartitionReassignmentStates(): Unit = {
+    val adminClient = new MockAdminClient.Builder().numBrokers(4).build()
+    try {
+      addTopics(adminClient)
+      // Create a reassignment and test findPartitionReassignmentStates.
+      assertEquals(Map(
+          new TopicPartition("quux", 0) -> classOf[UnknownTopicOrPartitionException]
+        ),
+        alterPartitionReassignments(adminClient, Map(
+          new TopicPartition("foo", 0) -> Seq(0,1,3),
+          new TopicPartition("quux", 0) -> Seq(1,2,3))).mapValues(_.getClass).toMap)
+      assertEquals((Map(
+          new TopicPartition("foo", 0) -> new PartitionReassignmentState(Seq(0,1,2), Seq(0,1,3), false),
+          new TopicPartition("foo", 1) -> new PartitionReassignmentState(Seq(1,2,3), Seq(1,2,3), true)
+        ), true),
+        findPartitionReassignmentStates(adminClient, Seq(
+          (new TopicPartition("foo", 0), Seq(0,1,3)),
+          (new TopicPartition("foo", 1), Seq(1,2,3))
+        )))
+      // Cancel the reassignment and test findPartitionReassignmentStates again.
+      assertEquals(Map(
+          new TopicPartition("quux", 2) -> classOf[UnknownTopicOrPartitionException]
+        ),
+        cancelPartitionReassignments(adminClient, Set(
+          new TopicPartition("foo", 0),
+          new TopicPartition("quux", 2))).mapValues(_.getClass).toMap)
+      assertEquals((Map(
+          new TopicPartition("foo", 0) -> new PartitionReassignmentState(Seq(0,1,2), Seq(0,1,3), true),
+          new TopicPartition("foo", 1) -> new PartitionReassignmentState(Seq(1,2,3), Seq(1,2,3), true)
+        ), false),
+          findPartitionReassignmentStates(adminClient, Seq(
+            (new TopicPartition("foo", 0), Seq(0,1,3)),
+            (new TopicPartition("foo", 1), Seq(1,2,3))
+          )))
+    } finally {
+      adminClient.close()
+    }
+  }
+
+  @Test
+  def testFindLogDirMoveStates(): Unit = {
+    val adminClient = new MockAdminClient.Builder().
+      numBrokers(4).
+      brokerLogDirs(Arrays.asList(
+          Arrays.asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"),
+          Arrays.asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"),
+          Arrays.asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"),
+          Arrays.asList("/tmp/kafka-logs0", null))).
+      build();
+    try {
+      addTopics(adminClient)
+      val b = adminClient.brokers()
+      adminClient.addTopic(false, "quux", Arrays.asList(
+        new TopicPartitionInfo(0, b.get(2),
+            Arrays.asList(b.get(1), b.get(2), b.get(3)),
+            Arrays.asList(b.get(1), b.get(2), b.get(3)))),
+        Collections.emptyMap())
+      adminClient.alterReplicaLogDirs(Map(
+        new TopicPartitionReplica("foo", 0, 0) -> "/tmp/kafka-logs1",
+        new TopicPartitionReplica("quux", 0, 0) -> "/tmp/kafka-logs1"
+      ).asJava).all().get()
+      assertEquals(Map(
+        new TopicPartitionReplica("bar", 0, 0) -> new CompletedMoveState("/tmp/kafka-logs0"),
+        new TopicPartitionReplica("foo", 0, 0) -> new ActiveMoveState("/tmp/kafka-logs0",
+            "/tmp/kafka-logs1", "/tmp/kafka-logs1"),
+        new TopicPartitionReplica("foo", 1, 0) -> new CancelledMoveState("/tmp/kafka-logs0",
+          "/tmp/kafka-logs1"),
+        new TopicPartitionReplica("quux", 1, 0) -> new MissingLogDirMoveState("/tmp/kafka-logs1"),
+        new TopicPartitionReplica("quuz", 0, 0) -> new MissingReplicaMoveState("/tmp/kafka-logs0")
+      ), findLogDirMoveStates(adminClient, Map(
+        new TopicPartitionReplica("bar", 0, 0) -> "/tmp/kafka-logs0",
+        new TopicPartitionReplica("foo", 0, 0) -> "/tmp/kafka-logs1",
+        new TopicPartitionReplica("foo", 1, 0) -> "/tmp/kafka-logs1",
+        new TopicPartitionReplica("quux", 1, 0) -> "/tmp/kafka-logs1",
+        new TopicPartitionReplica("quuz", 0, 0) -> "/tmp/kafka-logs0"
+      )))
+    } finally {
+      adminClient.close()
+    }
+  }
+
+  @Test
+  def testReplicaMoveStatesToString(): Unit = {
+    assertEquals(Seq(
+      "Reassignment of replica bar-0-0 completed successfully.",
+      "Reassignment of replica foo-0-0 is still in progress.",
+      "Partition foo-1 on broker 0 is not being moved from log dir /tmp/kafka-logs0 to /tmp/kafka-logs1.",
+      "Partition quux-0 cannot be found in any live log directory on broker 0.",
+      "Partition quux-1 on broker 1 is being moved to log dir /tmp/kafka-logs2 instead of /tmp/kafka-logs1.",
+      "Partition quux-2 is not found in any live log dir on broker 1. " +
+          "There is likely an offline log directory on the broker.").mkString(System.lineSeparator()),
+        replicaMoveStatesToString(Map(
+          new TopicPartitionReplica("bar", 0, 0) -> new CompletedMoveState("/tmp/kafka-logs0"),
+          new TopicPartitionReplica("foo", 0, 0) -> new ActiveMoveState("/tmp/kafka-logs0",
+            "/tmp/kafka-logs1", "/tmp/kafka-logs1"),
+          new TopicPartitionReplica("foo", 1, 0) -> new CancelledMoveState("/tmp/kafka-logs0",
+            "/tmp/kafka-logs1"),
+          new TopicPartitionReplica("quux", 0, 0) -> new MissingReplicaMoveState("/tmp/kafka-logs1"),
+          new TopicPartitionReplica("quux", 1, 1) -> new ActiveMoveState("/tmp/kafka-logs0",
+            "/tmp/kafka-logs1", "/tmp/kafka-logs2"),
+          new TopicPartitionReplica("quux", 2, 1) -> new MissingLogDirMoveState("/tmp/kafka-logs1")
+        )))
+  }
+
+  @Test
+  def testGetReplicaAssignments(): Unit = {
+    val adminClient = new MockAdminClient.Builder().numBrokers(4).build()
+    try {
+      addTopics(adminClient)
+      assertEquals(Map(
+          new TopicPartition("foo", 0) -> Seq(0, 1, 2),
+          new TopicPartition("foo", 1) -> Seq(1, 2, 3),
+        ),
+        getReplicaAssignmentForTopics(adminClient, Seq("foo")))
+      assertEquals(Map(
+          new TopicPartition("foo", 0) -> Seq(0, 1, 2),
+          new TopicPartition("bar", 0) -> Seq(2, 3, 0),
+        ),
+        getReplicaAssignmentForPartitions(adminClient, Set(
+          new TopicPartition("foo", 0), new TopicPartition("bar", 0))))
+    } finally {
+      adminClient.close()
+    }
+  }
+
+  @Test
+  def testGetBrokerRackInformation(): Unit = {
+    val adminClient = new MockAdminClient.Builder().
+      brokers(Arrays.asList(new Node(0, "locahost", 9092, "rack0"),
+        new Node(1, "locahost", 9093, "rack1"),
+        new Node(2, "locahost", 9094, null))).
+      build()
+    try {
+      assertEquals(Seq(
+        new BrokerMetadata(0, Some("rack0")),
+        new BrokerMetadata(1, Some("rack1"))
+      ), getBrokerMetadata(adminClient, Seq(0, 1), true))
+      assertEquals(Seq(
+        new BrokerMetadata(0, None),
+        new BrokerMetadata(1, None)
+      ), getBrokerMetadata(adminClient, Seq(0, 1), false))
+      assertStartsWith("Not all brokers have rack information",
+        assertThrows(classOf[AdminOperationException], new ThrowingRunnable {
+          override def run(): Unit = getBrokerMetadata(adminClient, Seq(1, 2), true)
+        }).getMessage)
+      assertEquals(Seq(
+        new BrokerMetadata(1, None),
+        new BrokerMetadata(2, None)
+      ), getBrokerMetadata(adminClient, Seq(1, 2), false))
+    } finally {
+      adminClient.close()
+    }
+  }
+
+  @Test
+  def testParseGenerateAssignmentArgs(): Unit = {
+    assertStartsWith("Broker list contains duplicate entries",
+      assertThrows("Expected to detect duplicate broker list entries",
+        classOf[AdminCommandFailedException], new ThrowingRunnable {
+          override def run():Unit = parseGenerateAssignmentArgs(
+            """{"topics": [{"topic": "foo"}], "version":1}""", "1,1,2")
+        }).getMessage)
+    assertStartsWith("Broker list contains duplicate entries",
+      assertThrows("Expected to detect duplicate broker list entries",
+        classOf[AdminCommandFailedException], new ThrowingRunnable {
+          override def run():Unit = parseGenerateAssignmentArgs(
+            """{"topics": [{"topic": "foo"}], "version":1}""", "5,2,3,4,5")
+        }).getMessage)
+    assertEquals((Seq(5,2,3,4),Seq("foo")),
+      parseGenerateAssignmentArgs("""{"topics": [{"topic": "foo"}], "version":1}""",
+        "5,2,3,4"))
+    assertStartsWith("List of topics to reassign contains duplicate entries",
+      assertThrows("Expected to detect duplicate topic entries",
+        classOf[AdminCommandFailedException], new ThrowingRunnable {
+          override def run():Unit = parseGenerateAssignmentArgs(
+            """{"topics": [{"topic": "foo"},{"topic": "foo"}], "version":1}""", "5,2,3,4")
+        }).getMessage)
+    assertEquals((Seq(5,3,4),Seq("foo","bar")),
+      parseGenerateAssignmentArgs(
+        """{"topics": [{"topic": "foo"},{"topic": "bar"}], "version":1}""",
+        "5,3,4"))
+  }
+
+  @Test
+  def testGenerateAssignmentFailsWithoutEnoughReplicas(): Unit = {
+    val adminClient = new MockAdminClient.Builder().numBrokers(4).build()
+    try {
+      addTopics(adminClient)
+      assertStartsWith("Replication factor: 3 larger than available brokers: 2",
+        assertThrows("Expected generateAssignment to fail",
+          classOf[InvalidReplicationFactorException], new ThrowingRunnable {
+            override def run():Unit = {
+              generateAssignment(adminClient,
+                """{"topics":[{"topic":"foo"},{"topic":"bar"}]}""", "0,1", false)
+            }
+          }).getMessage)
+    } finally {
+      adminClient.close()
+    }
+  }
+
+  @Test
+  def testGenerateAssignmentWithInconsistentRacks(): Unit = {
+    val adminClient = new MockAdminClient.Builder().
+      brokers(Arrays.asList(
+        new Node(0, "locahost", 9092, "rack0"),
+        new Node(1, "locahost", 9093, "rack0"),
+        new Node(2, "locahost", 9094, null),
+        new Node(3, "locahost", 9095, "rack1"),
+        new Node(4, "locahost", 9096, "rack1"),
+        new Node(5, "locahost", 9097, "rack2"))).
+      build()
+    try {
+      addTopics(adminClient)
+      assertStartsWith("Not all brokers have rack information.",
+        assertThrows("Expected generateAssignment to fail",
+          classOf[AdminOperationException], new ThrowingRunnable {
+            override def run():Unit = {
+              generateAssignment(adminClient,
+                """{"topics":[{"topic":"foo"}]}""", "0,1,2,3", true)
+            }
+          }).getMessage)
+      // It should succeed when --disable-rack-aware is used.
+      val (_, current) = generateAssignment(adminClient,
+        """{"topics":[{"topic":"foo"}]}""", "0,1,2,3", false)
+      assertEquals(Map(
+        new TopicPartition("foo", 0) -> Seq(0, 1, 2),
+        new TopicPartition("foo", 1) -> Seq(1, 2, 3),
+      ), current)
+    } finally {
+      adminClient.close()
+    }
+  }
+
+  @Test
+  def testGenerateAssignmentWithFewerBrokers(): Unit = {
+    val adminClient = new MockAdminClient.Builder().numBrokers(4).build()
+    try {
+      addTopics(adminClient)
+      val goalBrokers = Set(0,1,3)
+      val (proposed, current) = generateAssignment(adminClient,
+        """{"topics":[{"topic":"foo"},{"topic":"bar"}]}""",
+        goalBrokers.mkString(","), false)
+      assertEquals(Map(
+        new TopicPartition("foo", 0) -> Seq(0, 1, 2),
+        new TopicPartition("foo", 1) -> Seq(1, 2, 3),
+        new TopicPartition("bar", 0) -> Seq(2, 3, 0)
+      ), current)
+
+      // The proposed assignment should only span the provided brokers
+      proposed.values.foreach {
+        case replicas => {
+          if (!replicas.forall(goalBrokers.contains(_))) {
+            Assert.fail(s"Proposed assignment ${proposed} puts replicas on brokers " +
+              s"other than ${goalBrokers}")
+          }
+        }
+      }
+    } finally {
+      adminClient.close()
+    }
+  }
+
+  @Test
+  def testCurrentPartitionReplicaAssignmentToString(): Unit = {
+    assertEquals(Seq(
+        """Current partition replica assignment""",
+        """""",
+        """{"version":1,"partitions":""" +
+          """[{"topic":"bar","partition":0,"replicas":[7,8],"log_dirs":["any","any"]},""" +
+          """{"topic":"foo","partition":1,"replicas":[4,5,6],"log_dirs":["any","any","any"]}]""" +
+        """}""",
+        """""",
+        """Save this to use as the --reassignment-json-file option during rollback"""
+      ).mkString(System.lineSeparator()),
+      currentPartitionReplicaAssignmentToString(Map(
+          new TopicPartition("foo", 1) -> Seq(1,2,3),
+          new TopicPartition("bar", 0) -> Seq(7,8,9)
+        ),
+        Map(
+          new TopicPartition("foo", 0) -> Seq(1,2,3),
+          new TopicPartition("foo", 1) -> Seq(4,5,6),
+          new TopicPartition("bar", 0) -> Seq(7,8),
+          new TopicPartition("baz", 0) -> Seq(10,11,12)
+        ),
+      ))
+  }
+
+  @Test
+  def testMoveMap(): Unit = {
+    val moveMap = calculateMoveMap(Map(
+      new TopicPartition("foo", 0) -> new PartitionReassignment(
+        Arrays.asList(1,2,3),Arrays.asList(4),Arrays.asList(3)),
+      new TopicPartition("foo", 1) -> new PartitionReassignment(
+        Arrays.asList(4,5,6),Arrays.asList(7, 8),Arrays.asList(4, 5))
+    ), Map(
+      new TopicPartition("foo", 0) -> Seq(1,2,5),
+      new TopicPartition("bar", 0) -> Seq(1,2,3)
+    ), Map(
+      new TopicPartition("foo", 0) -> Seq(1,2,3),
+      new TopicPartition("foo", 1) -> Seq(4,5,6),
+      new TopicPartition("bar", 0) -> Seq(2,3,4),
+      new TopicPartition("baz", 0) -> Seq(1,2,3)
+    ))
+    assertEquals(
+      mutable.Map("foo" -> mutable.Map(
+          0 -> new PartitionMove(mutable.Set(1,2,3), mutable.Set(5)),
+          1 -> new PartitionMove(mutable.Set(4,5,6), mutable.Set(7, 8))
+        ),
+        "bar" -> mutable.Map(
+          0 -> new PartitionMove(mutable.Set(2,3,4), mutable.Set(1)),
+        )
+      ), moveMap)
+    assertEquals(Map(
+        "foo" -> "0:1,0:2,0:3,1:4,1:5,1:6",
+        "bar" -> "0:2,0:3,0:4"
+      ), calculateLeaderThrottles(moveMap))
+    assertEquals(Map(
+        "foo" -> "0:5,1:7,1:8",
+        "bar" -> "0:1"
+      ), calculateFollowerThrottles(moveMap))
+    assertEquals(Set(1,2,3,4,5,6,7,8), calculateReassigningBrokers(moveMap))
+    assertEquals(Set(0,2), calculateMovingBrokers(
+      Set(new TopicPartitionReplica("quux", 0, 0),
+          new TopicPartitionReplica("quux", 1, 2))))
+  }
+
+  @Test
+  def testParseExecuteAssignmentArgs(): Unit = {
+    assertStartsWith("Partition reassignment list cannot be empty",
+      assertThrows("Expected to detect empty partition reassignment list",
+        classOf[AdminCommandFailedException], new ThrowingRunnable {
+          override def run():Unit =
+            parseExecuteAssignmentArgs("""{"version":1,"partitions":[]}""")
+        }).getMessage)
+    assertStartsWith("Partition reassignment contains duplicate topic partitions",
+      assertThrows("Expected to detect a partition list with duplicate entries",
+        classOf[AdminCommandFailedException], new ThrowingRunnable {
+          override def run():Unit =
+            parseExecuteAssignmentArgs(
+              """{"version":1,"partitions":""" +
+                """[{"topic":"foo","partition":0,"replicas":[0,1],"log_dirs":["any","any"]},""" +
+                """{"topic":"foo","partition":0,"replicas":[2,3,4],"log_dirs":["any","any","any"]}""" +
+                """]}""")
+        }).getMessage)
+    assertStartsWith("Partition reassignment contains duplicate topic partitions",
+      assertThrows("Expected to detect a partition replica list with duplicate entries",
+        classOf[AdminCommandFailedException], new ThrowingRunnable {
+          override def run():Unit =
+            parseExecuteAssignmentArgs(
+              """{"version":1,"partitions":""" +
+                """[{"topic":"foo","partition":0,"replicas":[0,1],"log_dirs":["/abc","/def"]},""" +
+                """{"topic":"foo","partition":0,"replicas":[2,3],"log_dirs":["/abc","/def"]}""" +
+                """]}""")
+        }).getMessage)
+    assertStartsWith("Partition replica lists may not contain duplicate entries",
+      assertThrows("Expected to detect a partition replica list with duplicate entries",
+        classOf[AdminCommandFailedException], new ThrowingRunnable {
+          override def run():Unit =
+            parseExecuteAssignmentArgs(
+              """{"version":1,"partitions":""" +
+                """[{"topic":"foo","partition":0,"replicas":[0,0],"log_dirs":["/abc","/def"]},""" +
+                """{"topic":"foo","partition":1,"replicas":[2,3],"log_dirs":["/abc","/def"]}""" +
+                """]}""")
+        }).getMessage)
+    assertEquals((Map(
+        new TopicPartition("foo", 0) -> Seq(1, 2, 3),
+        new TopicPartition("foo", 1) -> Seq(3, 4, 5),
+      ), Map(
+      )),
+      parseExecuteAssignmentArgs(
+        """{"version":1,"partitions":""" +
+          """[{"topic":"foo","partition":0,"replicas":[1,2,3],"log_dirs":["any","any","any"]},""" +
+          """{"topic":"foo","partition":1,"replicas":[3,4,5],"log_dirs":["any","any","any"]}""" +
+          """]}"""))
+    assertEquals((Map(
+      new TopicPartition("foo", 0) -> Seq(1, 2, 3),
+    ), Map(
+      new TopicPartitionReplica("foo", 0, 1) -> "/tmp/a",
+      new TopicPartitionReplica("foo", 0, 2) -> "/tmp/b",
+      new TopicPartitionReplica("foo", 0, 3) -> "/tmp/c"
+    )),
+      parseExecuteAssignmentArgs(
+        """{"version":1,"partitions":""" +
+          """[{"topic":"foo","partition":0,"replicas":[1,2,3],"log_dirs":["/tmp/a","/tmp/b","/tmp/c"]}""" +
+          """]}"""))
+  }
+
+  @Test
+  def testExecuteWithInvalidPartitionsFails(): Unit = {
+    val adminClient = new MockAdminClient.Builder().numBrokers(5).build()
+    try {
+      addTopics(adminClient)
+      assertStartsWith("Topic quux not found",
+        assertThrows("Expected reassignment with non-existent topic to fail",
+          classOf[ExecutionException], new ThrowingRunnable {
+            override def run():Unit =
+              executeAssignment(adminClient, false,
+                """{"version":1,"partitions":""" +
+                """[{"topic":"foo","partition":0,"replicas":[0,1],"log_dirs":["any","any"]},""" +
+                """{"topic":"quux","partition":0,"replicas":[2,3,4],"log_dirs":["any","any","any"]}""" +
+                """]}""")
+          }).getCause.getMessage)
+    } finally {
+      adminClient.close()
+    }
+  }
+
+  @Test
+  def testExecuteWithInvalidBrokerIdFails(): Unit = {
+    val adminClient = new MockAdminClient.Builder().numBrokers(4).build()
+    try {
+      addTopics(adminClient)
+      assertStartsWith("Unknown broker id 4",
+        assertThrows("Expected reassignment with non-existent broker id to fail",
+          classOf[AdminCommandFailedException], new ThrowingRunnable {
+            override def run():Unit =
+              executeAssignment(adminClient, false,
+                """{"version":1,"partitions":""" +
+                  """[{"topic":"foo","partition":0,"replicas":[0,1],"log_dirs":["any","any"]},""" +
+                  """{"topic":"foo","partition":1,"replicas":[2,3,4],"log_dirs":["any","any","any"]}""" +
+                  """]}""")
+          }).getMessage)
+    } finally {
+      adminClient.close()
+    }
+  }
+
+  @Test
+  def testModifyBrokerThrottles(): Unit = {
+    val adminClient = new MockAdminClient.Builder().numBrokers(4).build()
+    try {
+      modifyBrokerThrottles(adminClient, Set(0, 1, 2), 1000, Set(0, 1, 2), 2000)
+      modifyBrokerThrottles(adminClient, Set(0, 3), 100, Set(0, 3), -1)
+      val brokers = Seq(0, 1, 2, 3).map(
+        id => new ConfigResource(ConfigResource.Type.BROKER, id.toString))
+      val results = adminClient.describeConfigs(brokers.asJava).all().get()
+      verifyBrokerThrottleResults(results.get(brokers(0)), 100, 2000)
+      verifyBrokerThrottleResults(results.get(brokers(1)), 1000, 2000)
+      verifyBrokerThrottleResults(results.get(brokers(2)), 1000, 2000)
+      verifyBrokerThrottleResults(results.get(brokers(3)), 100, -1)
+    } finally {
+      adminClient.close()
+    }
+  }
+
+  @Test
+  def testCurReassignmentsToString(): Unit = {
+    val adminClient = new MockAdminClient.Builder().numBrokers(4).build()
+    try {
+      addTopics(adminClient)
+      assertEquals("""No partition reassignments found.""",
+        curReassignmentsToString(adminClient))
+      assertEquals(Map(),
+        alterPartitionReassignments(adminClient, Map(
+          new TopicPartition("foo", 1) -> Seq(4,5,3),
+          new TopicPartition("foo", 0) -> Seq(0,1,4,2),
+          new TopicPartition("bar", 0) -> Seq(2,3))).mapValues(_.getClass).toMap)
+      assertEquals(Seq("Current partition reassignments:",
+                       "bar-0: replicas: 2,3,0. removing: 0.",
+                       "foo-0: replicas: 0,1,2. adding: 4.",
+                       "foo-1: replicas: 1,2,3. adding: 4,5. removing: 1,2.").mkString(System.lineSeparator()),
+                  curReassignmentsToString(adminClient))
+    } finally {
+      adminClient.close()
+    }
+  }
+
+  private def verifyBrokerThrottleResults(config: Config,
+                                          expectedInterBrokerThrottle: Long,
+                                          expectedReplicaAlterLogDirsThrottle: Long): Unit = {
+    val configs = new mutable.HashMap[String, String]
+    config.entries().asScala.foreach(entry => configs.put(entry.name(), entry.value()))
+    if (expectedInterBrokerThrottle >= 0) {
+      assertEquals(expectedInterBrokerThrottle.toString,
+        configs.getOrElse(brokerLevelLeaderThrottle, ""))
+      assertEquals(expectedInterBrokerThrottle.toString,
+        configs.getOrElse(brokerLevelFollowerThrottle, ""))
+    }
+    if (expectedReplicaAlterLogDirsThrottle >= 0) {
+      assertEquals(expectedReplicaAlterLogDirsThrottle.toString,
+        configs.getOrElse(brokerLevelLogDirThrottle, ""))
+    }
+  }
+
+  @Test
+  def testModifyTopicThrottles(): Unit = {
+    val adminClient = new MockAdminClient.Builder().numBrokers(4).build()
+    try {
+      addTopics(adminClient)
+      modifyTopicThrottles(adminClient,
+        Map("foo" -> "leaderFoo", "bar" -> "leaderBar"),
+        Map("bar" -> "followerBar"))
+      val topics = Seq("bar", "foo").map(
+        id => new ConfigResource(ConfigResource.Type.TOPIC, id.toString))
+      val results = adminClient.describeConfigs(topics.asJava).all().get()
+      verifyTopicThrottleResults(results.get(topics(0)), "leaderBar", "followerBar")
+      verifyTopicThrottleResults(results.get(topics(1)), "leaderFoo", "")
+    } finally {
+      adminClient.close()
+    }
+  }
+
+  private def verifyTopicThrottleResults(config: Config,
+                                         expectedLeaderThrottle: String,
+                                         expectedFollowerThrottle: String): Unit = {
+    val configs = new mutable.HashMap[String, String]
+    config.entries().asScala.foreach(entry => configs.put(entry.name(), entry.value()))
+    assertEquals(expectedLeaderThrottle.toString,
+      configs.getOrElse(topicLevelLeaderThrottle, ""))
+    assertEquals(expectedFollowerThrottle.toString,
+      configs.getOrElse(topicLevelFollowerThrottle, ""))
+  }
+
+  @Test
+  def testAlterReplicaLogDirs(): Unit = {
+    val adminClient = new MockAdminClient.Builder().
+      numBrokers(4).
+      brokerLogDirs(Collections.nCopies(4,
+        Arrays.asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"))).
+      build()
+    try {
+      addTopics(adminClient)
+      assertEquals(Set(
+          new TopicPartitionReplica("foo", 0, 0)
+        ),
+        alterReplicaLogDirs(adminClient, Map(
+          new TopicPartitionReplica("foo", 0, 0) -> "/tmp/kafka-logs1",
+          new TopicPartitionReplica("quux", 1, 0) -> "/tmp/kafka-logs1"
+        )))
+    } finally {
+      adminClient.close()
+    }
+  }
+
+  def assertStartsWith(prefix: String, str: String): Unit = {
+    assertTrue("Expected the string to start with %s, but it was %s".format(prefix, str),
+      str.startsWith(prefix))
+  }
+}
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
index 2ddafd0..c66874a 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
@@ -1964,7 +1964,8 @@ public class StreamThreadTest {
         final Node broker2 = new Node(1, "dummyHost-2", 1234);
         final List<Node> cluster = Arrays.asList(broker1, broker2);
 
-        final MockAdminClient adminClient = new MockAdminClient(cluster, broker1, null);
+        final MockAdminClient adminClient = new MockAdminClient.Builder().
+            brokers(cluster).clusterId(null).build();
 
         final Consumer<byte[], byte[]> consumer = EasyMock.createNiceMock(Consumer.class);
         final TaskManager taskManager = EasyMock.createNiceMock(TaskManager.class);
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetricsTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetricsTest.java
index d7059a8..acd429e 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetricsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/metrics/ThreadMetricsTest.java
@@ -335,4 +335,4 @@ public class ThreadMetricsTest {
         verify(StreamsMetricsImpl.class, streamsMetrics);
         assertThat(sensor, is(expectedSensor));
     }
-}
\ No newline at end of file
+}