Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Sumit Rangwala
I was planning to start a new thread by my experience is similar to the one
described here. I am currently using 0.8.2-beta and my setup has a producer
producing to topic "DC-DATA", kafka brokers with auto-topic-creation
enabled, and mirrormaker set to replicate any ".*-DATA" topics. After the
producer produces some data another thread in my code goes ahead and
deletes the topic DC-DATA. And I am seeing a similar behavior, i.e, the
topic reappears after deleting. I, however, changed the thread in my code
to wake up periodically and if it finds the topic (using getAllTopics) it
tries to delete the topic. What I found interesting is that after few
delete attempts the topic indeed gets deleted and does not reappear.  Seems
like some interplay between auto-topic creating and consumer notification
when a consumer is subscribed to a topic specified as a regex.


Sumit





On Mon, Jan 26, 2015 at 1:49 PM, Jun Rao  wrote:

> Joel,
>
> That's probably because console consumer always uses wildcard for
> consumption.
>
> Thanks,
>
> Jun
>
> On Mon, Jan 26, 2015 at 1:44 PM, Joel Koshy  wrote:
>
> > I think this should work even on non-wildcard consumers. (i.e., I just
> > tried it locally and it appears to work fine).
> >
> > Here is what I did:
> >
> > - Leave a console consumer up and running, bounce the broker to turn
> >   off auto-create
> > - Delete the topic
> > - The consumer should rebalance and stop consuming that topic
> > - Bounce the broker to turn on auto-create
> >
> > The only issue is that while your broker has auto-create off you will
> > see errors on some producers - which you can announce as a brief
> > outage to your users.
> >
> > Joel
> >
> > On Mon, Jan 26, 2015 at 01:02:57PM -0800, Jun Rao wrote:
> > > Actually, I think the deletion of a topic will trigger a rebalance on
> all
> > > wildcard subscribers. The time to complete the rebalance depends on
> the #
> > > of topic/partitions.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Jan 26, 2015 at 12:26 PM, Jason Rosenberg 
> > wrote:
> > >
> > > > Yeah Joel,
> > > >
> > > > I just thought of that idea too (e.g. temporarily disable auto topic
> > > > creation).  I can probably make that work (it may result in
> unexpected
> > lack
> > > > of topic creation, but I can message this as a maintenance downtime,
> > etc.).
> > > >
> > > > So, that was my next question, so a consumer rebalance will trigger a
> > fresh
> > > > topic pull from the consumers?  How long is 'long enough' to ensure a
> > > > rebalance has occurred everywhere?
> > > >
> > > > Jason
> > > >
> > > > On Mon, Jan 26, 2015 at 3:07 PM, Joel Koshy 
> > wrote:
> > > >
> > > > > Hey Jason,
> > > > >
> > > > > Is it an option for you to do the following:
> > > > >
> > > > > - Bounce in a config change to the brokers to turn off auto-create
> > > > > - (Batch)-delete the topic(s)
> > > > > - Wait long enough for consumers to rebalance (after which they
> will
> > > > >   no longer consume the topic(s))
> > > > > - Bounce in a config change to the brokers to turn on auto-create
> > > > >
> > > > > Joel
> > > > >
> > > > > On Mon, Jan 26, 2015 at 12:01:59PM -0800, Jun Rao wrote:
> > > > > > Hi, Jason,
> > > > > >
> > > > > > I am not sure that we should delay the 0.8.2 release. The reasons
> > are
> > > > (1)
> > > > > > There are other features such as the new java producer and
> > Kafka-based
> > > > > > offset management that are potentially useful to people. (2) It
> may
> > > > take
> > > > > > some time to completely fix the issue with deleting topic since
> it
> > > > > involves
> > > > > > wire protocol changes.
> > > > > >
> > > > > > What we can do is probably just to document the limitation of
> > delete
> > > > > topic
> > > > > > in 0.8.2.0 in our site.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Mon, Jan 26, 2015 at 11:05 AM, Jason Rosenberg <
> > j...@squareup.com>
> > > > > wrote:
> > > > > >
> > > > > > > I think this shortcoming should probably delay release of 0.8.2
> > until
> > > > > > > resolved, no?
> > > > > > >
> > > > > > > On Mon, Jan 26, 2015 at 2:01 PM, Guozhang Wang <
> > wangg...@gmail.com>
> > > > > wrote:
> > > > > > >
> > > > > > > > It then seems to me that delete-topic will not actually work
> > > > > "smoothly"
> > > > > > > > until create topic request is added since it is too much to
> > require
> > > > > > > people
> > > > > > > > to turn off their clients while deleting topics. In this case
> > shall
> > > > > we
> > > > > > > make
> > > > > > > > it clear in the release docs of in 0.8.2, or even still mark
> > it as
> > > > > > > > not-supported?
> > > > > > > >
> > > > > > > > Guozhang
> > > > > > > >
> > > > > > > > On Mon, Jan 26, 2015 at 8:14 AM, Harsha 
> > wrote:
> > > > > > > >
> > > > > > > > > Jun,
> > > > > > > > >   I made an attempt at fixing that issue as part of
> this
> > JIRA
> > > > > > > > >   https://issues.apache.org/jira/browse/KAFKA-1507 .
> > > > > > > > > As Jay 

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Joel Koshy
Hmm.. that's right. completely forgot about that.

On Mon, Jan 26, 2015 at 01:49:33PM -0800, Jun Rao wrote:
> Joel,
> 
> That's probably because console consumer always uses wildcard for
> consumption.
> 
> Thanks,
> 
> Jun
> 
> On Mon, Jan 26, 2015 at 1:44 PM, Joel Koshy  wrote:
> 
> > I think this should work even on non-wildcard consumers. (i.e., I just
> > tried it locally and it appears to work fine).
> >
> > Here is what I did:
> >
> > - Leave a console consumer up and running, bounce the broker to turn
> >   off auto-create
> > - Delete the topic
> > - The consumer should rebalance and stop consuming that topic
> > - Bounce the broker to turn on auto-create
> >
> > The only issue is that while your broker has auto-create off you will
> > see errors on some producers - which you can announce as a brief
> > outage to your users.
> >
> > Joel
> >
> > On Mon, Jan 26, 2015 at 01:02:57PM -0800, Jun Rao wrote:
> > > Actually, I think the deletion of a topic will trigger a rebalance on all
> > > wildcard subscribers. The time to complete the rebalance depends on the #
> > > of topic/partitions.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Jan 26, 2015 at 12:26 PM, Jason Rosenberg 
> > wrote:
> > >
> > > > Yeah Joel,
> > > >
> > > > I just thought of that idea too (e.g. temporarily disable auto topic
> > > > creation).  I can probably make that work (it may result in unexpected
> > lack
> > > > of topic creation, but I can message this as a maintenance downtime,
> > etc.).
> > > >
> > > > So, that was my next question, so a consumer rebalance will trigger a
> > fresh
> > > > topic pull from the consumers?  How long is 'long enough' to ensure a
> > > > rebalance has occurred everywhere?
> > > >
> > > > Jason
> > > >
> > > > On Mon, Jan 26, 2015 at 3:07 PM, Joel Koshy 
> > wrote:
> > > >
> > > > > Hey Jason,
> > > > >
> > > > > Is it an option for you to do the following:
> > > > >
> > > > > - Bounce in a config change to the brokers to turn off auto-create
> > > > > - (Batch)-delete the topic(s)
> > > > > - Wait long enough for consumers to rebalance (after which they will
> > > > >   no longer consume the topic(s))
> > > > > - Bounce in a config change to the brokers to turn on auto-create
> > > > >
> > > > > Joel
> > > > >
> > > > > On Mon, Jan 26, 2015 at 12:01:59PM -0800, Jun Rao wrote:
> > > > > > Hi, Jason,
> > > > > >
> > > > > > I am not sure that we should delay the 0.8.2 release. The reasons
> > are
> > > > (1)
> > > > > > There are other features such as the new java producer and
> > Kafka-based
> > > > > > offset management that are potentially useful to people. (2) It may
> > > > take
> > > > > > some time to completely fix the issue with deleting topic since it
> > > > > involves
> > > > > > wire protocol changes.
> > > > > >
> > > > > > What we can do is probably just to document the limitation of
> > delete
> > > > > topic
> > > > > > in 0.8.2.0 in our site.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Mon, Jan 26, 2015 at 11:05 AM, Jason Rosenberg <
> > j...@squareup.com>
> > > > > wrote:
> > > > > >
> > > > > > > I think this shortcoming should probably delay release of 0.8.2
> > until
> > > > > > > resolved, no?
> > > > > > >
> > > > > > > On Mon, Jan 26, 2015 at 2:01 PM, Guozhang Wang <
> > wangg...@gmail.com>
> > > > > wrote:
> > > > > > >
> > > > > > > > It then seems to me that delete-topic will not actually work
> > > > > "smoothly"
> > > > > > > > until create topic request is added since it is too much to
> > require
> > > > > > > people
> > > > > > > > to turn off their clients while deleting topics. In this case
> > shall
> > > > > we
> > > > > > > make
> > > > > > > > it clear in the release docs of in 0.8.2, or even still mark
> > it as
> > > > > > > > not-supported?
> > > > > > > >
> > > > > > > > Guozhang
> > > > > > > >
> > > > > > > > On Mon, Jan 26, 2015 at 8:14 AM, Harsha 
> > wrote:
> > > > > > > >
> > > > > > > > > Jun,
> > > > > > > > >   I made an attempt at fixing that issue as part of this
> > JIRA
> > > > > > > > >   https://issues.apache.org/jira/browse/KAFKA-1507 .
> > > > > > > > > As Jay pointed out there should be admin api if there is more
> > > > info
> > > > > on
> > > > > > > > > this api I am interested in adding/fixing this issue.
> > > > > > > > > Thanks,
> > > > > > > > > Harsha
> > > > > > > > >
> > > > > > > > > On Mon, Jan 26, 2015, at 07:28 AM, Jun Rao wrote:
> > > > > > > > > > Yes, that's the issue. Currently, topics can be
> > auto-created on
> > > > > > > > > > TopicMetadataRequest, which can be issued from both the
> > > > producer
> > > > > and
> > > > > > > > the
> > > > > > > > > > consumer. To prevent that, you would need to stop the
> > producer
> > > > > and
> > > > > > > the
> > > > > > > > > > consumer before deleting a topic. We plan to address this
> > issue
> > > > > once
> > > > > > > we
> > > > > > > > > > have a separate request for creating topics.
> > > > > > > > > >
> 

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Jun Rao
Joel,

That's probably because console consumer always uses wildcard for
consumption.

Thanks,

Jun

On Mon, Jan 26, 2015 at 1:44 PM, Joel Koshy  wrote:

