Thanks Grant, the design proposal LGTM overall.

One minor question about the error codes in CreateTopic Response, what are
the possible values? I know this may be out of the scope of this KIP, but
would also want to think about how producers should handle each one of them
accordingly, especially if the create topic request is for a batch of
topics, and different error codes are returned.

Guozhang



On Mon, Jun 13, 2016 at 6:54 PM, Grant Henke <ghe...@cloudera.com> wrote:

> Thanks for the review Jun.
>
> You probably want to make it clearer if timeout > 0, what waiting for topic
> > metadata is "complete" means. In the first implementation, it really
> means
> > that the topic metadata is propagated to the controller's metadata cache.
>
>
> I updated the wiki to be more descriptive. Below is the updated text:
>
> 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.
> >
> >
> Thanks,
> Grant
>
>
> On Sun, Jun 12, 2016 at 4:14 PM, Jun Rao <j...@confluent.io> wrote:
>
> > Grant,
> >
> > Thanks for the proposal. It looks good to me.
> >
> > You probably want to make it clearer if timeout > 0, what waiting for
> topic
> > metadata is "complete" means. In the first implementation, it really
> means
> > that the topic metadata is propagated to the controller's metadata cache.
> >
> > Jun
> >
> > On Fri, Jun 10, 2016 at 9:21 AM, Grant Henke <ghe...@cloudera.com>
> wrote:
> >
> > > Now that Kafka 0.10 has been released I would like to start work on the
> > new
> > > protocol messages and client implementation for KIP-4. In order to
> break
> > up
> > > the discussion and feedback I would like to continue breaking up the
> > > content in to smaller pieces.
> > >
> > > This discussion thread is for the CreateTopic request/response and
> server
> > > side implementation. 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
> > >
> > > I have included the exact content below for clarity:
> > >
> > > > Create Topic Request (KAFKA-2945
> > > > <https://issues.apache.org/jira/browse/KAFKA-2945>)
> > > >
> > > >
> > > > CreateTopic Request (Version: 0) => [create_topic_requests] timeout
> > > >   create_topic_requests => topic partitions replication_factor
> > > [replica_assignment] [configs]
> > > >     topic => STRING
> > > >     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 principle 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 (Partitions +
> ReplicationFactor),
> > > can
> > > >    be defined in one instruction. If both parameters are specified -
> > > >    ReplicaAssignment takes precedence.
> > > >    - In the case ReplicaAssignment is defined number of partitions
> and
> > > >       replicas will be calculated from the supplied
> ReplicaAssignment.
> > > >       - In the case of defined (Partitions + ReplicationFactor)
> 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.
> > > >    5. Setting a timeout > 0 will allow the request to block until the
> > > >    topic metadata is "complete" on the controller node.
> > > >       - Complete means the topic metadata has been completely
> populated
> > > >       (leaders, replicas, ISRs)
> > > >       - 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 other 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 of
> > fully
> > > >    consistent?
> > > >       - See Cluster Consistent Blocking
> > > >       <
> > >
> >
> https://cwiki.apache.org/#KIP-4-Commandlineandcentralizedadministrativeoperations-clusterconsistentblocking
> > > >
> > > >        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/#KIP-4-Commandlineandcentralizedadministrativeoperations-request
> > > >
> > > >       below
> > > >
> > > > Create Topic Response
> > > >
> > > >
> > > > CreateTopic Response (Version: 0) => [topic_error_codes]
> > > >   topic_error_codes => topic error_code
> > > >     topic => STRING
> > > >     error_code => INT16
> > > >
> > > > CreateTopicResponse 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
> > > >
> > > > ).
> > > >
> > >
> > > A sample PR is on github (https://github.com/apache/kafka/pull/1489)
> > > though
> > > it could change drastically based on the feedback here.
> > >
> > > Thanks,
> > > Grant
> > >
> > > --
> > > 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
>



-- 
-- Guozhang

Reply via email to