I have updated the wiki and pull request based on the feedback. If there
are no objections I will start a vote at the end of the day.

Details for this implementation can be read here:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-CreateTopicRequest

The updated pull request can be found here (feel free to review):
https://github.com/apache/kafka/pull/1489

Below is the exact content for clarity:

> Create Topics Request (KAFKA-2945
> <https://issues.apache.org/jira/browse/KAFKA-2945>)
>
>
>
> 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:
>
>    1. Must be sent to the controller broker
>    2. If there are multiple instructions for the same topic in one
>    request an InvalidRequestException will be logged on the broker and the
>    client will be disconnected.
>       - 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 (num_partitions + replication_factor
>    ), can be defined in one instruction.
>    - If both parameters are specified an InvalidRequestException will be
>       logged on the broker and the client will be disconnected.
>       - 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.
>    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. 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.
>    7. 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 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
>       
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-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
>       
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-request>
>        below
>
> Create Topics Response
>
>
>
> CreateTopics Response (Version: 0) => [topic_error_codes]
>   topic_error_codes => topic error_code
>     topic => STRING
>     error_code => INT16
>
> CreateTopicsResponse contains a map between topic and topic creation
> result error code (see New Protocol Errors
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-NewProtocolErrors>
> ).
>
>
Thank you,
Grant


On Wed, Jun 15, 2016 at 4:11 PM, Grant Henke <ghe...@cloudera.com> wrote:

> Turns out we already have an InvalidRequestException:
> https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/network/RequestChannel.scala#L75-L98
>
> We just don't map it in Errors.java so it results in an UNKNOWN error when
> sent back to the client.
>
> I will migrate the InvalidRequestException to the client package, add it
> to Errors and use that to signify any protocol parsing/rule errors.
>
>
>
> On Wed, Jun 15, 2016 at 2:55 PM, Dana Powers <dana.pow...@gmail.com>
> wrote:
>
>> On Wed, Jun 15, 2016 at 12:19 PM, Ismael Juma <ism...@juma.me.uk> wrote:
>> > Hi Grant,
>> >
>> > Comments below.
>> >
>> > On Wed, Jun 15, 2016 at 6:52 PM, Grant Henke <ghe...@cloudera.com>
>> wrote:
>> >>
>> >> The one thing I want to avoid is to many super specific error codes. I
>> am
>> >> not sure how much of a problem it really is but in the case of wire
>> >> protocol errors like multiple instances of the same topic, do you have
>> any
>> >> thoughts on the error? Should we make a generic InvalidRequest error
>> and
>> >> log the detailed message on the broker for client authors to debug?
>> >>
>> >
>> > That is a good question. It would be good to get input from client
>> > developers like Dana on this.
>>
>> I think generic error codes are fine if the wire protocol requirements
>> are documented [i.e., no duplicate topics and partitions/replicas are
>> either/or not both]. If I get a broker error at the protocol level
>> that I don't understand, the first place I look is the protocol docs.
>> It may cause a few more emails to the mailing lists asking for
>> clarification, but I think those will be easier to triage than
>> confused emails like "I said create topic with 10 partitions, but I
>> only got 5???"
>>
>> -Dana
>>
>
>
>
> --
> Grant Henke
> Software Engineer | Cloudera
> gr...@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
>



-- 
Grant Henke
Software Engineer | Cloudera
gr...@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke

Reply via email to