Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: Rewrite, adding dedicated throttle mgmt with support for auto-removal of throttles

...

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).e

Motivation

Describe the problems you are trying to solve.

...

  • The AdminClient API currently lacks any functionality for reassigning partitions: Users have to use the kafka-reassign-partitions.sh tool (ReassignPartitionsCommand) which talks directly to ZooKeeper. This prevents the tool being used in deployments where only the brokers are exposed to clients (i.e. where the zookeeper servers are intentionally not exposed). In addition, there is a general push to refactor/rewrite/replace tools which need ZooKeeper access with equivalents which use the AdminClient API.

...

  • ReassignPartitionsCommand currently has no proper facility to report progress of a reassignment; --verify

...

  • can be used periodically to check whether the request assignments have been achieved. It would be useful if the tool could report progress better.

Public Interfaces

Briefly list any new interfaces that will be introduced as part of this proposal or any existing interfaces that will be removed or changed. The purpose of this section is to concisely call out the public contract that will come along with this feature.

A public interface is any change to the following:

...

Binary log format

...

The network protocol and api behavior

...

Any class in the public packages under clientsConfiguration, especially client configuration

  • org/apache/kafka/common/serialization

  • org/apache/kafka/common

  • org/apache/kafka/common/errors

  • org/apache/kafka/clients/producer

  • org/apache/kafka/clients/consumer (eventually, once stable)

...

Monitoring

...

Command line tools and arguments

  • ReassignPartitionsCommand, when used with a replication throttle, requires a --verify invocation when the reassignment has finished in order to remove the throttled. Therefor there exists the possibility that throttles are not removed after reassignment, with negative consequences for the performance of the cluster.

Public Interfaces

The AdminClient API will have new methods added (plus overloads for options):

The options for reassignPartitions() will support setting a throttle, and a flag for its automatic removal at the end of the reassignment. Likewise the options for changing the throttled rates and replicas will include the ability to have the throttles automatically removed.

...

New network protocol APIs will be added :to support these AdminClient APIs

The AdminClient API will have new methods added (plus overloads for options):

Support for changing DynamicConfig broker configs via AdminClient.alterConfigs() will be added

The options accepted by kafka-reassign-partitions.sh command will change:

  • AlterInterbrokerThrottledReplicasRequest and AlterInterbrokerThrottledReplicasResponse

The options accepted by kafka-reassign-partitions.sh command will change:

  • ----zookeeper will be deprecated, with a warning message
  • a new --bootstrap-server option will be added
  • a new --progress action option will be added

Proposed Changes

...

When run with --bootstrap-server it will no longer be necessary to run kafka-reassign-partitions.sh --verify to remove a throttle: This will be done automatically.

Proposed Changes

Summary of use cases

Use casecommandAdminClient
Change replication factorkafka-reassign-partitions --execute --reassignment-json-file JreassignPartitions()
Change partition assignmentkafka-reassign-partitions --execute --reassignment-json-file JreassignPartitions()
Change partition assignment with throttlekafka-reassign-partitions --execute --reassignment-json-file J --throttle R

reassignPartitions(validateOnly) // check none in progress

alterConfigs() // (leader|follower).replication.throttled.replicas

reassignPartitions()

with throttle option

Change throttled ratekafka-reassign-partitions --execute --reassignment-json-file J --throttle R

alterConfigs()TODO how to do this conveniently?

Check progress of a reassignmentkafka-reassign-partitions --progress --reassignment-json-file JdescribeReplicaLogDirs() (see KIP-113)
Check result and clear throttlekafka-reassign-partitions --verify --reassignment-json-file J

reassignPartitions(validateOnly) // check none in progressalterConfigs() // (leader|follower).replication.throttled.replicas

 

 kafka-reassign-partitions.sh and ReassignPartitionsCommand

...

  1. Perform the reassignment via the AdminClient API (described below) using the given intermediating broker(s) as bootstrap brokers.
  2. When used with --execute and --throttle, the throttle will be an auto-removed one.

Using both --zookeeper and --bootstrap-server in the same command will produce an error message and the tool will exit without doing the intended operation.

...

  • This API is asynchronous in the sense that the client cannot assume that the request is complete (or the request was rejected) once they have obtained the result for the topic from the ReassignPartitionsResult.
  • The describeReplicaLogDirs() method from KIP-113 can be used to determine progress.
  • A call to reassignPartitions() with the validateOnly option can be used to determine whether a reassignment is currently running, and therefore whether the last reassignment has finished.

  • The API doesn't directly support setting a throttle itself. A prior set of calls to alterInterBrokerThrottle() can be used to set a throttle.

