This could be a bug with the topic discovery logic in the wildcard consumer. Please can you file a bug and attach your consumer logs there ?
Thanks, Neha On Thu, Mar 14, 2013 at 3:20 PM, Jason Rosenberg <j...@squareup.com> wrote: > Yes, you're description matches what I did. And the brokers have been > bounced many times since then (they are auto-deployed many times a day, > etc.). And the consumers also have been restarted many times since then. > > Could it be related to using the white-list topic selector, etc.? > > Let me know if you need me to look a the state of my zookeeper data.... > > Jason > > On Thu, Mar 14, 2013 at 2:39 PM, Neha Narkhede <neha.narkh...@gmail.com > >wrote: > > > Jason, > > > > Let me see if I understood what you did here. In Kafka 0.7.2, you deleted > > the Kafka log files from the server and bounced the broker. This > should've > > ideally deleted those topics from zookeeper, the consumer reads the same > > zookeeper paths that the broker writes. Doing this should cause > rebalancing > > on the consumer and it should readjust its consumption to include the > > remaining topics. But looks like that didn't happen somehow ? > > > > I agree that log line is redundant, I think it is removed in 0.8 > > > > Thanks, > > Neha > > > > > > On Thu, Mar 14, 2013 at 2:17 PM, Jason Rosenberg <j...@squareup.com> > wrote: > > > > > Also, > > > > > > I see a bazillion consecutive log lines like this: > > > > > > 2013-03-14 19:54:13,306 INFO [Thread-4] consumer.ConsumerIterator - > > > Clearing the current data chunk for this consumer iterator > > > > > > With the same content (not sure how useful that is!). > > > > > > Jason > > > > > > On Thu, Mar 14, 2013 at 2:03 PM, Jason Rosenberg <j...@squareup.com> > > wrote: > > > > > > > Hi Neha, > > > > > > > > So I did this, but I still see the full list of topics (most of which > > > have > > > > been deleted), in the consumer logs, e.g.: > > > > > > > > consumer.ZookeeperConsumerConnector - > > > > samsa-consumer-graphite_alg2.sjc1.square-1363290849309-2816c1cb > Topics > > to > > > > consume = List(<a bazillion topics, most of which have been deleted > > > listed > > > > here>) > > > > > > > > I select topics using the white list topic selector (using a > wild-card > > > > pattern, etc.). > > > > > > > > Thoughts? > > > > > > > > Does zookeeper keep separate info for the consumer, that needs to be > > > > removed? > > > > > > > > Jason > > > > > > > > On Thu, Mar 7, 2013 at 10:20 PM, Neha Narkhede < > > neha.narkh...@gmail.com > > > >wrote: > > > > > > > >> In Kafka 0.7.2, data log directory delete and then rolling restart > > will > > > >> work to get rid of the topic completely from zookeeper as well as > the > > > >> broker. > > > >> This is because the broker registers the topics and partitions from > > its > > > >> data directory into zookeeper on startup. > > > >> > > > >> Thanks, > > > >> Neha > > > >> > > > >> > > > >> On Thu, Mar 7, 2013 at 9:13 PM, Jason Rosenberg <j...@squareup.com> > > > wrote: > > > >> > > > >> > Thanks Neha, > > > >> > > > > >> > So are you saying that on 0.7.2, to delete a topic I need only > > remove > > > >> it's > > > >> > data log directory from each broker, and the restart the brokers? > > Is > > > >> it ok > > > >> > if it's a rolling restart? > > > >> > > > > >> > For some reason I thought I also had to do something on zookeeper > > > >> servers > > > >> > too, to remove the topics. Is that not true? > > > >> > > > > >> > Jason > > > >> > > > > >> > On Wed, Mar 6, 2013 at 6:07 PM, Neha Narkhede < > > > neha.narkh...@gmail.com > > > >> > >wrote: > > > >> > > > > >> > > >> The topics get automagically created if they don't exist. > Will > > > >> > > that still exist in 0.8? > > > >> > > > > > >> > > In 0.8, this is controlled by the auto.create.topics.enable > config > > > on > > > >> the > > > >> > > brokers. If this is set to true, topics will be created when a > > topic > > > >> > > metadata request is sent for a new topic. This feature is > provided > > > to > > > >> aid > > > >> > > with the 0.7->0.8 migration tool and the mirror maker tools. > These > > > >> tools > > > >> > > copy data for almost all topics from one cluster to the other, > so > > > auto > > > >> > > creating topics makes it much easier to migrate data between > > > clusters. > > > >> > > > > > >> > > >> What's the procedure for deleting topics in kafka 0.7.2? > > > >> > > > > > >> > > There is no official way to delete a topic. You have to delete > the > > > >> data > > > >> > > directory and bounce the brokers to take note of that. > > > >> > > > > > >> > > Thanks, > > > >> > > Neha > > > >> > > > > > >> > > > > > >> > > On Wed, Mar 6, 2013 at 4:20 PM, Jason Rosenberg < > j...@squareup.com > > > > > > >> > wrote: > > > >> > > > > > >> > > > Thanks Neha, > > > >> > > > > > > >> > > > I added a comment to that ticket. > > > >> > > > > > > >> > > > Another question, I'm not sure if the behavior is different in > > 0.8 > > > >> from > > > >> > > > 0.7. But currently, I'm not predeclaring topics, before > sending > > > >> > messages > > > >> > > > to topics. The topics get automagically created if they don't > > > >> exist. > > > >> > > Will > > > >> > > > that still exist in 0.8? If so, it should also be automatic > > > (using > > > >> a > > > >> > > > default expiration for all such topics), that they > automagically > > > go > > > >> > away > > > >> > > > after no longer being in use. > > > >> > > > > > > >> > > > What's the procedure for deleting topics in kafka 0.7.2? > > > >> > > > > > > >> > > > Jason > > > >> > > > > > > >> > > > On Wed, Mar 6, 2013 at 3:38 PM, Neha Narkhede < > > > >> neha.narkh...@gmail.com > > > >> > > > >wrote: > > > >> > > > > > > >> > > > > We plan to include support for deleting topics before the > > > official > > > >> > > > release. > > > >> > > > > The JIRA tracking this feature is > > > >> > > > > https://issues.apache.org/jira/browse/KAFKA-330 > > > >> > > > > > > > >> > > > > This is mainly an admin tool and is proposed to be run > > > manually. I > > > >> > > think > > > >> > > > > your suggestion is to set some sort of expiration time on > > topics > > > >> so > > > >> > > when > > > >> > > > > all data gets garbage collected and the expiration time > > lapses, > > > >> the > > > >> > > topic > > > >> > > > > is automatically deleted. > > > >> > > > > > > > >> > > > > For this expiration to work, we need some mechanism of > knowing > > > the > > > >> > size > > > >> > > > of > > > >> > > > > the data logs for that topic on all brokers. And only when > its > > > >> empty > > > >> > > for > > > >> > > > > all partitions of that topic, we delete it. This requires > some > > > API > > > >> > that > > > >> > > > > will expose the data size of a topic on a broker, which we > > don't > > > >> have > > > >> > > > right > > > >> > > > > now. It might be worth thinking about it. Do you mind > posting > > > your > > > >> > > > > suggestions for this tool on that JIRA ? > > > >> > > > > > > > >> > > > > Thanks, > > > >> > > > > Neha > > > >> > > > > > > > >> > > > > > > > >> > > > > On Wed, Mar 6, 2013 at 3:15 PM, Jason Rosenberg < > > > j...@squareup.com > > > >> > > > > >> > > > wrote: > > > >> > > > > > > > >> > > > > > I suspect this is not currently supported, but it seems to > > be > > > >> for > > > >> > us > > > >> > > a > > > >> > > > > real > > > >> > > > > > use case. > > > >> > > > > > > > > >> > > > > > If we have a topic that is no longer receiving messages, > and > > > all > > > >> > > > messages > > > >> > > > > > have been removed from the brokers, after the > > > >> log_retention_hours > > > >> > has > > > >> > > > > > expired, I'd love to have it then automatically remove the > > > topic > > > >> > from > > > >> > > > > > kafka. > > > >> > > > > > > > > >> > > > > > If subsequently a new message arrives for that topic, it > > could > > > >> just > > > >> > > > > > recreate the topic as originally, and continue on. > > > >> > > > > > > > > >> > > > > > Is this easy to implement? > > > >> > > > > > Has it been requested? > > > >> > > > > > Does it make sense? > > > >> > > > > > Should I file a jira? > > > >> > > > > > > > > >> > > > > > Currently, we had a test client that inadvertently created > > > >> 1000's > > > >> > of > > > >> > > > > > topics. The job of deleting all these appears to not be a > > > >> > cumbersome > > > >> > > > > > manual process (are there any quick links to the > recommended > > > way > > > >> > for > > > >> > > > > doing > > > >> > > > > > this?). > > > >> > > > > > > > > >> > > > > > I am using 0.7.2, but will eventually upgrade to 0.8, once > > > it's > > > >> > > > released > > > >> > > > > to > > > >> > > > > > beta.... > > > >> > > > > > > > > >> > > > > > Thanks, > > > >> > > > > > > > > >> > > > > > Jason > > > >> > > > > > > > > >> > > > > > > > >> > > > > > > >> > > > > > >> > > > > >> > > > > > > > > > > > > > >