> I think this should work even on non-wildcard consumers. (i.e., I just
> tried it locally and it appears to work fine).
>
> Here is what I did:
>
> - Leave a console consumer up and running, bounce the broker to turn
>   off auto-create
> - Delete the topic
> - The consumer should rebalance and stop consuming that topic
> - Bounce the broker to turn on auto-create
>
> The only issue is that while your broker has auto-create off you will
> see errors on some producers - which you can announce as a brief
> outage to your users.
>
> Joel
>
> On Mon, Jan 26, 2015 at 01:02:57PM -0800, Jun Rao wrote:
> > Actually, I think the deletion of a topic will trigger a rebalance on all
> > wildcard subscribers. The time to complete the rebalance depends on the #
> > of topic/partitions.
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Jan 26, 2015 at 12:26 PM, Jason Rosenberg 
> wrote:
> >
> > > Yeah Joel,
> > >
> > > I just thought of that idea too (e.g. temporarily disable auto topic
> > > creation).  I can probably make that work (it may result in unexpected
> lack
> > > of topic creation, but I can message this as a maintenance downtime,
> etc.).
> > >
> > > So, that was my next question, so a consumer rebalance will trigger a
> fresh
> > > topic pull from the consumers?  How long is 'long enough' to ensure a
> > > rebalance has occurred everywhere?
> > >
> > > Jason
> > >
> > > On Mon, Jan 26, 2015 at 3:07 PM, Joel Koshy 
> wrote:
> > >
> > > > Hey Jason,
> > > >
> > > > Is it an option for you to do the following:
> > > >
> > > > - Bounce in a config change to the brokers to turn off auto-create
> > > > - (Batch)-delete the topic(s)
> > > > - Wait long enough for consumers to rebalance (after which they will
> > > >   no longer consume the topic(s))
> > > > - Bounce in a config change to the brokers to turn on auto-create
> > > >
> > > > Joel
> > > >
> > > > On Mon, Jan 26, 2015 at 12:01:59PM -0800, Jun Rao wrote:
> > > > > Hi, Jason,
> > > > >
> > > > > I am not sure that we should delay the 0.8.2 release. The reasons
> are
> > > (1)
> > > > > There are other features such as the new java producer and
> Kafka-based
> > > > > offset management that are potentially useful to people. (2) It may
> > > take
> > > > > some time to completely fix the issue with deleting topic since it
> > > > involves
> > > > > wire protocol changes.
> > > > >
> > > > > What we can do is probably just to document the limitation of
> delete
> > > > topic
> > > > > in 0.8.2.0 in our site.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Mon, Jan 26, 2015 at 11:05 AM, Jason Rosenberg <
> j...@squareup.com>
> > > > wrote:
> > > > >
> > > > > > I think this shortcoming should probably delay release of 0.8.2
> until
> > > > > > resolved, no?
> > > > > >
> > > > > > On Mon, Jan 26, 2015 at 2:01 PM, Guozhang Wang <
> wangg...@gmail.com>
> > > > wrote:
> > > > > >
> > > > > > > It then seems to me that delete-topic will not actually work
> > > > "smoothly"
> > > > > > > until create topic request is added since it is too much to
> require
> > > > > > people
> > > > > > > to turn off their clients while deleting topics. In this case
> shall
> > > > we
> > > > > > make
> > > > > > > it clear in the release docs of in 0.8.2, or even still mark
> it as
> > > > > > > not-supported?
> > > > > > >
> > > > > > > Guozhang
> > > > > > >
> > > > > > > On Mon, Jan 26, 2015 at 8:14 AM, Harsha 
> wrote:
> > > > > > >
> > > > > > > > Jun,
> > > > > > > >   I made an attempt at fixing that issue as part of this
> JIRA
> > > > > > > >   https://issues.apache.org/jira/browse/KAFKA-1507 .
> > > > > > > > As Jay pointed out there should be admin api if there is more
> > > info
> > > > on
> > > > > > > > this api I am interested in adding/fixing this issue.
> > > > > > > > Thanks,
> > > > > > > > Harsha
> > > > > > > >
> > > > > > > > On Mon, Jan 26, 2015, at 07:28 AM, Jun Rao wrote:
> > > > > > > > > Yes, that's the issue. Currently, topics can be
> auto-created on
> > > > > > > > > TopicMetadataRequest, which can be issued from both the
> > > producer
> > > > and
> > > > > > > the
> > > > > > > > > consumer. To prevent that, you would need to stop the
> producer
> > > > and
> > > > > > the
> > > > > > > > > consumer before deleting a topic. We plan to address this
> issue
> > > > once
> > > > > > we
> > > > > > > > > have a separate request for creating topics.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Mon, Jan 26, 2015 at 7:21 AM, Harsha 
> > > wrote:
> > > > > > > > >
> > > > > > > > > > There could be another case where if you have
> > > > > > > auto.create.topics.enable
> > > > > > > > > > to set to true ( its true by default) . Any
> > > > TopicMetadataRequest
> > > > > > can
> > > > > > > > > > rec

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Joel Koshy
I think this should work even on non-wildcard consumers. (i.e., I just
tried it locally and it appears to work fine).

Here is what I did:

- Leave a console consumer up and running, bounce the broker to turn
  off auto-create
- Delete the topic
- The consumer should rebalance and stop consuming that topic
- Bounce the broker to turn on auto-create

The only issue is that while your broker has auto-create off you will
see errors on some producers - which you can announce as a brief
outage to your users.

Joel

On Mon, Jan 26, 2015 at 01:02:57PM -0800, Jun Rao wrote:
> Actually, I think the deletion of a topic will trigger a rebalance on all
> wildcard subscribers. The time to complete the rebalance depends on the #
> of topic/partitions.
> 
> Thanks,
> 
> Jun
> 
> On Mon, Jan 26, 2015 at 12:26 PM, Jason Rosenberg  wrote:
> 
> > Yeah Joel,
> >
> > I just thought of that idea too (e.g. temporarily disable auto topic
> > creation).  I can probably make that work (it may result in unexpected lack
> > of topic creation, but I can message this as a maintenance downtime, etc.).
> >
> > So, that was my next question, so a consumer rebalance will trigger a fresh
> > topic pull from the consumers?  How long is 'long enough' to ensure a
> > rebalance has occurred everywhere?
> >
> > Jason
> >
> > On Mon, Jan 26, 2015 at 3:07 PM, Joel Koshy  wrote:
> >
> > > Hey Jason,
> > >
> > > Is it an option for you to do the following:
> > >
> > > - Bounce in a config change to the brokers to turn off auto-create
> > > - (Batch)-delete the topic(s)
> > > - Wait long enough for consumers to rebalance (after which they will
> > >   no longer consume the topic(s))
> > > - Bounce in a config change to the brokers to turn on auto-create
> > >
> > > Joel
> > >
> > > On Mon, Jan 26, 2015 at 12:01:59PM -0800, Jun Rao wrote:
> > > > Hi, Jason,
> > > >
> > > > I am not sure that we should delay the 0.8.2 release. The reasons are
> > (1)
> > > > There are other features such as the new java producer and Kafka-based
> > > > offset management that are potentially useful to people. (2) It may
> > take
> > > > some time to completely fix the issue with deleting topic since it
> > > involves
> > > > wire protocol changes.
> > > >
> > > > What we can do is probably just to document the limitation of delete
> > > topic
> > > > in 0.8.2.0 in our site.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Mon, Jan 26, 2015 at 11:05 AM, Jason Rosenberg 
> > > wrote:
> > > >
> > > > > I think this shortcoming should probably delay release of 0.8.2 until
> > > > > resolved, no?
> > > > >
> > > > > On Mon, Jan 26, 2015 at 2:01 PM, Guozhang Wang 
> > > wrote:
> > > > >
> > > > > > It then seems to me that delete-topic will not actually work
> > > "smoothly"
> > > > > > until create topic request is added since it is too much to require
> > > > > people
> > > > > > to turn off their clients while deleting topics. In this case shall
> > > we
> > > > > make
> > > > > > it clear in the release docs of in 0.8.2, or even still mark it as
> > > > > > not-supported?
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > > On Mon, Jan 26, 2015 at 8:14 AM, Harsha  wrote:
> > > > > >
> > > > > > > Jun,
> > > > > > >   I made an attempt at fixing that issue as part of this JIRA
> > > > > > >   https://issues.apache.org/jira/browse/KAFKA-1507 .
> > > > > > > As Jay pointed out there should be admin api if there is more
> > info
> > > on
> > > > > > > this api I am interested in adding/fixing this issue.
> > > > > > > Thanks,
> > > > > > > Harsha
> > > > > > >
> > > > > > > On Mon, Jan 26, 2015, at 07:28 AM, Jun Rao wrote:
> > > > > > > > Yes, that's the issue. Currently, topics can be auto-created on
> > > > > > > > TopicMetadataRequest, which can be issued from both the
> > producer
> > > and
> > > > > > the
> > > > > > > > consumer. To prevent that, you would need to stop the producer
> > > and
> > > > > the
> > > > > > > > consumer before deleting a topic. We plan to address this issue
> > > once
> > > > > we
> > > > > > > > have a separate request for creating topics.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Mon, Jan 26, 2015 at 7:21 AM, Harsha 
> > wrote:
> > > > > > > >
> > > > > > > > > There could be another case where if you have
> > > > > > auto.create.topics.enable
> > > > > > > > > to set to true ( its true by default) . Any
> > > TopicMetadataRequest
> > > > > can
> > > > > > > > > recreate topics. So if you issued a delete topic command and
> > > you
> > > > > have
> > > > > > > > > producers running or consumers? too which is issuing a
> > > > > > > > > TopicMetadataRequest than the topic will be recreated.
> > > > > > > > > -Harsha
> > > > > > > > >
> > > > > > > > > On Sun, Jan 25, 2015, at 11:26 PM, Jason Rosenberg wrote:
> > > > > > > > > > cversion did change (incremented by 2) when I issue the
> > > delete
> > > > > > > command.
> > > > > > > > > >
> > > > > >

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Jun Rao
Actually, I think the deletion of a topic will trigger a rebalance on all
wildcard subscribers. The time to complete the rebalance depends on the #
of topic/partitions.

Thanks,

Jun

On Mon, Jan 26, 2015 at 12:26 PM, Jason Rosenberg  wrote:

> Yeah Joel,
>
> I just thought of that idea too (e.g. temporarily disable auto topic
> creation).  I can probably make that work (it may result in unexpected lack
> of topic creation, but I can message this as a maintenance downtime, etc.).
>
> So, that was my next question, so a consumer rebalance will trigger a fresh
> topic pull from the consumers?  How long is 'long enough' to ensure a
> rebalance has occurred everywhere?
>
> Jason
>
> On Mon, Jan 26, 2015 at 3:07 PM, Joel Koshy  wrote:
>
> > Hey Jason,
> >
> > Is it an option for you to do the following:
> >
> > - Bounce in a config change to the brokers to turn off auto-create
> > - (Batch)-delete the topic(s)
> > - Wait long enough for consumers to rebalance (after which they will
> >   no longer consume the topic(s))
> > - Bounce in a config change to the brokers to turn on auto-create
> >
> > Joel
> >
> > On Mon, Jan 26, 2015 at 12:01:59PM -0800, Jun Rao wrote:
> > > Hi, Jason,
> > >
> > > I am not sure that we should delay the 0.8.2 release. The reasons are
> (1)
> > > There are other features such as the new java producer and Kafka-based
> > > offset management that are potentially useful to people. (2) It may
> take
> > > some time to completely fix the issue with deleting topic since it
> > involves
> > > wire protocol changes.
> > >
> > > What we can do is probably just to document the limitation of delete
> > topic
> > > in 0.8.2.0 in our site.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Jan 26, 2015 at 11:05 AM, Jason Rosenberg 
> > wrote:
> > >
> > > > I think this shortcoming should probably delay release of 0.8.2 until
> > > > resolved, no?
> > > >
> > > > On Mon, Jan 26, 2015 at 2:01 PM, Guozhang Wang 
> > wrote:
> > > >
> > > > > It then seems to me that delete-topic will not actually work
> > "smoothly"
> > > > > until create topic request is added since it is too much to require
> > > > people
> > > > > to turn off their clients while deleting topics. In this case shall
> > we
> > > > make
> > > > > it clear in the release docs of in 0.8.2, or even still mark it as
> > > > > not-supported?
> > > > >
> > > > > Guozhang
> > > > >
> > > > > On Mon, Jan 26, 2015 at 8:14 AM, Harsha  wrote:
> > > > >
> > > > > > Jun,
> > > > > >   I made an attempt at fixing that issue as part of this JIRA
> > > > > >   https://issues.apache.org/jira/browse/KAFKA-1507 .
> > > > > > As Jay pointed out there should be admin api if there is more
> info
> > on
> > > > > > this api I am interested in adding/fixing this issue.
> > > > > > Thanks,
> > > > > > Harsha
> > > > > >
> > > > > > On Mon, Jan 26, 2015, at 07:28 AM, Jun Rao wrote:
> > > > > > > Yes, that's the issue. Currently, topics can be auto-created on
> > > > > > > TopicMetadataRequest, which can be issued from both the
> producer
> > and
> > > > > the
> > > > > > > consumer. To prevent that, you would need to stop the producer
> > and
> > > > the
> > > > > > > consumer before deleting a topic. We plan to address this issue
> > once
> > > > we
> > > > > > > have a separate request for creating topics.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Mon, Jan 26, 2015 at 7:21 AM, Harsha 
> wrote:
> > > > > > >
> > > > > > > > There could be another case where if you have
> > > > > auto.create.topics.enable
> > > > > > > > to set to true ( its true by default) . Any
> > TopicMetadataRequest
> > > > can
> > > > > > > > recreate topics. So if you issued a delete topic command and
> > you
> > > > have
> > > > > > > > producers running or consumers? too which is issuing a
> > > > > > > > TopicMetadataRequest than the topic will be recreated.
> > > > > > > > -Harsha
> > > > > > > >
> > > > > > > > On Sun, Jan 25, 2015, at 11:26 PM, Jason Rosenberg wrote:
> > > > > > > > > cversion did change (incremented by 2) when I issue the
> > delete
> > > > > > command.
> > > > > > > > >
> > > > > > > > > From the logs on the conroller broker (also the leader for
> > the
> > > > > > topic), it
> > > > > > > > > looks like the delete proceeds, and then the topic gets
> > recreated
> > > > > > > > > immediately (highlighted in yellow). It appears maybe it’s
> > due
> > > > to a
> > > > > > > > > consumer client app trying to consume the topic. Also, the
> > > > consumer
> > > > > > is
> > > > > > > > > not
> > > > > > > > > yet updated to 0.8.2 (it’s using 0.8.1.1), perhaps that’s
> > part of
> > > > > the
> > > > > > > > > problem?
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > 2015-01-26 07:02:14,281  INFO
> > > > > > > > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > > > > > > > controller.PartitionStateMachine$DeleteTopicsListener -
> > > > > > > > > [DeleteTopicsListener on 6]: Starti

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Jason Rosenberg
Yeah Jun,

I realize it's not worth blocking things, since it's not trivial to solve.

Thanks for all the work on this.

Jason

On Mon, Jan 26, 2015 at 3:01 PM, Jun Rao  wrote:

> Hi, Jason,
>
> I am not sure that we should delay the 0.8.2 release. The reasons are (1)
> There are other features such as the new java producer and Kafka-based
> offset management that are potentially useful to people. (2) It may take
> some time to completely fix the issue with deleting topic since it involves
> wire protocol changes.
>
> What we can do is probably just to document the limitation of delete topic
> in 0.8.2.0 in our site.
>
> Thanks,
>
> Jun
>
> On Mon, Jan 26, 2015 at 11:05 AM, Jason Rosenberg 
> wrote:
>
> > I think this shortcoming should probably delay release of 0.8.2 until
> > resolved, no?
> >
> > On Mon, Jan 26, 2015 at 2:01 PM, Guozhang Wang 
> wrote:
> >
> > > It then seems to me that delete-topic will not actually work "smoothly"
> > > until create topic request is added since it is too much to require
> > people
> > > to turn off their clients while deleting topics. In this case shall we
> > make
> > > it clear in the release docs of in 0.8.2, or even still mark it as
> > > not-supported?
> > >
> > > Guozhang
> > >
> > > On Mon, Jan 26, 2015 at 8:14 AM, Harsha  wrote:
> > >
> > > > Jun,
> > > >   I made an attempt at fixing that issue as part of this JIRA
> > > >   https://issues.apache.org/jira/browse/KAFKA-1507 .
> > > > As Jay pointed out there should be admin api if there is more info on
> > > > this api I am interested in adding/fixing this issue.
> > > > Thanks,
> > > > Harsha
> > > >
> > > > On Mon, Jan 26, 2015, at 07:28 AM, Jun Rao wrote:
> > > > > Yes, that's the issue. Currently, topics can be auto-created on
> > > > > TopicMetadataRequest, which can be issued from both the producer
> and
> > > the
> > > > > consumer. To prevent that, you would need to stop the producer and
> > the
> > > > > consumer before deleting a topic. We plan to address this issue
> once
> > we
> > > > > have a separate request for creating topics.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Mon, Jan 26, 2015 at 7:21 AM, Harsha  wrote:
> > > > >
> > > > > > There could be another case where if you have
> > > auto.create.topics.enable
> > > > > > to set to true ( its true by default) . Any TopicMetadataRequest
> > can
> > > > > > recreate topics. So if you issued a delete topic command and you
> > have
> > > > > > producers running or consumers? too which is issuing a
> > > > > > TopicMetadataRequest than the topic will be recreated.
> > > > > > -Harsha
> > > > > >
> > > > > > On Sun, Jan 25, 2015, at 11:26 PM, Jason Rosenberg wrote:
> > > > > > > cversion did change (incremented by 2) when I issue the delete
> > > > command.
> > > > > > >
> > > > > > > From the logs on the conroller broker (also the leader for the
> > > > topic), it
> > > > > > > looks like the delete proceeds, and then the topic gets
> recreated
> > > > > > > immediately (highlighted in yellow). It appears maybe it’s due
> > to a
> > > > > > > consumer client app trying to consume the topic. Also, the
> > consumer
> > > > is
> > > > > > > not
> > > > > > > yet updated to 0.8.2 (it’s using 0.8.1.1), perhaps that’s part
> of
> > > the
> > > > > > > problem?
> > > > > > >
> > > > > > >
> > > > > > > 2015-01-26 07:02:14,281  INFO
> > > > > > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > > > > > controller.PartitionStateMachine$DeleteTopicsListener -
> > > > > > > [DeleteTopicsListener on 6]: Starting topic deletion for topics
> > > > > > > mytopic
> > > > > > > 2015-01-26 07:02:14,282  INFO [delete-topics-thread-6]
> > > > > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > > > > [delete-topics-thread-6], Handling deletion for topics mytopic
> > > > > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > > > > [delete-topics-thread-6], Deletion of topic mytopic (re)started
> > > > > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > > > > controller.TopicDeletionManager - [Topic Deletion Manager 6],
> > Topic
> > > > > > > deletion callback for mytopic
> > > > > > > 2015-01-26 07:02:14,289  INFO [delete-topics-thread-6]
> > > > > > > controller.TopicDeletionManager - [Topic Deletion Manager 6],
> > > > > > > Partition deletion callback for [mytopic,0]
> > > > > > > 2015-01-26 07:02:14,295  INFO [delete-topics-thread-6]
> > > > > > > controller.ReplicaStateMachine - [Replica state machine on
> > > controller
> > > > > > > 6]: Invoking state change to OfflineReplica for replicas
> > > > > > >
> > > > > >
> > > >
> > >
> >
> [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > > > > > > 2015-01-26 07:02:14,303  INFO [delete-topics-thread-6]
> > > > > > > controller.KafkaController - [Controller 6]: New leader and ISR
> > for
> > > > > > > partition [mytopic,0] is

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Jason Rosenberg
Yeah Joel,

I just thought of that idea too (e.g. temporarily disable auto topic
creation).  I can probably make that work (it may result in unexpected lack
of topic creation, but I can message this as a maintenance downtime, etc.).

So, that was my next question, so a consumer rebalance will trigger a fresh
topic pull from the consumers?  How long is 'long enough' to ensure a
rebalance has occurred everywhere?

Jason

On Mon, Jan 26, 2015 at 3:07 PM, Joel Koshy  wrote:

> Hey Jason,
>
> Is it an option for you to do the following:
>
> - Bounce in a config change to the brokers to turn off auto-create
> - (Batch)-delete the topic(s)
> - Wait long enough for consumers to rebalance (after which they will
>   no longer consume the topic(s))
> - Bounce in a config change to the brokers to turn on auto-create
>
> Joel
>
> On Mon, Jan 26, 2015 at 12:01:59PM -0800, Jun Rao wrote:
> > Hi, Jason,
> >
> > I am not sure that we should delay the 0.8.2 release. The reasons are (1)
> > There are other features such as the new java producer and Kafka-based
> > offset management that are potentially useful to people. (2) It may take
> > some time to completely fix the issue with deleting topic since it
> involves
> > wire protocol changes.
> >
> > What we can do is probably just to document the limitation of delete
> topic
> > in 0.8.2.0 in our site.
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Jan 26, 2015 at 11:05 AM, Jason Rosenberg 
> wrote:
> >
> > > I think this shortcoming should probably delay release of 0.8.2 until
> > > resolved, no?
> > >
> > > On Mon, Jan 26, 2015 at 2:01 PM, Guozhang Wang 
> wrote:
> > >
> > > > It then seems to me that delete-topic will not actually work
> "smoothly"
> > > > until create topic request is added since it is too much to require
> > > people
> > > > to turn off their clients while deleting topics. In this case shall
> we
> > > make
> > > > it clear in the release docs of in 0.8.2, or even still mark it as
> > > > not-supported?
> > > >
> > > > Guozhang
> > > >
> > > > On Mon, Jan 26, 2015 at 8:14 AM, Harsha  wrote:
> > > >
> > > > > Jun,
> > > > >   I made an attempt at fixing that issue as part of this JIRA
> > > > >   https://issues.apache.org/jira/browse/KAFKA-1507 .
> > > > > As Jay pointed out there should be admin api if there is more info
> on
> > > > > this api I am interested in adding/fixing this issue.
> > > > > Thanks,
> > > > > Harsha
> > > > >
> > > > > On Mon, Jan 26, 2015, at 07:28 AM, Jun Rao wrote:
> > > > > > Yes, that's the issue. Currently, topics can be auto-created on
> > > > > > TopicMetadataRequest, which can be issued from both the producer
> and
> > > > the
> > > > > > consumer. To prevent that, you would need to stop the producer
> and
> > > the
> > > > > > consumer before deleting a topic. We plan to address this issue
> once
> > > we
> > > > > > have a separate request for creating topics.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Mon, Jan 26, 2015 at 7:21 AM, Harsha  wrote:
> > > > > >
> > > > > > > There could be another case where if you have
> > > > auto.create.topics.enable
> > > > > > > to set to true ( its true by default) . Any
> TopicMetadataRequest
> > > can
> > > > > > > recreate topics. So if you issued a delete topic command and
> you
> > > have
> > > > > > > producers running or consumers? too which is issuing a
> > > > > > > TopicMetadataRequest than the topic will be recreated.
> > > > > > > -Harsha
> > > > > > >
> > > > > > > On Sun, Jan 25, 2015, at 11:26 PM, Jason Rosenberg wrote:
> > > > > > > > cversion did change (incremented by 2) when I issue the
> delete
> > > > > command.
> > > > > > > >
> > > > > > > > From the logs on the conroller broker (also the leader for
> the
> > > > > topic), it
> > > > > > > > looks like the delete proceeds, and then the topic gets
> recreated
> > > > > > > > immediately (highlighted in yellow). It appears maybe it’s
> due
> > > to a
> > > > > > > > consumer client app trying to consume the topic. Also, the
> > > consumer
> > > > > is
> > > > > > > > not
> > > > > > > > yet updated to 0.8.2 (it’s using 0.8.1.1), perhaps that’s
> part of
> > > > the
> > > > > > > > problem?
> > > > > > > >
> > > > > > > >
> > > > > > > > 2015-01-26 07:02:14,281  INFO
> > > > > > > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > > > > > > controller.PartitionStateMachine$DeleteTopicsListener -
> > > > > > > > [DeleteTopicsListener on 6]: Starting topic deletion for
> topics
> > > > > > > > mytopic
> > > > > > > > 2015-01-26 07:02:14,282  INFO [delete-topics-thread-6]
> > > > > > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > > > > > [delete-topics-thread-6], Handling deletion for topics
> mytopic
> > > > > > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > > > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > > > > > [delete-topics-thread-6], Deletion of topic mytopic
> (re)started
> > > > > > > > 20

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Joel Koshy
Hey Jason,

Is it an option for you to do the following:

- Bounce in a config change to the brokers to turn off auto-create
- (Batch)-delete the topic(s)
- Wait long enough for consumers to rebalance (after which they will
  no longer consume the topic(s))
- Bounce in a config change to the brokers to turn on auto-create

Joel

On Mon, Jan 26, 2015 at 12:01:59PM -0800, Jun Rao wrote:
> Hi, Jason,
> 
> I am not sure that we should delay the 0.8.2 release. The reasons are (1)
> There are other features such as the new java producer and Kafka-based
> offset management that are potentially useful to people. (2) It may take
> some time to completely fix the issue with deleting topic since it involves
> wire protocol changes.
> 
> What we can do is probably just to document the limitation of delete topic
> in 0.8.2.0 in our site.
> 
> Thanks,
> 
> Jun
> 
> On Mon, Jan 26, 2015 at 11:05 AM, Jason Rosenberg  wrote:
> 
> > I think this shortcoming should probably delay release of 0.8.2 until
> > resolved, no?
> >
> > On Mon, Jan 26, 2015 at 2:01 PM, Guozhang Wang  wrote:
> >
> > > It then seems to me that delete-topic will not actually work "smoothly"
> > > until create topic request is added since it is too much to require
> > people
> > > to turn off their clients while deleting topics. In this case shall we
> > make
> > > it clear in the release docs of in 0.8.2, or even still mark it as
> > > not-supported?
> > >
> > > Guozhang
> > >
> > > On Mon, Jan 26, 2015 at 8:14 AM, Harsha  wrote:
> > >
> > > > Jun,
> > > >   I made an attempt at fixing that issue as part of this JIRA
> > > >   https://issues.apache.org/jira/browse/KAFKA-1507 .
> > > > As Jay pointed out there should be admin api if there is more info on
> > > > this api I am interested in adding/fixing this issue.
> > > > Thanks,
> > > > Harsha
> > > >
> > > > On Mon, Jan 26, 2015, at 07:28 AM, Jun Rao wrote:
> > > > > Yes, that's the issue. Currently, topics can be auto-created on
> > > > > TopicMetadataRequest, which can be issued from both the producer and
> > > the
> > > > > consumer. To prevent that, you would need to stop the producer and
> > the
> > > > > consumer before deleting a topic. We plan to address this issue once
> > we
> > > > > have a separate request for creating topics.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Mon, Jan 26, 2015 at 7:21 AM, Harsha  wrote:
> > > > >
> > > > > > There could be another case where if you have
> > > auto.create.topics.enable
> > > > > > to set to true ( its true by default) . Any TopicMetadataRequest
> > can
> > > > > > recreate topics. So if you issued a delete topic command and you
> > have
> > > > > > producers running or consumers? too which is issuing a
> > > > > > TopicMetadataRequest than the topic will be recreated.
> > > > > > -Harsha
> > > > > >
> > > > > > On Sun, Jan 25, 2015, at 11:26 PM, Jason Rosenberg wrote:
> > > > > > > cversion did change (incremented by 2) when I issue the delete
> > > > command.
> > > > > > >
> > > > > > > From the logs on the conroller broker (also the leader for the
> > > > topic), it
> > > > > > > looks like the delete proceeds, and then the topic gets recreated
> > > > > > > immediately (highlighted in yellow). It appears maybe it’s due
> > to a
> > > > > > > consumer client app trying to consume the topic. Also, the
> > consumer
> > > > is
> > > > > > > not
> > > > > > > yet updated to 0.8.2 (it’s using 0.8.1.1), perhaps that’s part of
> > > the
> > > > > > > problem?
> > > > > > >
> > > > > > >
> > > > > > > 2015-01-26 07:02:14,281  INFO
> > > > > > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > > > > > controller.PartitionStateMachine$DeleteTopicsListener -
> > > > > > > [DeleteTopicsListener on 6]: Starting topic deletion for topics
> > > > > > > mytopic
> > > > > > > 2015-01-26 07:02:14,282  INFO [delete-topics-thread-6]
> > > > > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > > > > [delete-topics-thread-6], Handling deletion for topics mytopic
> > > > > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > > > > [delete-topics-thread-6], Deletion of topic mytopic (re)started
> > > > > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > > > > controller.TopicDeletionManager - [Topic Deletion Manager 6],
> > Topic
> > > > > > > deletion callback for mytopic
> > > > > > > 2015-01-26 07:02:14,289  INFO [delete-topics-thread-6]
> > > > > > > controller.TopicDeletionManager - [Topic Deletion Manager 6],
> > > > > > > Partition deletion callback for [mytopic,0]
> > > > > > > 2015-01-26 07:02:14,295  INFO [delete-topics-thread-6]
> > > > > > > controller.ReplicaStateMachine - [Replica state machine on
> > > controller
> > > > > > > 6]: Invoking state change to OfflineReplica for replicas
> > > > > > >
> > > > > >
> > > >
> > >
> > [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Rep

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Jun Rao
Hi, Jason,

I am not sure that we should delay the 0.8.2 release. The reasons are (1)
There are other features such as the new java producer and Kafka-based
offset management that are potentially useful to people. (2) It may take
some time to completely fix the issue with deleting topic since it involves
wire protocol changes.

What we can do is probably just to document the limitation of delete topic
in 0.8.2.0 in our site.

Thanks,

Jun

On Mon, Jan 26, 2015 at 11:05 AM, Jason Rosenberg  wrote:

> I think this shortcoming should probably delay release of 0.8.2 until
> resolved, no?
>
> On Mon, Jan 26, 2015 at 2:01 PM, Guozhang Wang  wrote:
>
> > It then seems to me that delete-topic will not actually work "smoothly"
> > until create topic request is added since it is too much to require
> people
> > to turn off their clients while deleting topics. In this case shall we
> make
> > it clear in the release docs of in 0.8.2, or even still mark it as
> > not-supported?
> >
> > Guozhang
> >
> > On Mon, Jan 26, 2015 at 8:14 AM, Harsha  wrote:
> >
> > > Jun,
> > >   I made an attempt at fixing that issue as part of this JIRA
> > >   https://issues.apache.org/jira/browse/KAFKA-1507 .
> > > As Jay pointed out there should be admin api if there is more info on
> > > this api I am interested in adding/fixing this issue.
> > > Thanks,
> > > Harsha
> > >
> > > On Mon, Jan 26, 2015, at 07:28 AM, Jun Rao wrote:
> > > > Yes, that's the issue. Currently, topics can be auto-created on
> > > > TopicMetadataRequest, which can be issued from both the producer and
> > the
> > > > consumer. To prevent that, you would need to stop the producer and
> the
> > > > consumer before deleting a topic. We plan to address this issue once
> we
> > > > have a separate request for creating topics.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Mon, Jan 26, 2015 at 7:21 AM, Harsha  wrote:
> > > >
> > > > > There could be another case where if you have
> > auto.create.topics.enable
> > > > > to set to true ( its true by default) . Any TopicMetadataRequest
> can
> > > > > recreate topics. So if you issued a delete topic command and you
> have
> > > > > producers running or consumers? too which is issuing a
> > > > > TopicMetadataRequest than the topic will be recreated.
> > > > > -Harsha
> > > > >
> > > > > On Sun, Jan 25, 2015, at 11:26 PM, Jason Rosenberg wrote:
> > > > > > cversion did change (incremented by 2) when I issue the delete
> > > command.
> > > > > >
> > > > > > From the logs on the conroller broker (also the leader for the
> > > topic), it
> > > > > > looks like the delete proceeds, and then the topic gets recreated
> > > > > > immediately (highlighted in yellow). It appears maybe it’s due
> to a
> > > > > > consumer client app trying to consume the topic. Also, the
> consumer
> > > is
> > > > > > not
> > > > > > yet updated to 0.8.2 (it’s using 0.8.1.1), perhaps that’s part of
> > the
> > > > > > problem?
> > > > > >
> > > > > >
> > > > > > 2015-01-26 07:02:14,281  INFO
> > > > > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > > > > controller.PartitionStateMachine$DeleteTopicsListener -
> > > > > > [DeleteTopicsListener on 6]: Starting topic deletion for topics
> > > > > > mytopic
> > > > > > 2015-01-26 07:02:14,282  INFO [delete-topics-thread-6]
> > > > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > > > [delete-topics-thread-6], Handling deletion for topics mytopic
> > > > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > > > [delete-topics-thread-6], Deletion of topic mytopic (re)started
> > > > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > > > controller.TopicDeletionManager - [Topic Deletion Manager 6],
> Topic
> > > > > > deletion callback for mytopic
> > > > > > 2015-01-26 07:02:14,289  INFO [delete-topics-thread-6]
> > > > > > controller.TopicDeletionManager - [Topic Deletion Manager 6],
> > > > > > Partition deletion callback for [mytopic,0]
> > > > > > 2015-01-26 07:02:14,295  INFO [delete-topics-thread-6]
> > > > > > controller.ReplicaStateMachine - [Replica state machine on
> > controller
> > > > > > 6]: Invoking state change to OfflineReplica for replicas
> > > > > >
> > > > >
> > >
> >
> [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > > > > > 2015-01-26 07:02:14,303  INFO [delete-topics-thread-6]
> > > > > > controller.KafkaController - [Controller 6]: New leader and ISR
> for
> > > > > > partition [mytopic,0] is {"leader":6,"leader_epoch":1,"isr":[6]}
> > > > > > 2015-01-26 07:02:14,312  INFO [delete-topics-thread-6]
> > > > > > controller.KafkaController - [Controller 6]: New leader and ISR
> for
> > > > > > partition [mytopic,0] is {"leader":-1,"leader_epoch":2,"isr":[]}
> > > > > > 2015-01-26 07:02:14,313  INFO [delete-topics-thread-6]
> > > > > > controller.ReplicaStateMachine - [Replica state machine on
> > controller
>

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Jason Rosenberg
I think this shortcoming should probably delay release of 0.8.2 until
resolved, no?

On Mon, Jan 26, 2015 at 2:01 PM, Guozhang Wang  wrote:

> It then seems to me that delete-topic will not actually work "smoothly"
> until create topic request is added since it is too much to require people
> to turn off their clients while deleting topics. In this case shall we make
> it clear in the release docs of in 0.8.2, or even still mark it as
> not-supported?
>
> Guozhang
>
> On Mon, Jan 26, 2015 at 8:14 AM, Harsha  wrote:
>
> > Jun,
> >   I made an attempt at fixing that issue as part of this JIRA
> >   https://issues.apache.org/jira/browse/KAFKA-1507 .
> > As Jay pointed out there should be admin api if there is more info on
> > this api I am interested in adding/fixing this issue.
> > Thanks,
> > Harsha
> >
> > On Mon, Jan 26, 2015, at 07:28 AM, Jun Rao wrote:
> > > Yes, that's the issue. Currently, topics can be auto-created on
> > > TopicMetadataRequest, which can be issued from both the producer and
> the
> > > consumer. To prevent that, you would need to stop the producer and the
> > > consumer before deleting a topic. We plan to address this issue once we
> > > have a separate request for creating topics.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Jan 26, 2015 at 7:21 AM, Harsha  wrote:
> > >
> > > > There could be another case where if you have
> auto.create.topics.enable
> > > > to set to true ( its true by default) . Any TopicMetadataRequest can
> > > > recreate topics. So if you issued a delete topic command and you have
> > > > producers running or consumers? too which is issuing a
> > > > TopicMetadataRequest than the topic will be recreated.
> > > > -Harsha
> > > >
> > > > On Sun, Jan 25, 2015, at 11:26 PM, Jason Rosenberg wrote:
> > > > > cversion did change (incremented by 2) when I issue the delete
> > command.
> > > > >
> > > > > From the logs on the conroller broker (also the leader for the
> > topic), it
> > > > > looks like the delete proceeds, and then the topic gets recreated
> > > > > immediately (highlighted in yellow). It appears maybe it’s due to a
> > > > > consumer client app trying to consume the topic. Also, the consumer
> > is
> > > > > not
> > > > > yet updated to 0.8.2 (it’s using 0.8.1.1), perhaps that’s part of
> the
> > > > > problem?
> > > > >
> > > > >
> > > > > 2015-01-26 07:02:14,281  INFO
> > > > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > > > controller.PartitionStateMachine$DeleteTopicsListener -
> > > > > [DeleteTopicsListener on 6]: Starting topic deletion for topics
> > > > > mytopic
> > > > > 2015-01-26 07:02:14,282  INFO [delete-topics-thread-6]
> > > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > > [delete-topics-thread-6], Handling deletion for topics mytopic
> > > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > > [delete-topics-thread-6], Deletion of topic mytopic (re)started
> > > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > > controller.TopicDeletionManager - [Topic Deletion Manager 6], Topic
> > > > > deletion callback for mytopic
> > > > > 2015-01-26 07:02:14,289  INFO [delete-topics-thread-6]
> > > > > controller.TopicDeletionManager - [Topic Deletion Manager 6],
> > > > > Partition deletion callback for [mytopic,0]
> > > > > 2015-01-26 07:02:14,295  INFO [delete-topics-thread-6]
> > > > > controller.ReplicaStateMachine - [Replica state machine on
> controller
> > > > > 6]: Invoking state change to OfflineReplica for replicas
> > > > >
> > > >
> >
> [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > > > > 2015-01-26 07:02:14,303  INFO [delete-topics-thread-6]
> > > > > controller.KafkaController - [Controller 6]: New leader and ISR for
> > > > > partition [mytopic,0] is {"leader":6,"leader_epoch":1,"isr":[6]}
> > > > > 2015-01-26 07:02:14,312  INFO [delete-topics-thread-6]
> > > > > controller.KafkaController - [Controller 6]: New leader and ISR for
> > > > > partition [mytopic,0] is {"leader":-1,"leader_epoch":2,"isr":[]}
> > > > > 2015-01-26 07:02:14,313  INFO [delete-topics-thread-6]
> > > > > controller.ReplicaStateMachine - [Replica state machine on
> controller
> > > > > 6]: Invoking state change to ReplicaDeletionStarted for replicas
> > > > >
> > > >
> >
> [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > > > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-5]
> > > > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> > > > > Removed fetcher for partitions [mytopic,0]
> > > > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> > > > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> > > > > Removed fetcher for partitions [mytopic,0]
> > > > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> > > > > log.OffsetIndex - Deleting index
> > > > > /mypath/mytopic-0/

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Jason Rosenberg
So, this is rather disappointing, especially since topic deletion is really
the primary feature in 0.8.2 I'm interested in.  The topic I was trying to
delete above had no data for many months.  The consumer which is triggering
recreation of that topic has been restarted several times since that topic
stopped receiving data.

In our case, we have a large number of topics, that no longer receive
messages (many have not received messages in over 12 months).  However, we
have consumers that use a regex which matches multiple topics, some of
which are these empty zombie topics.  Thus, we have a chicken-and-egg
problem:

1.  Consumer uses a regex to discover matching topics.
2.  Consumer starts consuming 'topic.foo.which.is.empty'.  This results in
regular meta-data requests for data for that topic.
3.  'topic.foo.which.is.empty' is deleted.
4.  Consumer encounters an error trying to fetch this topic, so issues a
meta-data request to find the leader for this topic.
5.  Broker recreates the topic in response to this.

We can stop consumers for maintenance and do a batch delete, but it is
problematic, because there are realtime dependencies on those consumers
being up and running.

I expect this will necessarily result in my voting no for 0.8.2 RC2 (if I
have a vote :)).

Just read up on KAFKA-1507, and left my 2 cents:

"I think relegating topic creation to an admin client would be very
limitiing. It's extremely useful to have a self-service system where new
applications can just create a new topic on demand (with reasonable
defaults), without the need for an admin to come in and prepare topics
ahead of a code release (leave that to dba's managing transactional
databases!).

I do like the idea of an automatic create topic request from a producer, in
response to a topic not found exception, rather than auto-creating topics
from meta-data requests (which happens asynchronously and causes the
initial meta data request to fail usually!). Consumers should never create
a topic, I should think."

On Mon, Jan 26, 2015 at 11:14 AM, Harsha  wrote:

> Jun,
>   I made an attempt at fixing that issue as part of this JIRA
>   https://issues.apache.org/jira/browse/KAFKA-1507 .
> As Jay pointed out there should be admin api if there is more info on
> this api I am interested in adding/fixing this issue.
> Thanks,
> Harsha
>
> On Mon, Jan 26, 2015, at 07:28 AM, Jun Rao wrote:
> > Yes, that's the issue. Currently, topics can be auto-created on
> > TopicMetadataRequest, which can be issued from both the producer and the
> > consumer. To prevent that, you would need to stop the producer and the
> > consumer before deleting a topic. We plan to address this issue once we
> > have a separate request for creating topics.
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Jan 26, 2015 at 7:21 AM, Harsha  wrote:
> >
> > > There could be another case where if you have auto.create.topics.enable
> > > to set to true ( its true by default) . Any TopicMetadataRequest can
> > > recreate topics. So if you issued a delete topic command and you have
> > > producers running or consumers? too which is issuing a
> > > TopicMetadataRequest than the topic will be recreated.
> > > -Harsha
> > >
> > > On Sun, Jan 25, 2015, at 11:26 PM, Jason Rosenberg wrote:
> > > > cversion did change (incremented by 2) when I issue the delete
> command.
> > > >
> > > > From the logs on the conroller broker (also the leader for the
> topic), it
> > > > looks like the delete proceeds, and then the topic gets recreated
> > > > immediately (highlighted in yellow). It appears maybe it’s due to a
> > > > consumer client app trying to consume the topic. Also, the consumer
> is
> > > > not
> > > > yet updated to 0.8.2 (it’s using 0.8.1.1), perhaps that’s part of the
> > > > problem?
> > > >
> > > >
> > > > 2015-01-26 07:02:14,281  INFO
> > > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > > controller.PartitionStateMachine$DeleteTopicsListener -
> > > > [DeleteTopicsListener on 6]: Starting topic deletion for topics
> > > > mytopic
> > > > 2015-01-26 07:02:14,282  INFO [delete-topics-thread-6]
> > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > [delete-topics-thread-6], Handling deletion for topics mytopic
> > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > [delete-topics-thread-6], Deletion of topic mytopic (re)started
> > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > controller.TopicDeletionManager - [Topic Deletion Manager 6], Topic
> > > > deletion callback for mytopic
> > > > 2015-01-26 07:02:14,289  INFO [delete-topics-thread-6]
> > > > controller.TopicDeletionManager - [Topic Deletion Manager 6],
> > > > Partition deletion callback for [mytopic,0]
> > > > 2015-01-26 07:02:14,295  INFO [delete-topics-thread-6]
> > > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > > 6]: Invoking state change to OfflineReplica f

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Guozhang Wang
It then seems to me that delete-topic will not actually work "smoothly"
until create topic request is added since it is too much to require people
to turn off their clients while deleting topics. In this case shall we make
it clear in the release docs of in 0.8.2, or even still mark it as
not-supported?

Guozhang

On Mon, Jan 26, 2015 at 8:14 AM, Harsha  wrote:

> Jun,
>   I made an attempt at fixing that issue as part of this JIRA
>   https://issues.apache.org/jira/browse/KAFKA-1507 .
> As Jay pointed out there should be admin api if there is more info on
> this api I am interested in adding/fixing this issue.
> Thanks,
> Harsha
>
> On Mon, Jan 26, 2015, at 07:28 AM, Jun Rao wrote:
> > Yes, that's the issue. Currently, topics can be auto-created on
> > TopicMetadataRequest, which can be issued from both the producer and the
> > consumer. To prevent that, you would need to stop the producer and the
> > consumer before deleting a topic. We plan to address this issue once we
> > have a separate request for creating topics.
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Jan 26, 2015 at 7:21 AM, Harsha  wrote:
> >
> > > There could be another case where if you have auto.create.topics.enable
> > > to set to true ( its true by default) . Any TopicMetadataRequest can
> > > recreate topics. So if you issued a delete topic command and you have
> > > producers running or consumers? too which is issuing a
> > > TopicMetadataRequest than the topic will be recreated.
> > > -Harsha
> > >
> > > On Sun, Jan 25, 2015, at 11:26 PM, Jason Rosenberg wrote:
> > > > cversion did change (incremented by 2) when I issue the delete
> command.
> > > >
> > > > From the logs on the conroller broker (also the leader for the
> topic), it
> > > > looks like the delete proceeds, and then the topic gets recreated
> > > > immediately (highlighted in yellow). It appears maybe it’s due to a
> > > > consumer client app trying to consume the topic. Also, the consumer
> is
> > > > not
> > > > yet updated to 0.8.2 (it’s using 0.8.1.1), perhaps that’s part of the
> > > > problem?
> > > >
> > > >
> > > > 2015-01-26 07:02:14,281  INFO
> > > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > > controller.PartitionStateMachine$DeleteTopicsListener -
> > > > [DeleteTopicsListener on 6]: Starting topic deletion for topics
> > > > mytopic
> > > > 2015-01-26 07:02:14,282  INFO [delete-topics-thread-6]
> > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > [delete-topics-thread-6], Handling deletion for topics mytopic
> > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > [delete-topics-thread-6], Deletion of topic mytopic (re)started
> > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > controller.TopicDeletionManager - [Topic Deletion Manager 6], Topic
> > > > deletion callback for mytopic
> > > > 2015-01-26 07:02:14,289  INFO [delete-topics-thread-6]
> > > > controller.TopicDeletionManager - [Topic Deletion Manager 6],
> > > > Partition deletion callback for [mytopic,0]
> > > > 2015-01-26 07:02:14,295  INFO [delete-topics-thread-6]
> > > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > > 6]: Invoking state change to OfflineReplica for replicas
> > > >
> > >
> [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > > > 2015-01-26 07:02:14,303  INFO [delete-topics-thread-6]
> > > > controller.KafkaController - [Controller 6]: New leader and ISR for
> > > > partition [mytopic,0] is {"leader":6,"leader_epoch":1,"isr":[6]}
> > > > 2015-01-26 07:02:14,312  INFO [delete-topics-thread-6]
> > > > controller.KafkaController - [Controller 6]: New leader and ISR for
> > > > partition [mytopic,0] is {"leader":-1,"leader_epoch":2,"isr":[]}
> > > > 2015-01-26 07:02:14,313  INFO [delete-topics-thread-6]
> > > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > > 6]: Invoking state change to ReplicaDeletionStarted for replicas
> > > >
> > >
> [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-5]
> > > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> > > > Removed fetcher for partitions [mytopic,0]
> > > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> > > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> > > > Removed fetcher for partitions [mytopic,0]
> > > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> > > > log.OffsetIndex - Deleting index
> > > > /mypath/mytopic-0/.index
> > > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> log.LogManager
> > > > - Deleted log for partition [mytopic,0] in /mypath/mytopic-0.
> > > > 2015-01-26 07:02:14,314  INFO [Controller-6-to-broker-6-send-thread]
> > > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > > 6]: Invoking stat

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Ari Flink
unsubscribe

On Mon, Jan 26, 2015 at 8:14 AM, Harsha  wrote:

> Jun,
>   I made an attempt at fixing that issue as part of this JIRA
>   https://issues.apache.org/jira/browse/KAFKA-1507 .
> As Jay pointed out there should be admin api if there is more info on
> this api I am interested in adding/fixing this issue.
> Thanks,
> Harsha
>
> On Mon, Jan 26, 2015, at 07:28 AM, Jun Rao wrote:
> > Yes, that's the issue. Currently, topics can be auto-created on
> > TopicMetadataRequest, which can be issued from both the producer and the
> > consumer. To prevent that, you would need to stop the producer and the
> > consumer before deleting a topic. We plan to address this issue once we
> > have a separate request for creating topics.
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Jan 26, 2015 at 7:21 AM, Harsha  wrote:
> >
> > > There could be another case where if you have auto.create.topics.enable
> > > to set to true ( its true by default) . Any TopicMetadataRequest can
> > > recreate topics. So if you issued a delete topic command and you have
> > > producers running or consumers? too which is issuing a
> > > TopicMetadataRequest than the topic will be recreated.
> > > -Harsha
> > >
> > > On Sun, Jan 25, 2015, at 11:26 PM, Jason Rosenberg wrote:
> > > > cversion did change (incremented by 2) when I issue the delete
> command.
> > > >
> > > > From the logs on the conroller broker (also the leader for the
> topic), it
> > > > looks like the delete proceeds, and then the topic gets recreated
> > > > immediately (highlighted in yellow). It appears maybe it’s due to a
> > > > consumer client app trying to consume the topic. Also, the consumer
> is
> > > > not
> > > > yet updated to 0.8.2 (it’s using 0.8.1.1), perhaps that’s part of the
> > > > problem?
> > > >
> > > >
> > > > 2015-01-26 07:02:14,281  INFO
> > > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > > controller.PartitionStateMachine$DeleteTopicsListener -
> > > > [DeleteTopicsListener on 6]: Starting topic deletion for topics
> > > > mytopic
> > > > 2015-01-26 07:02:14,282  INFO [delete-topics-thread-6]
> > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > [delete-topics-thread-6], Handling deletion for topics mytopic
> > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > [delete-topics-thread-6], Deletion of topic mytopic (re)started
> > > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > > controller.TopicDeletionManager - [Topic Deletion Manager 6], Topic
> > > > deletion callback for mytopic
> > > > 2015-01-26 07:02:14,289  INFO [delete-topics-thread-6]
> > > > controller.TopicDeletionManager - [Topic Deletion Manager 6],
> > > > Partition deletion callback for [mytopic,0]
> > > > 2015-01-26 07:02:14,295  INFO [delete-topics-thread-6]
> > > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > > 6]: Invoking state change to OfflineReplica for replicas
> > > >
> > >
> [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > > > 2015-01-26 07:02:14,303  INFO [delete-topics-thread-6]
> > > > controller.KafkaController - [Controller 6]: New leader and ISR for
> > > > partition [mytopic,0] is {"leader":6,"leader_epoch":1,"isr":[6]}
> > > > 2015-01-26 07:02:14,312  INFO [delete-topics-thread-6]
> > > > controller.KafkaController - [Controller 6]: New leader and ISR for
> > > > partition [mytopic,0] is {"leader":-1,"leader_epoch":2,"isr":[]}
> > > > 2015-01-26 07:02:14,313  INFO [delete-topics-thread-6]
> > > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > > 6]: Invoking state change to ReplicaDeletionStarted for replicas
> > > >
> > >
> [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-5]
> > > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> > > > Removed fetcher for partitions [mytopic,0]
> > > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> > > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> > > > Removed fetcher for partitions [mytopic,0]
> > > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> > > > log.OffsetIndex - Deleting index
> > > > /mypath/mytopic-0/.index
> > > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> log.LogManager
> > > > - Deleted log for partition [mytopic,0] in /mypath/mytopic-0.
> > > > 2015-01-26 07:02:14,314  INFO [Controller-6-to-broker-6-send-thread]
> > > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > > 6]: Invoking state change to ReplicaDeletionSuccessful for replicas
> > > > [Topic=mytopic,Partition=0,Replica=6]
> > > > 2015-01-26 07:02:14,314  INFO [delete-topics-thread-6]
> > > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > > [delete-topics-thread-6], Handling deletion for topics mytopic
> > > > 201

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Harsha
Jun,
  I made an attempt at fixing that issue as part of this JIRA
  https://issues.apache.org/jira/browse/KAFKA-1507 . 