Code Block
languagejava
Code Block
languagejava
linenumberstrue
/**
 * <p>Assign<p>Reassign the partitions given as the key of the given <code>assignments</code> to the corresponding 
 * list of brokers. This can be used to change the replica assignment or change the topic's replication factor. 
 * The first broker in each list is the one which holds the "preferred replica".</p>
 *
 * <p>To<p>Inter-broker changereassignment thecauses replicationsignificant factorinter-broker fortraffic aand topiccan theretake must be a key for each partition in long time
 * thein {@code assignment}s map and the corresponding list of brokers must each be of the same 
 * length which will become the new replication factororder to copy the replica data to brokers. The given options can be used impose a quota on
 * inter-broker traffic for the duration of the reassignment so that client-broker traffic is not
 * adversely affected.</p>
 *
 * <p>If only a subset of the partitions of a particular topic are present in {@code assignments} 
 * the change is taken  to be a reassignment of replicas to brokers and each list must have 
 * the same length as the current topic replication factor.</p>
 *
 * <h3>Throttling</h3>
 * <p>Inter-broker reassignment and/or increasing the replication factor causes significant 
 * inter-broker traffic and can take a long time 
 * in order to copy the replica data to brokers. It may be necessary to impose a throttle on 
 * inter-broker traffic for the duration of the reassignment so that client-broker traffic is not
 * adversely affected.</p>
 *
 * <h3>Preferred replica</h3>
 * <p>When brokers are configured with <code>auto.leader.rebalance.enable=true</code>, the broker
 * with the preferred replica will be elected leader automatically. 
 * <code>kafka-preferred-replica-election.sh</code> provides a manual trigger for this 
 * election when <code>auto.leader.rebalance.enable=false</code>.</p>
 */
public ReassignPartitionsResult reassignPartitions(Map<TopicPartition, List<Integer>> assignments)
public ReassignPartitionsResult reassignPartitions(Map<TopicPartition, List<Integer>> assignments, 
       <h3>Preferred replica</h3>
 * <p>When brokers are configured with <code>auto.leader.rebalance.enable=true</code>, the broker
 * with the preferred replica will be elected leader automatically.
 * <code>kafka-preferred-replica-election.sh</code> provides a manual trigger for this
 * election when <code>auto.leader.rebalance.enable=false</code>.</p>
 *
 * @param assignments The partition assignments.
 * @param options The options to use when reassigning the partitions
 * @return The ReassignPartitionsResult
 */
public abstract ReassignPartitionsResult reassignPartitions(Map<TopicPartition, List<Integer>> assignments,
                                            ReassignPartitionsOptions options);

Where:

Code Block
languagejava
linenumberstrue
public class ReassignPartitionsOptions extends AbstractOptions<ReassignPartitionsOptions> {

    // Note timeoutMs() inherited from AbstractOptions

    public boolean validateOnly()

    /**
     * Validate the request only: Do not actually trigger replica reassignment.
     */
    public ReassignPartitionsOptions validateOnly(boolean validateOnly)

    public long throttle() {
                 ReassignPartitionsOptions options)

Where:

Code Block
languagejava
linenumberstrue
public class ReassignPartitionsOptions {

return throttle;
    public boolean validateOnly()}

    /**
     * Validate<p>Set the requestthrottle only:rate Doand notthrottled actuallyreplicas triggerfor replicathe reassignmentreassignments.
     */
 The given  publicthrottle ReassignPartitionsOptionsis validateOnly(boolean validateOnly)

    public long timeoutMs()

    /**in bytes/second and should be at least 1 KB/s.
     * SetInterbroker replication atraffic timeoutwill forbe thethrottled startingto ofapproximately the reassignmentgiven value. 
     * Note this timeout does not include the time take to actuallyUse Long.MAX_VALUE if the reassignment should not be throttled.</p>
     *
     * move<p>A replicaspositive betweenthrottle brokers.
is equivalent    */to setting:</p>
    public ReassignPartitionsOptions timeoutMs(long timeoutMs)


 }
public class ReassignPartitionsResult {
    public Map<TopicPartition, KafkaFuture<Void>> values();
    public KafkaFuture<Void> all();
}

Network Protocol: ReassignPartitionsRequest and ReassignPartitionsResponse

...

Notes:

  • The request can be sent to any broker.
  • The request  requires the Alter operation on the CLUSTER resource, since it can require significant inter-broker communication.
  • The request is subject to the CreateTopicPolicy of the broker as configured by the broker's create.topic.policy.class.name config property. This is to ensure that the policy applies to topics modified after creation.
  • After validating the request the broker writes reassignment JSON to the /admin/reassign_partitions znode
No Format
ReassignPartitionsRequest => [topic_reassignments] timeout validate_only
  topic_reassignments => topic [partition_reassignments]
    topic => STRING
    partition_reassignments => partition_id [broker]
      partition_id => INT32
      broker => INT32
  timeout => INT32
  validate_only => BOOLEAN

Where

FieldDescription
topicthe name of a topic
partition_ida partition of that topic
brokera broker id
timeoutThe maximum time to await a response in ms.
validate_onlywhen true: validate the request, but don't actually reassign the partitions

...

* <ul>
     *     <li>The leader and follower throttled rates to the given value given by throttle.</li>
     *     <li>The leader throttled replicas of each topic in the request to include the existing brokers having
     *     replicas of the partitions in the request.</li>
     *     <li>The follower throttled replicas of each topic in the request to include the new brokers
     *     for each partition in that topic.</li>
     * </ul>
     *
     * <p>The value of {@link #autoRemoveThrottle()} will determine whether these
     * throttles will be removed automatically when the reassignment completes.</p>
     *
     * @see AdminClient#alterInterbrokerThrottledRate(int, long, long)
     * @see AdminClient#alterInterbrokerThrottledReplicas(Map)
     */
    public ReassignPartitionsOptions throttle(long throttle) { ... }

    public boolean autoRemoveThrottle() { ... }

    /**
     * True to automatically remove the throttle at the end of the current reassignment.
     */
    public ReassignPartitionsOptions autoRemoveThrottle(boolean autoRemoveThrottle) { ... }
}

 }
