[ 
https://issues.apache.org/jira/browse/KAFKA-2275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14634080#comment-14634080
 ] 

Ashish K Singh commented on KAFKA-2275:
---------------------------------------

I guess, it will be better to keep the design level discussion on the JIRA and 
not on RB. Copying relevant conversation from RB here.

{quote}
Jason:
Adding the topic to the Metadata object means that from this point forward, we 
will always fetch the associated metadata for whatever topics were used in 
partitionsFor, even if we don't actually care about them anymore. Seems a 
little unfortunate, though I doubt it's much of an issue since users would 
probably only call this method for subscribed topics.

Ashish:
Jason, thanks for your review! I looked into ConsumerNetworkClient/ 
NetwrokClient, Metadata and Cluster classes. On receiving metadataUpdate, 
cluster instance in metadata is updated. However, when a topic is added by 
consumer, it is added to metadata.topics. After considering various options, I 
have updated the patch with what I think is the least obtrusive changes. So, we 
still keep metadata.topics as the list of topics we are interested in 
maintaining the state for, however we can choose to get metadata for all topics 
by setting metadata.needMetadataForAllTopics.

One thing to notice is that in the current implementation there is no caching 
for allTopics metadata, which might be OK depending on how we are planning to 
use it. We can discuss further once you take a look at the latest patch.

Jason Gustafson 1 hour, 50 minutes ago (July 20, 2015, 7:12 p.m.)
Hey Ashish, that makes sense and I agree that it seems less obtrusive. One 
concern I have is that we're using the same Cluster object in Metadata for 
representing both the set of all metadata and for just a subset (those topics 
that have been added through subscriptions). It seems like there might be 
potential for conflict there. Additionally I'm not sure how we'll be able to 
extend this to handle regex subscriptions. Basically we need to be able to 
"listen" for metadata changes and update our subscriptions based on any topic 
changes. We could block to get all metdata, but it's probably best if we can do 
this asynchronously. Do you have any thoughts on this?

Ashish Singh 4 minutes ago (July 20th, 2015, 8:58 p.m.)
> One concern I have is that we're using the same Cluster object in Metadata 
> for representing both the set of all metadata and for just a subset (those 
> topics that have been added through subscriptions). It seems like there might 
> be potential for conflict there.

Maybe I should move the flag, indicating cluster has metadata for all topics or 
subset of topics, to Cluster. Makes sense?

>Additionally I'm not sure how we'll be able to extend this to handle regex 
>subscriptions. Basically we need to be able to "listen" for metadata changes 
>and update our subscriptions based on any topic changes. We could block to get 
>all metdata, but it's probably best if we can do this asynchronously. Do you 
>have any thoughts on this?

I do not think there is a way to directly subscribe to metadata changes as of 
now. Correct me if my understanding is wrong. One would have to periodically 
poll to get metadata updates. Now, the question becomes where should this 
polling be done? With the current modification, the regex subscriber will have 
to manage the polling logic. We can definitely push the polling logic down to 
say Network client, but then the question will be is it required? Let me know 
your thoughts.
{quote}

[~hachikuji], [~guozhang] thoughts?

> Add a ListTopics() API to the new consumer
> ------------------------------------------
>
>                 Key: KAFKA-2275
>                 URL: https://issues.apache.org/jira/browse/KAFKA-2275
>             Project: Kafka
>          Issue Type: Sub-task
>          Components: consumer
>            Reporter: Guozhang Wang
>            Assignee: Ashish K Singh
>            Priority: Critical
>             Fix For: 0.8.3
>
>         Attachments: KAFKA-2275.patch, KAFKA-2275_2015-07-17_21:39:27.patch, 
> KAFKA-2275_2015-07-20_10:44:19.patch
>
>
> With regex subscription like
> {code}
> consumer.subscribe("topic*")
> {code}
> The partition assignment is automatically done at the Kafka side, while there 
> are some use cases where consumers want regex subscriptions but not 
> Kafka-side partition assignment, rather with their own specific partition 
> assignment. With ListTopics() they can periodically check for topic list 
> changes and specifically subscribe to the partitions of the new topics.
> For implementation, it involves sending a TopicMetadataRequest to a random 
> broker and parse the response.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to