As Jay pointed out there should be admin api if there is more info on
this api I am interested in adding/fixing this issue.
Thanks,
Harsha

On Mon, Jan 26, 2015, at 07:28 AM, Jun Rao wrote:
> Yes, that's the issue. Currently, topics can be auto-created on
> TopicMetadataRequest, which can be issued from both the producer and the
> consumer. To prevent that, you would need to stop the producer and the
> consumer before deleting a topic. We plan to address this issue once we
> have a separate request for creating topics.
> 
> Thanks,
> 
> Jun
> 
> On Mon, Jan 26, 2015 at 7:21 AM, Harsha  wrote:
> 
> > There could be another case where if you have auto.create.topics.enable
> > to set to true ( its true by default) . Any TopicMetadataRequest can
> > recreate topics. So if you issued a delete topic command and you have
> > producers running or consumers? too which is issuing a
> > TopicMetadataRequest than the topic will be recreated.
> > -Harsha
> >
> > On Sun, Jan 25, 2015, at 11:26 PM, Jason Rosenberg wrote:
> > > cversion did change (incremented by 2) when I issue the delete command.
> > >
> > > From the logs on the conroller broker (also the leader for the topic), it
> > > looks like the delete proceeds, and then the topic gets recreated
> > > immediately (highlighted in yellow). It appears maybe it’s due to a
> > > consumer client app trying to consume the topic. Also, the consumer is
> > > not
> > > yet updated to 0.8.2 (it’s using 0.8.1.1), perhaps that’s part of the
> > > problem?
> > >
> > >
> > > 2015-01-26 07:02:14,281  INFO
> > > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > > controller.PartitionStateMachine$DeleteTopicsListener -
> > > [DeleteTopicsListener on 6]: Starting topic deletion for topics
> > > mytopic
> > > 2015-01-26 07:02:14,282  INFO [delete-topics-thread-6]
> > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > [delete-topics-thread-6], Handling deletion for topics mytopic
> > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > [delete-topics-thread-6], Deletion of topic mytopic (re)started
> > > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > > controller.TopicDeletionManager - [Topic Deletion Manager 6], Topic
> > > deletion callback for mytopic
> > > 2015-01-26 07:02:14,289  INFO [delete-topics-thread-6]
> > > controller.TopicDeletionManager - [Topic Deletion Manager 6],
> > > Partition deletion callback for [mytopic,0]
> > > 2015-01-26 07:02:14,295  INFO [delete-topics-thread-6]
> > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > 6]: Invoking state change to OfflineReplica for replicas
> > >
> > [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > > 2015-01-26 07:02:14,303  INFO [delete-topics-thread-6]
> > > controller.KafkaController - [Controller 6]: New leader and ISR for
> > > partition [mytopic,0] is {"leader":6,"leader_epoch":1,"isr":[6]}
> > > 2015-01-26 07:02:14,312  INFO [delete-topics-thread-6]
> > > controller.KafkaController - [Controller 6]: New leader and ISR for
> > > partition [mytopic,0] is {"leader":-1,"leader_epoch":2,"isr":[]}
> > > 2015-01-26 07:02:14,313  INFO [delete-topics-thread-6]
> > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > 6]: Invoking state change to ReplicaDeletionStarted for replicas
> > >
> > [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-5]
> > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> > > Removed fetcher for partitions [mytopic,0]
> > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> > > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> > > Removed fetcher for partitions [mytopic,0]
> > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> > > log.OffsetIndex - Deleting index
> > > /mypath/mytopic-0/.index
> > > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7] log.LogManager
> > > - Deleted log for partition [mytopic,0] in /mypath/mytopic-0.
> > > 2015-01-26 07:02:14,314  INFO [Controller-6-to-broker-6-send-thread]
> > > controller.ReplicaStateMachine - [Replica state machine on controller
> > > 6]: Invoking state change to ReplicaDeletionSuccessful for replicas
> > > [Topic=mytopic,Partition=0,Replica=6]
> > > 2015-01-26 07:02:14,314  INFO [delete-topics-thread-6]
> > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > [delete-topics-thread-6], Handling deletion for topics mytopic
> > > 2015-01-26 07:02:14,316  INFO [delete-topics-thread-6]
> > > controller.TopicDeletionManager$DeleteTopicsThread -
> > > [delete-topics-thread-6], Deletion for replicas 7 for partition
> > > [mytopic,0] of topic mytopic in progress
> > > 2015-01-26 07:02:1

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Jun Rao
Yes, that's the issue. Currently, topics can be auto-created on
TopicMetadataRequest, which can be issued from both the producer and the
consumer. To prevent that, you would need to stop the producer and the
consumer before deleting a topic. We plan to address this issue once we
have a separate request for creating topics.

