...
No Format |
---|
AlterTopicsRequest => [alter_topic_requests] validate_only alter_topic_requests => topic num_partitions replication_factor partition_assignment topic => STRING ; the topic name num_partitions => INT32 ; the number of partitions replication_factor => INT16 ; the replication factor partition_assignment => partition_id brokers partition_id => INT32 brokers ; 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
FIELD | DESCRIPTION |
---|---|
topic | the topic name |
num_partition | the number of partitions. A |
replication_factor | the replication factor. A |
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]
validate_only => BOOLEAN |
Where
FIELD | DESCRIPTION |
---|---|
topic | the topic name |
num_partition | the number of partitions. A |
replication_factor | the replication factor. A |
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.
Anchor | ||||
---|---|---|---|---|
|
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 |
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 error_message ; the error code for altering this topic error_message => NULLABLE_STRING ; detailed error information=> NULLABLE_STRING |
Where
Field | Description |
---|---|
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 |
...
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
...
No Format |
---|
ReplicaStatusRequest => [Replica]
Replica => Topic Partition Broker
Topic => string ; a topic name
Partition => int32 ; a partition id of this topic
Broker => int32 ; a follower broker id for this partition |
...
concurrently, hence the addition of the PARTITION_REASSIGNMENT_IN_PROGRESS
error code.
Network Protocol: ReplicaStatusRequest and ReplicaStatusResponse
Anchor | ||||
---|---|---|---|---|
|
ReplicaStatusRequest
requests information about the progress of a number of replicas.No Format |
---|
ReplicaStatusRequest => [replica_status_requests]
replica_status_requests => topic partition_id broker
topic => STRING
partition_id => INT32
broker => INT32 |
Where
Field | Description |
---|---|
topic | a topic name |
partition_id | a partition id of this topic |
broker | a follower broker id for this partition |
Anchor | ||||
---|---|---|---|---|
|
No Format |
---|
ReplicaStatusResponse => [replica_status]
replica_status => topic partition_id broker error_code status_time lag
topic => STRING
partition_id => INT32
broker => INT32
error_code => INT16
status_time => INT64
lag => INT64 |
Where
Field | Description |
---|---|
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 knownINVALID_PARTITIONS
(37) ThePartion
Thepartion
_id
of the given topic is not validUNKNOWN_MEMBER_ID
(25) The givenBroker
givenbroker
id is not known.UNKNOWN_TOPIC_OR_PARTITION
(3) The givenBroker
givenbroker
is not a follower for the partition identified byTopic
topic
,Partition
partition
.NONE
(0) if the status request completed normally,
...