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
> >
> >
>

Reply via email to