Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: Minor

...

Discussion thread: here (when initially misnumbered as KIP-178) and here (when assigned KIP-179)

JIRA: here

Jira
serverASF JIRA
columnskey,summary,type,created,updated,due,assignee,reporter,priority,status,resolution
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-5601
Jira
serverASF JIRA
columnskey,summary,type,created,updated,due,assignee,reporter,priority,status,resolution
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-5561

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

...

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

Code Blocknoformat
AlterTopicsRequest => [AlteredTopic]
  AlteredTopic => Topic, PartitionsNumPartitions, ReplicasReplicationFactor, PartitionAssignment
    Topic => string               ; the topic name
    PartitionsNumPartitions => int32        ; the number of partitions
    ReplicasReplicationFactor => int32    ; the replication factor
    PartitionAssignment => PartitionPartitionId Brokers
      PartitionPartitionId =>int32         ; the partition id
      Brokers => [int32]          ; the ids of the assigned brokers for this partition

Partitions NumPartitions or Replicas ReplicationFactor of -1 that would mean "no change". An empty PartitionAssignment would mean that the broker should calculate a suitable assignment. Such broker calculated assignment is unlikely to be balanced.

...

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

Code Blocknoformat
AlterTopicsResponse => [AlteredTopic]
  AlteredTopic => Topic Error Detail
    Topic => string          ; the topic name
    ErrorErrorCode => int32       ; the error code for altering this topic
    Detail => string         ; detailed error information

...

  • CLUSTER_AUTHORIZATION_FAILED (31) Authorization failed
  • INVALID_TOPIC_EXCEPTION (17) If the topic doesn't exist
  • INVALID_PARTITIONS (37) If the Partitions NumPartitions was invalid
  • INVALID_REPLICATION_FACTOR (38) If the Replicas ReplicationFactor was invalid
  • UNKNOWN_MEMBER_ID (25) If any broker ids in the PartitionAssignment included an unknown broker id
  • PARTITION_REASSIGNMENT_IN_PROGRESS (new)
  • INVALID_REPLICA_ASSIGNMENT (39) If a partition, replica or broker id in the PartitionAssignment doesn't exist or is incompatible with the requested Partitions and /or Replicasrequested NumPartitions and /or ReplicationFactor. The Detail 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 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.

Code Blocknoformat
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

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

Code Blocknoformat
ReplicaStatusResponse => [ReplicaStatus]
  ReplicaStatus => Topic Partition Broker Error StatusTime, IsrTime, FollowerTime Lag MaxLag
    Topic => string         ; the topic name
    Partition => int32      ; the partition id
    Broker => int32         ; the follower broker id
    Error => int16          ; an error code
    StatusTime -> int64     ; the time the status was current
    Lag => int64            ; the lag (#messages)

...