Versions Compared

Key

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

...

 

CreateTopic Request (Version: 0) => [create_topic_requests] timeout 
  create_topic_requests => topic num_partitions replication_factor [replica_assignment] [configs] 
    topic => STRING
    num_partitions => INT32
    replication_factor => INT32
    replica_assignment => partition_id [replicas] 
      partition_id => INT32
      replicas => INT32
    configs => config_key config_value 
      config_key => STRING
      config_value => STRING
  timeout => INT32

CreateTopicRequest is a batch request to initiate topic creation with either predefined or automatic replica assignment and optionally topic configuration.

Request semantics:

  1. Must be sent to the controller broker
  2. Multiple instructions for the same topic in one request will be silently ignored, only the last from the list will be executed.
    • This is because the list of topics is modeled server side as a map with TopicName as the key
  3. The principal must be authorized to the "Create" Operation on the "Cluster" resource to create topics. 
    • Unauthorized requests will receive a ClusterAuthorizationException
  4. Only one from ReplicaAssignment or (Partitionsnum_partitions + ReplicationFactorreplication_factor), can be defined in one instruction. If both parameters are specified - ReplicaAssignment takes precedence replica_assignment takes precedence.

    • In the case ReplicaAssignment is defined number of partitions and replicas will be calculated from the supplied ReplicaAssignmentreplica_assignment
    • In the case of defined (Partitionsnum_partitions + ReplicationFactor replication_factor) replica assignment will be automatically generated by the server.
    • One or the other must be defined. The existing broker side auto create defaults will not be used (default.replication.factor, num.partitions). The client implementation can have defaults for these options when generating the messages.
    • The first replica in [replicas] is assumed to be the preferred leader. This matches current behavior elsewhere.
  5. Setting a timeout > 0 will allow the request to block until the topic metadata is "complete" on the controller node.
    • Complete means the local topic metadata cache been completely populated and all partitions have leaders
      • The topic metadata is updated when the controller sends out update metadata requests to the brokers
    • If a timeout error occurs, the topic could still be created successfully at a later time. Its up to the client to query for the state at that point.
  6. The request is not transactional. 
    1. If an error occurs on one topic, the others could still be created.
    2. Errors are reported independently.

QA:

  • Why is CreateTopicRequest a batch request?
    • Scenarios where tools or admins want to create many topics should be able to with fewer requests
    • Example: MirrorMaker may want to create the topics downstream
  • What happens if some topics error immediately? Will it return immediately
    • The request will block until all topics have either been created, errors, or the timeout has been hit
    • There is no "short circuiting" where 1 error stops the other topics from being created
  • Why implement "partial blocking" instead of fully async or fully consistent?
  • Why require the request to go to the controller?
    • The controller is responsible for the cluster metadata and its propogation 
    • See Request Forwarding below

...