Thanks for the review Ismael. `partition_count` or `num_partitions` seems clearer to me.
Agreed, I updated the wiki and patch to use num_partitions. 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. > Since Integer is used throughout for these values I think we should keep it INT32. I suppose in the future, there could be a case where more than 32K partitions are used too. > 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? > This is describing how the wire protocol would handle this. The API level does not have this problem. It's difficult to model a Map using arrays in the wire protocol but that is essentially what I am doing here. So just like a map the last put method is the only one that matters. It would be up to clients to make sure they are building their requests in a logical way. At the API level (CreateTopicRequest.java) its impossible to have multiple since the constructor takes a Map. The behavior of map is the same as the wire protocol behavior too. I could make the parsing fail in CreateTopicRequest.java if there are multiple instance of a topic. But it would require a new error code, since as far as I can tell we don't have an existing error code for invalid/malformed protocol messages and it seams like it might be hard to make that error descriptive to the client. > > 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? > This is the same as above. In the java api its impossible to pass both. Its only possible at the wire protocol level. 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? > I used the privileges that were defined and voted on in KIP-11: Authorization Interface here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface#KIP-11-AuthorizationInterface-Operation Though during KIP-11 privileges for ACLs was not discussed. My ACL schema <https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ACLAdminSchema.1> section discusses it a little and I mention we should consider adjusting them in my "Follow Up Changes <https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-FollowUpChangesfollow-up-changes>" section. The initial idea is that in order to modify ACLs you need to be granted the "All" Operation on the "Cluster". The intention here is that this is the most permissive you could possibly be, and we could get more finer grained easily in the future and the decision doesn't need to block this KIP. This also means having the "Create" Operation on the cluster won't give you unintended access. I agree though that we should be sure its clear for any future "create" type actions. What are your thoughts on having the following: - CREATE_TOPIC operation on the CLUSTER resource - This replaces the generic CREATE and is more specific to prevent future issues - ALTER, DELETE and DESCRIBE can stay the same since they are on the TOPIC resource - ACL_ACTION operation on the CLUSTER resource - This replaces my "*in order to modify ACLs you need to be granted the "All" Operation on the "Cluster"*" statement. What happens if timeout < 0? > We 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. (I added this answer to the wiki) > > - Why is CreateTopicRequest a batch request? > > > > Should it be `CreateTopicsRequest` then? > Sure, I will update that in the patch and wiki. P.S. I fixed a couple of typos I spotted on the wiki page, I hope that's OK. > Absolutely. Feel free to improve the wiki anytime. Thanks, Grant On Tue, Jun 14, 2016 at 3:09 PM, Ismael Juma <ism...@juma.me.uk> wrote: > 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. > -- Grant Henke Software Engineer | Cloudera gr...@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke