Hi Grant, Thanks for the proposal. A few comments and questions below.
On Fri, Jun 10, 2016 at 6:21 PM, Grant Henke <ghe...@cloudera.com> wrote: > > CreateTopic Request (Version: 0) => [create_topic_requests] timeout > > create_topic_requests => topic partitions replication_factor > [replica_assignment] [configs] > > topic => STRING > > partitions => INT32 > `partition_count` or `num_partitions` seems clearer to me. > replication_factor => INT32 > I wondered if this should be `INT16`. Maybe not worth it as it won't make much of a difference in terms of the request size though. > 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 > Silently ignoring what is likely a user error makes me uncomfortable generally. Is this really the best option? > > 3. The principle must be authorized to the "Create" Operation on the > > "Cluster" resource to create topics. > > - Unauthorized requests will receive a > ClusterAuthorizationException > Now that we are starting to use the `Create` operation, are we sure that the right model doesn't involve specifying the resource type? It seems to me that a `Create Topics` permission would make more sense as that would allow someone to be given `Create Topics` permission, but not `Create ACLs` for example. Was this discussed and discarded already? > > 4. > > > > Only one from ReplicaAssignment or (Partitions + ReplicationFactor), > can > > be defined in one instruction. If both parameters are specified - > > ReplicaAssignment takes precedence. > This is similar to `2`, do we want to silently ignore data from the user or fail-fast? > > - 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. > What happens if timeout < 0? > > - Why is CreateTopicRequest a batch request? > Should it be `CreateTopicsRequest` then? Thanks, Ismael P.S. I fixed a couple of typos I spotted on the wiki page, I hope that's OK.