Thanks,

Jun

On Mon, Jan 26, 2015 at 7:21 AM, Harsha  wrote:

> There could be another case where if you have auto.create.topics.enable
> to set to true ( its true by default) . Any TopicMetadataRequest can
> recreate topics. So if you issued a delete topic command and you have
> producers running or consumers? too which is issuing a
> TopicMetadataRequest than the topic will be recreated.
> -Harsha
>
> On Sun, Jan 25, 2015, at 11:26 PM, Jason Rosenberg wrote:
> > cversion did change (incremented by 2) when I issue the delete command.
> >
> > From the logs on the conroller broker (also the leader for the topic), it
> > looks like the delete proceeds, and then the topic gets recreated
> > immediately (highlighted in yellow). It appears maybe it’s due to a
> > consumer client app trying to consume the topic. Also, the consumer is
> > not
> > yet updated to 0.8.2 (it’s using 0.8.1.1), perhaps that’s part of the
> > problem?
> >
> >
> > 2015-01-26 07:02:14,281  INFO
> > [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> > controller.PartitionStateMachine$DeleteTopicsListener -
> > [DeleteTopicsListener on 6]: Starting topic deletion for topics
> > mytopic
> > 2015-01-26 07:02:14,282  INFO [delete-topics-thread-6]
> > controller.TopicDeletionManager$DeleteTopicsThread -
> > [delete-topics-thread-6], Handling deletion for topics mytopic
> > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > controller.TopicDeletionManager$DeleteTopicsThread -
> > [delete-topics-thread-6], Deletion of topic mytopic (re)started
> > 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> > controller.TopicDeletionManager - [Topic Deletion Manager 6], Topic
> > deletion callback for mytopic
> > 2015-01-26 07:02:14,289  INFO [delete-topics-thread-6]
> > controller.TopicDeletionManager - [Topic Deletion Manager 6],
> > Partition deletion callback for [mytopic,0]
> > 2015-01-26 07:02:14,295  INFO [delete-topics-thread-6]
> > controller.ReplicaStateMachine - [Replica state machine on controller
> > 6]: Invoking state change to OfflineReplica for replicas
> >
> [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > 2015-01-26 07:02:14,303  INFO [delete-topics-thread-6]
> > controller.KafkaController - [Controller 6]: New leader and ISR for
> > partition [mytopic,0] is {"leader":6,"leader_epoch":1,"isr":[6]}
> > 2015-01-26 07:02:14,312  INFO [delete-topics-thread-6]
> > controller.KafkaController - [Controller 6]: New leader and ISR for
> > partition [mytopic,0] is {"leader":-1,"leader_epoch":2,"isr":[]}
> > 2015-01-26 07:02:14,313  INFO [delete-topics-thread-6]
> > controller.ReplicaStateMachine - [Replica state machine on controller
> > 6]: Invoking state change to ReplicaDeletionStarted for replicas
> >
> [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-5]
> > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> > Removed fetcher for partitions [mytopic,0]
> > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> > server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> > Removed fetcher for partitions [mytopic,0]
> > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> > log.OffsetIndex - Deleting index
> > /mypath/mytopic-0/.index
> > 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7] log.LogManager
> > - Deleted log for partition [mytopic,0] in /mypath/mytopic-0.
> > 2015-01-26 07:02:14,314  INFO [Controller-6-to-broker-6-send-thread]
> > controller.ReplicaStateMachine - [Replica state machine on controller
> > 6]: Invoking state change to ReplicaDeletionSuccessful for replicas
> > [Topic=mytopic,Partition=0,Replica=6]
> > 2015-01-26 07:02:14,314  INFO [delete-topics-thread-6]
> > controller.TopicDeletionManager$DeleteTopicsThread -
> > [delete-topics-thread-6], Handling deletion for topics mytopic
> > 2015-01-26 07:02:14,316  INFO [delete-topics-thread-6]
> > controller.TopicDeletionManager$DeleteTopicsThread -
> > [delete-topics-thread-6], Deletion for replicas 7 for partition
> > [mytopic,0] of topic mytopic in progress
> > 2015-01-26 07:02:14,316  INFO [Controller-6-to-broker-7-send-thread]
> > controller.ReplicaStateMachine - [Replica state machine on controller
> > 6]: Invoking state change to ReplicaDeletionSuccessful for replicas
> > [Topic=mytopic,Partition=0,Replica=7]
> > 2015-01-26 07:02:14,316  INFO [delete-topics-thread-6]
> > controller.TopicDeletionManager$DeleteTopicsThread -
> > [delete-topics-thread-6], Handling deletion for topics mytopic
> > 2015-01-26 07:02:14,318  INFO [delete-topics-thread-6]
> > controll

Re: unable to delete topic with 0.8.2 rc2

2015-01-26 Thread Harsha
There could be another case where if you have auto.create.topics.enable
to set to true ( its true by default) . Any TopicMetadataRequest can
recreate topics. So if you issued a delete topic command and you have
producers running or consumers? too which is issuing a
TopicMetadataRequest than the topic will be recreated.
-Harsha

On Sun, Jan 25, 2015, at 11:26 PM, Jason Rosenberg wrote:
> cversion did change (incremented by 2) when I issue the delete command.
> 
> From the logs on the conroller broker (also the leader for the topic), it
> looks like the delete proceeds, and then the topic gets recreated
> immediately (highlighted in yellow). It appears maybe it’s due to a
> consumer client app trying to consume the topic. Also, the consumer is
> not
> yet updated to 0.8.2 (it’s using 0.8.1.1), perhaps that’s part of the
> problem?
> 
> 
> 2015-01-26 07:02:14,281  INFO
> [ZkClient-EventThread-21-myzkserver:12345/mynamespace]
> controller.PartitionStateMachine$DeleteTopicsListener -
> [DeleteTopicsListener on 6]: Starting topic deletion for topics
> mytopic
> 2015-01-26 07:02:14,282  INFO [delete-topics-thread-6]
> controller.TopicDeletionManager$DeleteTopicsThread -
> [delete-topics-thread-6], Handling deletion for topics mytopic
> 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> controller.TopicDeletionManager$DeleteTopicsThread -
> [delete-topics-thread-6], Deletion of topic mytopic (re)started
> 2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
> controller.TopicDeletionManager - [Topic Deletion Manager 6], Topic
> deletion callback for mytopic
> 2015-01-26 07:02:14,289  INFO [delete-topics-thread-6]
> controller.TopicDeletionManager - [Topic Deletion Manager 6],
> Partition deletion callback for [mytopic,0]
> 2015-01-26 07:02:14,295  INFO [delete-topics-thread-6]
> controller.ReplicaStateMachine - [Replica state machine on controller
> 6]: Invoking state change to OfflineReplica for replicas
> [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> 2015-01-26 07:02:14,303  INFO [delete-topics-thread-6]
> controller.KafkaController - [Controller 6]: New leader and ISR for
> partition [mytopic,0] is {"leader":6,"leader_epoch":1,"isr":[6]}
> 2015-01-26 07:02:14,312  INFO [delete-topics-thread-6]
> controller.KafkaController - [Controller 6]: New leader and ISR for
> partition [mytopic,0] is {"leader":-1,"leader_epoch":2,"isr":[]}
> 2015-01-26 07:02:14,313  INFO [delete-topics-thread-6]
> controller.ReplicaStateMachine - [Replica state machine on controller
> 6]: Invoking state change to ReplicaDeletionStarted for replicas
> [Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
> 2015-01-26 07:02:14,313  INFO [kafka-request-handler-5]
> server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> Removed fetcher for partitions [mytopic,0]
> 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
> Removed fetcher for partitions [mytopic,0]
> 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
> log.OffsetIndex - Deleting index
> /mypath/mytopic-0/.index
> 2015-01-26 07:02:14,313  INFO [kafka-request-handler-7] log.LogManager
> - Deleted log for partition [mytopic,0] in /mypath/mytopic-0.
> 2015-01-26 07:02:14,314  INFO [Controller-6-to-broker-6-send-thread]
> controller.ReplicaStateMachine - [Replica state machine on controller
> 6]: Invoking state change to ReplicaDeletionSuccessful for replicas
> [Topic=mytopic,Partition=0,Replica=6]
> 2015-01-26 07:02:14,314  INFO [delete-topics-thread-6]
> controller.TopicDeletionManager$DeleteTopicsThread -
> [delete-topics-thread-6], Handling deletion for topics mytopic
> 2015-01-26 07:02:14,316  INFO [delete-topics-thread-6]
> controller.TopicDeletionManager$DeleteTopicsThread -
> [delete-topics-thread-6], Deletion for replicas 7 for partition
> [mytopic,0] of topic mytopic in progress
> 2015-01-26 07:02:14,316  INFO [Controller-6-to-broker-7-send-thread]
> controller.ReplicaStateMachine - [Replica state machine on controller
> 6]: Invoking state change to ReplicaDeletionSuccessful for replicas
> [Topic=mytopic,Partition=0,Replica=7]
> 2015-01-26 07:02:14,316  INFO [delete-topics-thread-6]
> controller.TopicDeletionManager$DeleteTopicsThread -
> [delete-topics-thread-6], Handling deletion for topics mytopic
> 2015-01-26 07:02:14,318  INFO [delete-topics-thread-6]
> controller.ReplicaStateMachine - [Replica state machine on controller
> 6]: Invoking state change to NonExistentReplica for replicas
> [Topic=mytopic,Partition=0,Replica=6],[Topic=mytopic,Partition=0,Replica=7]
> 2015-01-26 07:02:14,318  INFO [delete-topics-thread-6]
> controller.PartitionStateMachine - [Partition state machine on
> Controller 6]: Invoking state change to OfflinePartition for
> partitions [mytopic,0]
> 2015-01-26 07:02:14,318  INFO [delete-topics-thread-6]
> controller.PartitionStateMachine - [Partition state machine on
> Controller 6]:

Re: unable to delete topic with 0.8.2 rc2

2015-01-25 Thread Jason Rosenberg
cversion did change (incremented by 2) when I issue the delete command.

>From the logs on the conroller broker (also the leader for the topic), it
looks like the delete proceeds, and then the topic gets recreated
immediately (highlighted in yellow). It appears maybe it’s due to a
consumer client app trying to consume the topic. Also, the consumer is not
yet updated to 0.8.2 (it’s using 0.8.1.1), perhaps that’s part of the
problem?


2015-01-26 07:02:14,281  INFO
[ZkClient-EventThread-21-myzkserver:12345/mynamespace]
controller.PartitionStateMachine$DeleteTopicsListener -
[DeleteTopicsListener on 6]: Starting topic deletion for topics
mytopic
2015-01-26 07:02:14,282  INFO [delete-topics-thread-6]
controller.TopicDeletionManager$DeleteTopicsThread -
[delete-topics-thread-6], Handling deletion for topics mytopic
2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
controller.TopicDeletionManager$DeleteTopicsThread -
[delete-topics-thread-6], Deletion of topic mytopic (re)started
2015-01-26 07:02:14,286  INFO [delete-topics-thread-6]
controller.TopicDeletionManager - [Topic Deletion Manager 6], Topic
deletion callback for mytopic
2015-01-26 07:02:14,289  INFO [delete-topics-thread-6]
controller.TopicDeletionManager - [Topic Deletion Manager 6],
Partition deletion callback for [mytopic,0]
2015-01-26 07:02:14,295  INFO [delete-topics-thread-6]
controller.ReplicaStateMachine - [Replica state machine on controller
6]: Invoking state change to OfflineReplica for replicas
[Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
2015-01-26 07:02:14,303  INFO [delete-topics-thread-6]
controller.KafkaController - [Controller 6]: New leader and ISR for
partition [mytopic,0] is {"leader":6,"leader_epoch":1,"isr":[6]}
2015-01-26 07:02:14,312  INFO [delete-topics-thread-6]
controller.KafkaController - [Controller 6]: New leader and ISR for
partition [mytopic,0] is {"leader":-1,"leader_epoch":2,"isr":[]}
2015-01-26 07:02:14,313  INFO [delete-topics-thread-6]
controller.ReplicaStateMachine - [Replica state machine on controller
6]: Invoking state change to ReplicaDeletionStarted for replicas
[Topic=mytopic,Partition=0,Replica=7],[Topic=mytopic,Partition=0,Replica=6]
2015-01-26 07:02:14,313  INFO [kafka-request-handler-5]
server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
Removed fetcher for partitions [mytopic,0]
2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
server.ReplicaFetcherManager - [ReplicaFetcherManager on broker 6]
Removed fetcher for partitions [mytopic,0]
2015-01-26 07:02:14,313  INFO [kafka-request-handler-7]
log.OffsetIndex - Deleting index
/mypath/mytopic-0/.index
2015-01-26 07:02:14,313  INFO [kafka-request-handler-7] log.LogManager
- Deleted log for partition [mytopic,0] in /mypath/mytopic-0.
2015-01-26 07:02:14,314  INFO [Controller-6-to-broker-6-send-thread]
controller.ReplicaStateMachine - [Replica state machine on controller
6]: Invoking state change to ReplicaDeletionSuccessful for replicas
[Topic=mytopic,Partition=0,Replica=6]
2015-01-26 07:02:14,314  INFO [delete-topics-thread-6]
controller.TopicDeletionManager$DeleteTopicsThread -
[delete-topics-thread-6], Handling deletion for topics mytopic
2015-01-26 07:02:14,316  INFO [delete-topics-thread-6]
controller.TopicDeletionManager$DeleteTopicsThread -
[delete-topics-thread-6], Deletion for replicas 7 for partition
[mytopic,0] of topic mytopic in progress
2015-01-26 07:02:14,316  INFO [Controller-6-to-broker-7-send-thread]
controller.ReplicaStateMachine - [Replica state machine on controller
6]: Invoking state change to ReplicaDeletionSuccessful for replicas
[Topic=mytopic,Partition=0,Replica=7]
2015-01-26 07:02:14,316  INFO [delete-topics-thread-6]
controller.TopicDeletionManager$DeleteTopicsThread -
[delete-topics-thread-6], Handling deletion for topics mytopic
2015-01-26 07:02:14,318  INFO [delete-topics-thread-6]
controller.ReplicaStateMachine - [Replica state machine on controller
6]: Invoking state change to NonExistentReplica for replicas
[Topic=mytopic,Partition=0,Replica=6],[Topic=mytopic,Partition=0,Replica=7]
2015-01-26 07:02:14,318  INFO [delete-topics-thread-6]
controller.PartitionStateMachine - [Partition state machine on
Controller 6]: Invoking state change to OfflinePartition for
partitions [mytopic,0]
2015-01-26 07:02:14,318  INFO [delete-topics-thread-6]
controller.PartitionStateMachine - [Partition state machine on
Controller 6]: Invoking state change to NonExistentPartition for
partitions [mytopic,0]
2015-01-26 07:02:14,325  WARN [kafka-request-handler-2]
server.ReplicaManager - [Replica Manager on Broker 6]: Fetch request
with correlation id 2619442 from client ReplicaFetcherThread-0-6 on
partition [mytopic,0] failed due to Partition [mytopic,0] doesn't
exist on 6
2015-01-26 07:02:14,371  INFO [delete-topics-thread-6]
controller.TopicDeletionManager$DeleteTopicsThread -
[delete-topics-thread-6], Deletion of topic mytopic successfully
completed
2015-01-26 07:02:14,376 

Re: unable to delete topic with 0.8.2 rc2

2015-01-25 Thread Jun Rao
Could you do a get on /admin/deleted_topics and see if its cversion changes
after issuing the delete topic command? This will tell us whether the
path /admin/deleted_topics//mytopic
is actually created or not. Anything in the controller log that indicates a
delete topic event is fired?

Thanks,

Jun

On Sun, Jan 25, 2015 at 9:55 PM, Jason Rosenberg  wrote:

> yes
>
> On Mon, Jan 26, 2015 at 12:18 AM, Jun Rao  wrote:
>
> > Do you have delete.topic.enable turned on in all brokers?
> >
> > Thanks,
> >
> > Jun
> >
> > On Sun, Jan 25, 2015 at 7:56 PM, Jason Rosenberg 
> wrote:
> >
> > > So far, I have been unable to get delete topic to work, with release
> > > candidate 2 for 0.8.2.
> > >
> > > It worked ok when I ran it in the debugger locally, on a single node
> > > instance. But when I run it in our staging environment, it is not
> > > successfully even marking the topic for delete, for some reason.
> > >
> > > I am setting delete.topic.enable to true (and see the confirmation of
> > this
> > > in the startup logs, e.g.):
> > >
> > > INFO [main] utils.VerifiableProperties - Property delete.topic.enable
> > > is overridden to true
> > >
> > > I run this command:
> > >
> > > java -cp app.jar kafka.admin.TopicCommand --zookeeper
> > > myzkconnect:12345/mynamespace --delete --topic mytopic
> > >
> > > log4j:WARN No appenders could be found for logger
> > > (org.I0Itec.zkclient.ZkConnection).
> > > log4j:WARN Please initialize the log4j system properly.
> > > log4j:WARN No appenders could be found for logger
> > > (org.I0Itec.zkclient.ZkEventThread).
> > > log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig
> > > for more info.
> > > log4j:WARN Please initialize the log4j system properly.
> > > log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig
> > > for more info.
> > > Topic mytopic is marked for deletion.
> > > Note: This will have no impact if delete.topic.enable is not set to
> true.
> > >
> > > I then do a —list which should at least show the topic marked for
> > deletion:
> > >
> > > java -cp app.jar kafka.admin.TopicCommand --zookeeper
> > > myzkconnect:12345/mynamespace --list --topic mytopic
> > >
> > > log4j:WARN No appenders could be found for logger
> > > (org.I0Itec.zkclient.ZkConnection).
> > > log4j:WARN No appenders could be found for logger
> > > (org.I0Itec.zkclient.ZkEventThread).
> > > log4j:WARN Please initialize the log4j system properly.
> > > log4j:WARN Please initialize the log4j system properly.
> > > log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig
> > > for more info.
> > > log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig
> > > for more info.
> > > mytopic
> > >
> > > Note, it doesn’t list it as ‘marked for deletion’. Furthermore, waiting
> > > multiple hours still doesn’t result in the topic being deleted.
> > >
> > > The topic has 1 partition, 2 replicas, and no data stored.
> > >
> > > In zookeeper, the /admin/deleted_topics/ path is empty.
> > >
> > > The zookeeper code looks pretty straightforward, but for some reason is
> > not
> > > writing the deleted_topics path.  We are running zookeeper 3.4.6.
> > >
> > > Thoughts?
> > >
> > > Jason
> > > ​
> > >
> >
>


Re: unable to delete topic with 0.8.2 rc2

2015-01-25 Thread Jason Rosenberg
yes

On Mon, Jan 26, 2015 at 12:18 AM, Jun Rao  wrote:

> Do you have delete.topic.enable turned on in all brokers?
>
> Thanks,
>
> Jun
>
> On Sun, Jan 25, 2015 at 7:56 PM, Jason Rosenberg  wrote:
>
> > So far, I have been unable to get delete topic to work, with release
> > candidate 2 for 0.8.2.
> >
> > It worked ok when I ran it in the debugger locally, on a single node
> > instance. But when I run it in our staging environment, it is not
> > successfully even marking the topic for delete, for some reason.
> >
> > I am setting delete.topic.enable to true (and see the confirmation of
> this
> > in the startup logs, e.g.):
> >
> > INFO [main] utils.VerifiableProperties - Property delete.topic.enable
> > is overridden to true
> >
> > I run this command:
> >
> > java -cp app.jar kafka.admin.TopicCommand --zookeeper
> > myzkconnect:12345/mynamespace --delete --topic mytopic
> >
> > log4j:WARN No appenders could be found for logger
> > (org.I0Itec.zkclient.ZkConnection).
> > log4j:WARN Please initialize the log4j system properly.
> > log4j:WARN No appenders could be found for logger
> > (org.I0Itec.zkclient.ZkEventThread).
> > log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig
> > for more info.
> > log4j:WARN Please initialize the log4j system properly.
> > log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig
> > for more info.
> > Topic mytopic is marked for deletion.
> > Note: This will have no impact if delete.topic.enable is not set to true.
> >
> > I then do a —list which should at least show the topic marked for
> deletion:
> >
> > java -cp app.jar kafka.admin.TopicCommand --zookeeper
> > myzkconnect:12345/mynamespace --list --topic mytopic
> >
> > log4j:WARN No appenders could be found for logger
> > (org.I0Itec.zkclient.ZkConnection).
> > log4j:WARN No appenders could be found for logger
> > (org.I0Itec.zkclient.ZkEventThread).
> > log4j:WARN Please initialize the log4j system properly.
> > log4j:WARN Please initialize the log4j system properly.
> > log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig
> > for more info.
> > log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig
> > for more info.
> > mytopic
> >
> > Note, it doesn’t list it as ‘marked for deletion’. Furthermore, waiting
> > multiple hours still doesn’t result in the topic being deleted.
> >
> > The topic has 1 partition, 2 replicas, and no data stored.
> >
> > In zookeeper, the /admin/deleted_topics/ path is empty.
> >
> > The zookeeper code looks pretty straightforward, but for some reason is
> not
> > writing the deleted_topics path.  We are running zookeeper 3.4.6.
> >
> > Thoughts?
> >
> > Jason
> > ​
> >
>


Re: unable to delete topic with 0.8.2 rc2

2015-01-25 Thread Jun Rao
Do you have delete.topic.enable turned on in all brokers?

Thanks,

Jun

On Sun, Jan 25, 2015 at 7:56 PM, Jason Rosenberg  wrote:

> So far, I have been unable to get delete topic to work, with release
> candidate 2 for 0.8.2.
>
> It worked ok when I ran it in the debugger locally, on a single node
> instance. But when I run it in our staging environment, it is not
> successfully even marking the topic for delete, for some reason.
>
> I am setting delete.topic.enable to true (and see the confirmation of this
> in the startup logs, e.g.):
>
> INFO [main] utils.VerifiableProperties - Property delete.topic.enable
> is overridden to true
>
> I run this command:
>
> java -cp app.jar kafka.admin.TopicCommand --zookeeper
> myzkconnect:12345/mynamespace --delete --topic mytopic
>
> log4j:WARN No appenders could be found for logger
> (org.I0Itec.zkclient.ZkConnection).
> log4j:WARN Please initialize the log4j system properly.
> log4j:WARN No appenders could be found for logger
> (org.I0Itec.zkclient.ZkEventThread).
> log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig
> for more info.
> log4j:WARN Please initialize the log4j system properly.
> log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig
> for more info.
> Topic mytopic is marked for deletion.
> Note: This will have no impact if delete.topic.enable is not set to true.
>
> I then do a —list which should at least show the topic marked for deletion:
>
> java -cp app.jar kafka.admin.TopicCommand --zookeeper
> myzkconnect:12345/mynamespace --list --topic mytopic
>
> log4j:WARN No appenders could be found for logger
> (org.I0Itec.zkclient.ZkConnection).
> log4j:WARN No appenders could be found for logger
> (org.I0Itec.zkclient.ZkEventThread).
> log4j:WARN Please initialize the log4j system properly.
> log4j:WARN Please initialize the log4j system properly.
> log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig
> for more info.
> log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig
> for more info.
> mytopic
>
> Note, it doesn’t list it as ‘marked for deletion’. Furthermore, waiting
> multiple hours still doesn’t result in the topic being deleted.
>
> The topic has 1 partition, 2 replicas, and no data stored.
>
> In zookeeper, the /admin/deleted_topics/ path is empty.
>
> The zookeeper code looks pretty straightforward, but for some reason is not
> writing the deleted_topics path.  We are running zookeeper 3.4.6.
>
> Thoughts?
>
> Jason
> ​
>