Versions Compared

Key

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

...

New network protocol APIs will be added:

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

...

Anchor
alterPartitionCount
alterPartitionCount
The To support kafka-topics.sh --alter --partitions ... the following methods will be added to AdminClient to support changing topics' partition counts

...

Anchor
alterReplicationCount
alterReplicationCount
The To support kafka-topics.sh --alter --replication-factor ... the following methods will be added to AdminClient to support changing topics' replication factors.

...

Anchor
reassignPartitions
reassignPartitions
The To support kafka-topics.sh --alter --replica-assignment ... and kafka-reassign-partitions.sh the following methods will be added to AdminClient to support changing the brokers hosting the partitions of a topic

...

Partition reassignment is a long running operation, and the ReassignPartitionsResult indicates only that the reassignment has been started, not that the reassignment has been completed. The new replicaStatus() method can be used to check progress and completion of the reassignment.

TODO quotas

AdminClient: replicaStatus()

...

Anchor
AlterPartitionCountsRequest
AlterPartitionCountsRequest
The AlterPartitionCountsRequest is used to change the partition count for a batch of topics, and is the basis for the  AdminClient.alterPartitionCounts() method.

No Format
AlterPartitionCountsRequest => [topic_partition_count]
  topic_partition_count => topic partition_count
    topic => STRING
    partition_count => INT32
  // TODO: validate_only and/or timeout flags?

...

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.

Anchor
AlterPartitionCountsResponse
AlterPartitionCountsResponse
The response provides an error code and message for each of the topics present in the request.

No Format
No Format
AlterPartitionCountsResponse => throttle_time_ms [topic_partition_count_error]
  topic_partition_count_error => topic error_code error_message
    topic => STRING
    error_code => INT16
    error_message => NULLABLE_STRING

...

Anchor
AlterReplicationFactorsRequest
AlterReplicationFactorsRequest
The AlterReplicationFactorRequest is used to change the replication factor for a batch of topics, and is the basis for the AdminClient.alterReplicationFactors() method.

No Format
AlterReplicationFactorsRequest => [topic_replication_factor]
  topic_replication_factor => topic replication_factor
    topic => STRING
    replication_factor => INT16
  // TODO: validate_only and/or timeout flags?

...

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.

Anchor
AlterReplicationFactorResponse
AlterReplicationFactorResponse

...

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

No Format
ReassignPartitionsRequest => [reassigned_topic]
  reassigned_topic => topic [reassigned_partition]
    topic => STRING
    reassigned_partition => partitiod_id [broker]
      partition_id => INT32
      broker => INT32
  validate_only => BOOLEAN

...

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.

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.

...