...
MetadataResponse => [brokers] controllerId [topic_metadata] |
Adds rack, controller_id, and is_internal to the version 0 response.
The behavior of the replicas and isr arrays will be changed in order to support the admin tools, and better represent the state of the cluster:
- In version 0, if a broker is down the replicas and isr array will omit the brokers entry and add a REPLICA_NOT_AVAILABLE error code.
- In version 1, no error code will be set and a the broker id will be included in the replicas and isr array.
- Note: A user can still detect if the replica is not available, by checking if the broker is in the returned broker list.
Topic Admin Schema
Create Topics Request (KAFKA-2945): (Voted and Planned for in 0.10.1.0)
CreateTopics 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 => INT16 replica_assignment => partition_id [replicas] partition_id => INT32 replicas => INT32 configs => config_key config_value config_key => STRING config_value => STRING timeout => INT32 |
CreateTopicsRequest
is a batch request to initiate topic creation with either predefined or automatic replica assignment and optionally topic configuration.
Request semantics:
- Must be sent to the controller broker
- If there are multiple instructions for the same topic in one request an InvalidRequestException will be logged on the broker and a single error code for that topic will be returned to the client
- This is because the list of topics is modeled server side as a map with TopicName as the key
- The principal must be authorized to the "Create" Operation on the "Cluster" resource to create topics.
- Unauthorized requests will receive a ClusterAuthorizationException
Only one from
ReplicaAssignment or (num_partitions +
replication_factor),
can be defined in one instruction.- If both parameters are specified an InvalidRequestException will be logged on the broker and an error code for that topic will be returned to the client
- In the case
ReplicaAssignment
is defined number of partitions and replicas will be calculated from the supplied replica_assignment. - In the case of defined (num_partitions + 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.
- 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.
- Complete means the local topic metadata cache been completely populated and all partitions have leaders
- Setting a timeout <= 0 will validate arguments and trigger the create topics and return immediately.
- This is essentially the fully asynchronous mode we have in the Zookeeper tools today.
- The error code in the response will either contain an argument validation exception or a timeout exception. If you receive a timeout exception, because you asked for 0 timeout, you can assume the message was valid and the topic creation was triggered.
- The request is not transactional.
- If an error occurs on one topic, the others could still be created.
- Errors are reported independently.
QA:
- Why is CreateTopicsRequest 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?
- See Cluster Consistent Blocking below
- Why require the request to go to the controller?
- The controller is responsible for the cluster metadata and its propagation
- See Request Forwarding below
...