Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: AlterTopics field names

...

No Format
AlterTopicsRequest => [AlteredTopic]alter_topic_requests] validate_only
  AlteredTopicalter_topic_requests => Topic, NumPartitions, ReplicationFactor, PartitionAssignmenttopic num_partitions replication_factor partition_assignment
    Topictopic => STRING string               ; the topic name
    NumPartitionsnum_partitions => int32INT32        ; the number of partitions
    ReplicationFactorreplication_factor => int16INT16    ; the replication factor
    PartitionAssignmentpartition_assignment => PartitionIdpartition_id Brokersbrokers
      PartitionIdpartition_id =>int32> INT32        ; the partition id
      Brokersbrokers => [int32INT32]           ; 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 An empty PartitionAssignment 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 an AlterTopicsRequest to the leader for a given partition. Any broker will do.

...

No Format
AlterTopicsResponse => [AlteredTopic]
  AlteredTopic => Topic Error Detailthrottle_time_ms [topic_errors]
  throttle_time_ms => INT32  ; duration in milliseconds for which the request was throttled
  topic_errors => topic error_code error_message
    Topictopic => stringSTRING          ; the topic name
    ErrorCodeerror_code => int16 INT16      ; the error code for altering this topic
    Detailerror_message => string         ; detailed error information 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_codePossible Error Codes:

  • CLUSTER_AUTHORIZATION_FAILED (31) Authorization failed
  • INVALID_TOPIC_EXCEPTION (17) If the topic doesn't exist
  • INVALID_PARTITIONS (37) If the NumPartitions num_partitions was invalid
  • INVALID_REPLICATION_FACTOR (38) If the ReplicationFactor replication_factor was invalid
  • UNKNOWN_MEMBER_ID (25) If any broker ids in the PartitionAssignment 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 PartitionAssignment the partition_assignment doesn't exist or is incompatible with the requested NumPartitions requested num_partitions and /or ReplicationFactoror replication_factor. The Detail error_message would contain further information.
  • NONE (0) If the request was successful and the alteration/reassignment has been started.

...