public class ReassignPartitionsResult {
    public Map<TopicPartition, KafkaFuture<Void>> values();
    public KafkaFuture<Void> all();
}

Network Protocol: ReassignPartitionsRequest and ReassignPartitionsResponse

Anchor
ReassignPartitionsRequest
ReassignPartitionsRequest
ReassignPartitionsRequest initiates the movement of replicas between brokers, and is the basis of the AdminClient.reassignPartitions() method

Notes:

  • The request must be sent to the controller.
  • The request  requires the Alter operation on the CLUSTER resource, since it can require significant inter-broker communication.
  • The request will be subject to a policy, as described in KIP-201.
No Format
ReassignPartitionsRequest => [topic_reassignments] timeout validate_only throttle remove_throttle
  topic_reassignments => topic [partition_reassignments]
    topic => STRING
    partition_reassignments => partition_id [broker]
      partition_id => INT32
      broker => INT32
  timeout => INT32
  validate_only => BOOLEAN
  throttle => INT64
  remove_throttle => BOOLEAN

Where

FieldDescription
topicthe name of a topic
partition_ida partition of that topic
brokera broker id
timeoutThe maximum time to await a response in ms.
validate_onlywhen true: validate the request, but don't actually reassign the partitions

Algorithm

  1. The controller validates the request against configured authz, policy etc.
  2. The controller computes set of topics in the request, and writes this as JSON to the new /admin/throttled_replicas_removal znode
  3. The controller then updates the existing leader.replication.throttled.replicas and follower.replication.throttled.replicas properties of each topic config.
  4. The controller computes the union of 1) The brokers currently hosting replicas of the topic partitions in the request 2) the brokers assigned to host topic partitions in the request, and write this as JSON to the new /admin/throttled_rates_removal znode.
  5. The controller then updates the existing leader.replication.throttled.rates and follower.replication.throttled.rates properties of each broker config.
  6. The controller writes reassignment JSON to the /admin/reassign_partitions znode

Anchor
ReassignPartitionsResponse
ReassignPartitionsResponse
ReassignPartitionsResponse describes which partitions in the request will be moved, and what was wrong with the request for those partitions which will not be moved.

No Format
ReassignPartitionsResponse => throttle_time_ms [reassign_partition_result]
  throttle_time_ms => INT32
  reassign_partition_result => topic [partition_error]
    topic => STRING
    partition_error => partition_id error_code error_message
      partition_id => INT32
      error_code => INT16
      error_message => NULLABLE_STRING

Where

