Thanks, Colin! That makes sense. I have updated the KIP to use separate methods. Can you do take a quick look to see if the KIP is ready for voting?
Thank you, Rajini On Thu, Sep 19, 2019 at 5:22 PM Colin McCabe <cmcc...@apache.org> wrote: > On Thu, Sep 19, 2019, at 06:31, Rajini Sivaram wrote: > > Hi Colin, > > > > Thanks for reviewing the KIP! > > > > I have added default values for the RPC. Since other int fields seem to > be > > using -1, I used -1 as the default for NumPartitions and > ReplicationFactor. > > > > Thanks. > > > I think a single method that returns TopicConfig seems better because > tools > > that are interested in the response are likely to process all of them. > And > > we can describe exceptions etc. together in one place (and tools can > handle > > them together too). I am ok with three methods too if there is good > reason > > to change. Is there a specific reason why we would want to keep them > > separate? > > I guess my thinking was that if we add new things that we return in the > future, we will have to somehow mark those as not present when talking to > older brokers. So we won't really be able to add anything else to > TopicConfig unless we add it as a Future which could throw > UnsupportedVersionException. This would be inconsistent with what we > already have, which is a structure which does not contain futures. > > Also, the name TopicConfig seems a bit misleading since number of > replicas, etc. is not usually considered a topic configuration (although I > guess philosophically it is...). Usually "topic configuration" means the > key/value pairs, right? > > best, > Colin > > > > > Thank you, > > > > Rajini > > > > On Wed, Sep 18, 2019 at 9:23 PM Colin McCabe <cmcc...@apache.org> wrote: > > > > > Hi Rajini, > > > > > > Thanks for the KIP. I think this will be a great improvement. > > > > > > For NumPartitions, ReplicationFactor, and Configs, we need some > reasonable > > > default value in the RPC which can be used for requests that are too > old to > > > contain this information. I'd suggest 0, 0, and null, respectively. > That > > > way we can, for example, distinguish between a response with zero > configs > > > and a response that's too old to have config information. > > > > > > I'm curious what you think about having three functions in > > > CreateTopicsResult rather than one. Maybe: > > > > > > > public KafkaFuture<Config> config(String topic); > > > > public KafkaFuture<Integer> numPartitions(String topic); > > > > public KafkaFuture<Integer> replicationFactor(String topic); > > > > > > Or is it better to have the "public KafkaFuture<TopicConfig> > > > topicConfig(String topic)" method? > > > > > > best, > > > Colin > > > > > > > > > On Tue, Sep 17, 2019, at 02:12, Rajini Sivaram wrote: > > > > Hi all, > > > > > > > > Since this is minor KIP, I will start vote tomorrow if there are no > > > > concerns. > > > > > > > > Thank you, > > > > > > > > Rajini > > > > > > > > On Fri, Sep 13, 2019 at 10:17 PM Rajini Sivaram < > rajinisiva...@gmail.com > > > > > > > > wrote: > > > > > > > > > Hi all, > > > > > > > > > > I would like to start discussion on KIP-525 to return topic > configs in > > > > > CreateTopics response: > > > > > > > > > > - > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-525+-+Return+topic+metadata+and+configs+in+CreateTopics+response > > > > > > > > > > > > > > > When validateOnly=false, this will be the actual configs of the > created > > > > > config. If validateOnly=true, this will be the configs with which > the > > > topic > > > > > would have been created. This provides an alternative to > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-234%3A+add+support+for+getting+topic+defaults+from+AdminClient > > > > > . > > > > > > > > > > Comments and suggestions welcome. > > > > > > > > > > Thank you, > > > > > > > > > > Rajini > > > > > > > > > > > > > > > > > > > >