Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

Note this was initially erroneously assigned as KIP-178, which was already taken, and has been reassigned KIP-179.

Table of Contents

Status

Current state: Under Discussion [One of "Under Discussion", "Accepted", "Rejected"]

...

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

Motivation

Describe the problems you are trying to solve.

...

Secondly, 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.

...

Two new network protocol APIs will be added:

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

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

Describe the new thing you want to do in appropriate detail. This may be fairly extensive and have large subsections of its own. Or it may be a few sentences. Use judgement based on the scope of the change.

kafka-reassign-partitions.sh and ReassignPartitionsCommand

The --zookeeper option will be retained and will:

...

In all other respects, the public API of ReassignPartitionsCommand will not be changed.

AdminClient: alterTopics()

Anchor
alterTopics
alterTopics
The following methods will be added to AdminClient to support the ability to reassign partitions:

Code Block
/**
 * Request alteration of the given topics. The request can change the number of 
 * partitions, replication factor and/or the partition assignments. 
 * This can be a long running operation as replicas are migrated between brokers, 
 * therefore the returned result conveys whether the alteration has been 
 * started, not that it is complete. Progress information 
 * can be obtained by calling the lead broker's 
 * {@link #replicaStatus(Collection)}.
 */
public AlterTopicsResult alterTopics(Collection<AlteredTopic> alteredTopics)
public AlterTopicsResult alterTopics(Collection<AlteredTopic> alteredTopics,  AssignPartitionOptionsAlterTopicsOptions options)

Where:

Code Block
public class AlteredTopic {
    public AlteredTopic(String name, int numPartitions, int replicationFactor, Map<Integer,List<Integer>> replicasAssignment) {
        // ...
    }
    /** The name of the topic to alter. */
    public String name();
    /** The new number of partitions, or -1 if the number of partitions should not be changed. */
    public int numPartitions();
    /** The new replication factor, or -1 if the replication factor should not be changed. */
    public intshort replicationFactor();
    /** 
     * The new assignments of partition to brokers, or the empty map 
     * if the broker should assign replicas automatically. 
     */
    Map<Integer,List<Integer>> replicasAssignment();
}

public class AlterTopicsResultAlterTopicsOptions {
    public // package-access constructorAlterTopicsOptions validateOnly(boolean validateOnly);
    /**public A mapping of the name of a requested topic to the error for that topic */boolean validateOnly();
    public AlterTopicsOptions timeoutMs(long timeoutMs);
	public long timeoutMs();
 }

public class AlterTopicsResult {
    KafkaFuture<Map<String, Errors>> result(); // package-access constructor
    /** The A mapping of the name of a requested topic to the error for a givethat topic. */
	KafkaFuture<Errors> topicResult(String topicName);
    Map<String, KafkaFuture<Void>> values(); 
    /** Return a future which succeeds if all the topic alterations were accepted. */
	KafkaFuture<Void> all();
}

AdminClient: replicaStatus()

Anchor
replicaStatus
replicaStatus
The following methods will be added to AdminClient to support the progress reporting functionality:

Code Block
/**
 * Query the replication status of the given partitions of which. this
 */
public broker is the leader.
 */
public ReplicaStatusResult ReplicaStatusResult replicaStatus(Collection<TopicPartition> replicas)   
public ReplicaStatusResult replicaStatus(Collection<TopicPartition> replicas,  ReplicaStatusOptions options)

Where:

Code Block
public class ReplicaStatusOptions {
    
}

public class ReplicaStatusResult {
    public KafkaFuture<Map<TopicPartition, List<ReplicaStatus>>> all()
}

/** 
 * Represents the replication status of a partition 
 * on a particular broker.
 */ 
public class ReplicaStatus {
    /** The topic about which this is the status of */
    String topic()
    /** The partition about which this is the status of */
    int partition()
    /** The broker about which this is the status of */
    int broker()
    
    /** 
     * The time (as milliseconds since the epoch) that 
     * this status data was collected. In general this may
     * be some time before the replicaStatus() request time.
     */
    public long statusTime()
    
    /** 
     * The number of messages that the replica on this broker is behind
     * the leader.
     */
    public long lag()
    
}

Authorization

With broker-mediated reassignment it becomes possible limit the authority to perform reassignment to something finer-grained than "anyone with access to zookeeper".

The reasons for reassignment are usually operational. For example, migrating partitions to new brokers when expanding the cluster, or attempting to find a more balanced assignment (according to some notion of balance). These are cluster-wide considerations and so authority should be for the reassign operation being performed on the cluster. Therefore alterTopics() will require ClusterAction on the CLUSTER.

replicaStatus() will require Describe on the CLUSTER (or the TOPIC?).

Network Protocol: AlterTopicsRequest

...

and AlterTopicsResponse

Anchor
AlterTopicsRequest
AlterTopicsRequest
An AlterTopicsRequest will initiate the process of topic alteration/partition reassignment

No Format
AlterTopicsRequest => [alter_topic_requests] validate_only
  alter_topic_requests => topic num_partitions replication_factor [partition_assignment]
    topic => STRING
    num_partitions => INT32
    replication_factor => INT16
    ; the topic namepartition_assignment => partition_id brokers
    num  partition_partitionsid => INT32
      brokers  ; the number of partitions=> [INT32]
    replicationvalidate_factoronly => INT16    ; the replication factorBOOLEAN
   timeout partition_assignment => partition_id brokers
      partition_id => INT32        ; the partition id
      brokers => [INT32]           ; the ids of the assigned brokers for this partition
  validate_only => BOOLEAN         ; true to just validate the request, but not actually alter the topics

Where

FIELDDESCRIPTION
topic

the topic name

num_partition

the number of partitions. A num_partitions of -1 that would mean "no change"

replication_factor

the replication factor. A  replication_factor of -1 would mean "no change"

partition_id

the partition id

brokers

the ids of the assigned brokers for this partition

validate_only

true to just validate the request, but not actually alter the topics

An empty partition_assignment would mean that the broker should calculate a suitable assignment. Such broker calculated assignment is unlikely to be balanced.

It is not necessary to send an AlterTopicsRequest to the leader for a given partition. Any broker will do.

...

INT32

Where

FIELDDESCRIPTION
topic

the topic name

num_partition

the number of partitions. A num_partitions of -1 that would mean "no change"

replication_factor

the replication factor. A  replication_factor of -1 would mean "no change"

partition_id

the partition id

brokers

the ids of the assigned brokers for this partition

validate_only

true to just validate the request, but not actually alter the topics

timeoutthe timeout, in ms, to wait for the topic to be altered.

An empty partition_assignment would mean that the broker should calculate a suitable assignment. Such broker calculated assignment is unlikely to be balanced.

It is not necessary to send an AlterTopicsRequest to the leader for a given partition. Any broker will do.

Anchor
AlterTopicsResponse
AlterTopicsResponse
The AlterTopicsResponse enumerates those topics in the request, together with any error in initiating alteration:

No Format
AlterTopicsResponse => throttle_time_ms [topic_errors]
  throttle_time_ms => INT32
  topic_errors => topic error_code error_message
    topic => STRING
    error_code => INT16
    error_message => NULLABLE_STRING

Where

FieldDescription
throttle_time_ms

duration in milliseconds for which the request was throttled

topic

the topic name

error_code

the error code for altering this topic

error_message

detailed error information

Possible values for error_code:

  • CLUSTER_AUTHORIZATION_FAILED (31) Authorization failed
  • INVALID_TOPIC_EXCEPTION (17) If the topic doesn't exist
  • INVALID_PARTITIONS (37) If the num_partitions was invalid
  • INVALID_REPLICATION_FACTOR (38) If the replication_factor was invalid
  • UNKNOWN_MEMBER_ID (25) If any broker ids in the partition_assignment included an unknown broker id
  • INVALID_REQUEST (42) If trying to modify the partition assignment and the number of partitions or the partition assignment and the replication factor in the same request. Or if duplicate topics appeared in the request.
  • PARTITION_REASSIGNMENT_IN_PROGRESS (new)
  • 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) If the request was successful and the alteration/reassignment has been started.

As currently, it will not be possible to have multiple reassignments running concurrently, hence the addition of the PARTITION_REASSIGNMENT_IN_PROGRESS error code.

Policy

The existing CreateTopicPolicy can be used to apply a cluster-wide policy on topic configuration at the point of creation via the create.topic.policy.class.name config property. To avoid an obvious loophole, it is necessary to also be able to apply a policy to topic alteration. Maintaining two separate policies in sync is a burden both in terms of class implementation and configuring the policy. It seems unlikely that many use cases would require a different policy for alteration than creation. On the other hand, just applying the CreateTopicPolicy to alterations is undesirable because:

  • Its name doesn't convey that it would be applied to alterations too
  • Its API (specifically its RequestMetadata member class) includes topic configs (i.e. Map<String, String>) which is not part of the API for topic alteration even though it is part of the API for topic creation.
  • It prevents any use cases which legitimately did need to apply a different policy for alteration than creation.

Finding a balance between compatibility with existing deployments, and not opening the loophole is difficult.

The existing create.topic.policy.class.name config would continue to work, and would continue to name an implementation of CreateTopicPolicy. That policy would be applied to alterations automatically. The topic's config would be presented to the validate() method (via the RequestMetadata) even though it's not actually part of the AlterTopicsRequest. The documentation for the interface and config property would be updated.

No Format
AlterTopicsResponse => throttle_time_ms [topic_errors]
  throttle_time_ms => INT32  ; duration in milliseconds for which the request was throttled
  topic_errors => topic error_code error_message
    topic => STRING          ; the topic name
    error_code => INT16      ; the error code for altering this topic
    error_message => NULLABLE_STRING  ; detailed error information

Where

FieldDescription
throttle_time_ms

duration in milliseconds for which the request was throttled

topic

the topic name

error_code

the error code for altering this topic

error_message

detailed error information

Possible values for error_code:

  • CLUSTER_AUTHORIZATION_FAILED (31) Authorization failed
  • INVALID_TOPIC_EXCEPTION (17) If the topic doesn't exist
  • INVALID_PARTITIONS (37) If the num_partitions was invalid
  • INVALID_REPLICATION_FACTOR (38) If the replication_factor was invalid
  • UNKNOWN_MEMBER_ID (25) If any broker ids in the partition_assignment included an unknown broker id
  • INVALID_REQUEST (42) If trying to modify the partition assignment and the number of partitions or the partition assignment and the replication factor in the same request.
  • PARTITION_REASSIGNMENT_IN_PROGRESS (new)
  • 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) If the request was successful and the alteration/reassignment has been started.

As currently, it will not be possible to have multiple reassignments running concurrently, hence the addition of the PARTITION_REASSIGNMENT_IN_PROGRESS error code.

...

Network Protocol: ReplicaStatusRequest and 

ReplicaStatusResponse

Anchor
ReplicaStatusRequest
ReplicaStatusRequest
ReplicaStatusRequest requests information about the progress of a number of replicas.

No Format
ReplicaStatusRequest => [Replicareplica_status_requests]
  Replicareplica_status_requests => Topictopic Partitionpartition_id Brokerbroker
    Topictopic => stringSTRING
    partition_id ; a topic name=> INT32
    Partitionbroker => int32  ; a partition id of this topic
    Broker => int32     ; a follower broker id for this partitionINT32

Where

FieldDescription
topic

a topic name

partition_id

a partition id of this topic

broker

a follower broker id for this partition

 

Anchor
ReplicaStatusResponse
ReplicaStatusResponse
The response includes replication information for each of the replicas in the request:

No Format
ReplicaStatusResponse => [ReplicaStatus]
  ReplicaStatus => Topic Partition Broker Error StatusTime, IsrTime, FollowerTime Lag MaxLagreplica_status]
  replica_status  Topic => stringtopic partition_id broker error_code    status_time lag
  ; the topic name
    Partition => int32STRING
      ; the partition id
    Broker => int32         ; the follower broker idpartition_id => INT32
    Errorbroker => int16INT32
    error_code => INT16
    ; an error codestatus_time => INT64
    StatusTimelag -> int64     ; the time the status was current
    Lag => int64            ; the lag (#messages)=> INT64

Where

FieldDescription
topic

the topic name

partition_id

the partition id of this topic

broker

the follower broker id

error_code

an error code

status_time

the time the status was current

lag

the lag (#messages) of this broker, for this partition

Anticipated errors are:

  • CLUSTER_AUTHORIZATION_FAILED (31) Authorization failed. (or the TOPIC?)
  • INVALID_TOPIC_EXCEPTION (17) The topic is not known
  • INVALID_PARTITIONS (37)  The Partion The partion_id of the given topic is not valid
  • UNKNOWN_MEMBER_ID (25) The given Broker given broker id is not known.
  • UNKNOWN_TOPIC_OR_PARTITION (3) The given Broker given broker is not a follower for the partition identified by Topic topic, Partition partition.
  • NONE (0) if the status request completed normally,

Implementation

The AdminClient.replicaStatus() has to be made (an will make the underlying ReplicaStatusRequest sent) to the leader for the given partition. This saves the need for every broker (because any broker could be the --bootstrap-server ) to have knowledge of the replication status of every replica, which would be inefficient in network IO and/or memory use. This means that, in general, the ReassignPartitionsCommand would need to discover other brokers in the cluster (via AdminClient.describeCluster()) and make a separate request to each broker implied by the reassignment in order to report progress.

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?

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.

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.

...