FieldDescription
throttle_time_msduration in milliseconds for which the request was throttled
topica topic name from the request
partition_ida partition id for that topic, from the request
error_codean error code for that topic partition
error_messagemore detailed information about any error for that topic

Anticipated errors:

  • CLUSTER_AUTHORIZATION_FAILED (31) Authorization failed
  • POLICY_VIOLATION(44) The request violated the configured policy
  • INVALID_TOPIC_EXCEPTION (17) If the topic doesn't exist
  • UNKNOWN_MEMBER_ID (25) If any broker ids in the partition_reassignments included an unknown broker id
  • INVALID_REQUEST (42) If duplicate topics appeared in the request
  • PARTITION_REASSIGNMENT_IN_PROGRESS (new) If the reassignment cannot be started because a reassignment is currently running (i.e. the /admin/reassign_partitions znode exists)
  • INVALID_THROTTLE (new) If the given throttle is <=0.
  • INVALID_REPLICA_ASSIGNMENT (39) If a partition, replica or broker id in the partition_assignment doesn't exist or is incompatible with the requested num_partitions and /or replication_factor. The error_message would contain further information.
  • NONE (0) reassignment has started

AdminClient: alterInterbrokerThrottledRates()

Code Block
languagejava
linenumberstrue
    /** 
     * Change the rate at which interbroker replication is throttled, replacing existing throttled rates. 
     * The given {@code leaderRate} is the throttled rate when the broker is acting as leader.
     * The given {@code followerRate} is the throttled rate when the broker is acting as follower.
     * For the throttled rates to take effect, the given broker must also be present in the
     * list of throttled replicas, which can be set by {@link #alterInterbrokerThrottledReplicas()}.
     * The throttled rates will be removed at the end of the current reassignment.
     */
    AlterInterbrokerThrottledRateResult alterInterbrokerThrottledRate(Map<Integer, ThrottledRate>, AlterInterbrokerThrottledRateOptions options)
    // HOW do I get the current throttled rate?
    class AlterInterbrokerThrottledRateOptions {
        boolean autoRemoveThrottle()
        ReassignPartitionsOptions autoRemoveThrottle(boolean)
    }

Where:

Code Block
public class ThrottledRate {
  public ThrottledRate(long leaderRate, long followerRate) { ... }
  long leaderRate() { ... }
  long followerRate() { ... }

}

public class AlterInterbrokerThrottledRateOptions extends AbstractOptions<AlterInterbrokerThrottledRateOptions> {

    public boolean autoRemoveThrottle() { ... }

    /**
     * True to automatically remove the throttle at the end of the current reassignment.
     */
    public AlterInterbrokerThrottledRateOptions autoRemoveThrottle(boolean autoRemoveThrottle) { ... }
}

public class AlterInterbrokerThrottledRateResult {
    // package-access ctor

    public Map<Integer, KafkaFuture<Void>> values() { ... }

    public KafkaFuture<Void> all() { ... }
}

Network API: AlterInterbrokerThrottledRatesRequest and AlterInterbrokerThrottledRatesResponse

No Format
AlterInterbrokerThrottledRatesRequest => [broker_throttles] remove_throttle timeout validate_only
  broker_throttles => broker_id leader_rate follower_rate
    broker_id => INT32
    leader_rate => INT64
    follower_rate => INT64
  timeout => INT32
  validate_only => BOOLEAN
  remove_throttle => BOOLEAN

Algorithm:

  1. The controller validates the brokers and rates in the request.
     
  2. The controller gets the current value of the /admin/throttled_rates_removal znode, forms the union of those brokers with those in the request and updates the /admin/throttled_rates_removal znode with JSON representation of this union
  3. The controller then subtracts the brokers in the request from the current brokers and removes the leader.replication.throttled.rates and follower.replication.throttled.rates properties from each broker config
  4. The controller then, for each broker in the request, adds the leader.replication.throttled.rates and follower.replication.throttled.rates properties to each broker config.
  5. The controller then updates /admin/throttled_rates_removal znode with JSON representation of brokers in the request.
No Format
AlterInterbrokerThrottledRatesResponse => [broker_error]
  broker_error => broker_id error_code error_messgae
      broker_id => INT32
      error_code => INT16
      error_message => NULLABLE_STRING

AdminClient: alterInterbrokerThrottledReplicas()

Code Block
languagejava
linenumberstrue
/**
 * Set the partitions and brokers subject to the
 * {@linkplain #alterInterbrokerThrottledRate(Map)
 * interbroker throttled rate}.
 * The brokers specified as the {@link ThrottledReplicas#leaders()} corresponding to a
 * topic partition given in {@code replicas} will be subject to the leader throttled rate
 * when acting as the leader for that partition.
 * The brokers specified as the {@link ThrottledReplicas#followers()} corresponding to a
 * topic partition given in {@code repicas} will be subject to the follower throttled rate
 * when acting as the follower for that partition.
 * If the given {@code replicas} is null then the throttle will apply to all topic partitions on all brokers.
 * TODO No! It should depend on the current reassignment
 *
 * The throttle will be automatically removed at the end of the current reassignment,
 * unless overridden in the given options.
 *
 * The current throttled replicas can be obtained via {@link #describeConfigs(Collection)} with a
 * ConfigResource with type {@link ConfigResource.Type#TOPIC TOPIC} and name "leader.replication.throttled.replicas"
 * or "follower.replication.throttled.replicas".
 */
public abstract AlterInterbrokerThrottledReplicasResult alterInterbrokerThrottledReplicas(
        Map<TopicPartition, ThrottledReplicas> replicas,
        AlterInterbrokerThrottledReplicasOptions options);

Where:

Code Block
public class ThrottledReplicas {

    public ThrottledReplicas(Collection<Integer> leaders, Collection<Integer> followers) { ... }

    /**
     * The brokers which should be throttled when acting as leader
     */
    public Collection<Integer> leaders() { .. }

    /**
     * The brokers which should be throttled when acting as follower
     */
    public Collection<Integer> followers() { ... }

}

public class AlterInterbrokerThrottledReplicasOptions extends AbstractOptions<AlterInterbrokerThrottledReplicasOptions> {

    public boolean autoRemoveThrottle() { ... }

    /**
     * True to automatically remove the throttle at the end of the current reassignment.
     */
    public AlterInterbrokerThrottledReplicasOptions autoRemoveThrottle(boolean autoRemoveThrottle) { ... }
}

public class AlterInterbrokerThrottledReplicasResult {
    // package-access ctor

    public Map<TopicPartition, KafkaFuture<Void>> values() { ... }

    public KafkaFuture<Void> all() { ... }
}

Network API: AlterInterbrokerThrottledReplicasRequest and AlterInterbrokerThrottledReplicasResponse

No Format
AlterInterbrokerThrottledRatesRequest => [topic_throttles] remove_throttle timeout validate_only
  topic_throttles
No Format
ReassignPartitionsResponse => throttle_time_ms [reassign_partition_result]
  throttle_time_ms => INT32
  reassign_partition_result => topic [partition_errorthrottles]
    topic => STRING
    partition_errorthrottles => partition_id error_code error_message[broker_id]
      partition_id => INT32
      partitionbroker_id => INT32
  timeout => INT32
      error_codevalidate_only => INT16BOOLEAN
      error_messageremove_throttle => NULLABLE_STRING

Where

FieldDescription
throttle_time_msduration in milliseconds for which the request was throttled
topica topic name from the request
partition_ida partition id for that topic, from the request
error_codean error code for that topic partition
error_messagemore detailed information about any error for that topic

Anticipated errors:

  • CLUSTER_AUTHORIZATION_FAILED (31) Authorization failed
  • POLICY_VIOLATION(44) The request violated the configured policy
  • INVALID_TOPIC_EXCEPTION (17) If the topic doesn't exist
  • UNKNOWN_MEMBER_ID (25) If any broker ids in the partition_reassignments included an unknown broker id
  • INVALID_REQUEST (42) If duplicate topics appeared in the request
  • PARTITION_REASSIGNMENT_IN_PROGRESS (new) If the reassignment cannot be started because a reassignment is currently running (i.e. the /admin/reassign_partitions znode exists)
  • INVALID_REPLICA_ASSIGNMENT (39) If a partition, replica or broker id in the partition_assignment doesn't exist or is incompatible with the requested num_partitions and /or replication_factor. The error_message would contain further information.
  • NONE (0) reassignment has started

AdminClient: describeConfigs() and alterConfigs()

...

  1. A previous call to reassignPartitions()  has been made and the user wants to apply or change the throttle.
  2. A previous call to reassignPartitions() is complete and the user wants to remove the throttle.

Inter-broker throttling is implemented via DynamicConfigs on the broker (DynamicConfigs are configs "which have no physical manifestation in the server.properties and can only be set dynamically"):

  • leader.replication.throttled.rate
  • follower.replication.throttled.rate

(And via normal topic configs leader.replication.throttled.replicas and follower.replication.throttled.replicas)

The describeConfigs() API already exists, but doesn't current include Broker DynamicConfigs in its results. The describeConfigs() API will therefore be changed to include broker DynamicConfigs in its output.

The alterConfigs() API already exists, but doesn't currently support changing broker configs (dynamic or otherwise). The alterConfigs() API will therefore be changed to allow altering DynamicConfigs. This KIP does not propose to support altering broker configs that are not DynamicConfigs.

This will involve changing the API docs of these methods in the AdminClient.

Network Protocol: (Describe|Alter)Configs(Request|Response)

Thes protocols already exist, but support will be added for describing and altering broker DynamicConfigs, as described above for the corresponding AdminClient methods. The network format will not need to change, but the AlterConfigsResponse will be able to fail with a new error code (CLUSTER_AUTHORIZATION_FAILED) when the request attempts to alter broker configs.

Compatibility, Deprecation, and Migration Plan

  • What impact (if any) will there be on existing users?
  • If we are changing behavior how will we phase out the older behavior?
  • If we need special migration tools, describe them here.
  • When will we remove the existing behavior?

...

BOOLEAN

Algorithm:

  1. The controller validates the partitions and brokers in the request.
     
  2. The controller gets the current value of the /admin/throttled_replicas_removal znode, forms the union of those topics with those in the request and updates the /admin/throttled_rates_removal znode with JSON representation of this union
  3. The controller then subtracts the topics in the request from the current topics and removes the leader.replication.throttled.replicas and follower.replication.throttled.replicas properties from each topic config
  4. The controller then, for each topic in the request, adds the leader.replication.throttled.rates and follower.replication.throttled.rates properties to each topic config.
  5. The controller then updates /admin/throttled_replicas_removal znode with JSON representation of topics in the request.
No Format
AlterInterbrokerThrottledReplicasResponse => [topic_errors]
  topic_errors => topic [partition_errors]
    topic => STRING
    partition_errors => partition_id error_code error_messgae
      partition_id => INT32
      error_code => INT16
      error_message => NULLABLE_STRING

On Controller Failover

The algorithms presented above, using the new znodes, are constructed so that should the controller fail, on election of a new controller ZooKeeper is not left in an inconsistent state where throttles which should be removed automatically are not removed at the end of the reassignment. The recovery algorithm is as follows:

  1. If the /admin/reassign_partitions znode exists we assume a reassignment is on-going and do nothing.
  2. Otherwise, if the /admin/reassign_partitions znode does not exists, we proceed to remove the throttles, as detailed in "Throttle removal" section below.

On reassignment completion

When reassignment is complete:

  1. The /admin/reassign_partitions znode gets removed.
  2. We remove the throttles, as detailed in "Throttle removal" section below.

Throttle removal

  1. If /admin/remove_throttled_replicas is set:
    1. For each of the topics listed in that znode:

      1. Remove the (leader|follower).replication.throttled.replicas propertes for that topic config.

    2. Remove the /admin/remove_throttled_replicas znode

The symmetric algorithm is used for /admin/remove_throttled_rates, only with broker configs.

Compatibility, Deprecation, and Migration Plan

Existing users of the kafka-reassign-partitions.sh will receive a deprecation warning when they use the --zookeeper option. The option will be removed in a future version of Kafka. If this KIP is introduced in version 1.0.0 the removal could happen in 2.0.0.

Implementing AdminClient.alterConfigs() for (dynamic) broker configs was considered as a way of implementing throttle management but this would not support the auto removal feature.

Not supporting passing a throttle in the AdminClient.reassignPartitions() (and just using the APIs for altering throttles) was considered, but:

  • Being able to specify a throttle at the same time at starting the reassignment is very convenient.
  • Race conditions are possible if the APIs requires throttles set up before reassignment starts. What if reassignPartitions() doesn't get called, or none of the partitions in the call can be be reassigned?

Rejected Alternatives

If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.

...

Similarly a alterReplicationFactors() method, separate from reassignPartitions() was considered, but both require a partition to brokers assignment, and both are implemented in the same way (by writing to the /admin/reassign_partitions znode), so there didn't seem much point in making an API which distinguished them.

Algorithms making use of the ZooKeeper "multi" feature for atomic update of multiple znodes were considered. It wasn't clear that these would be better than the algorithms presented above.