You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Shlomi Hazan <sh...@viber.com> on 2014/06/18 13:06:02 UTC

delete topic ?

Hi,

Doing some evaluation testing, and accidently create a queue with wrong
replication factor.

Trying to delete as in:

kafka_2.10-0.8.1.1/bin/kafka-topics.sh --zookeeper localhost:2181 --delete
--topic replicated-topic

Yeilded:

Command must include exactly one action: --list, --describe, --create or
-alter

Event though this page (https://kafka.apache.org/documentation.html) says:

 

And finally deleting a topic:

 > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete --topic
my_topic_name

WARNING: Delete topic functionality is beta in 0.8.1. Please report any bugs
that you encounter on the <ma...@kafka.apache.org> mailing list or
<https://issues.apache.org/jira/browse/KAFKA> JIRA.

Kafka does not currently support reducing the number of partitions for a
topic or changing the replication factor.

What should I do?

Shlomi


Re: delete topic ?

Posted by Timothy Chen <tn...@gmail.com>.
Hi Jason,

You do want to wait for the next release as a lot of stability fixes
are going into that.

Tim

On Thu, Aug 7, 2014 at 10:25 AM, Gwen Shapira <gs...@cloudera.com> wrote:
> Looking at the delete topic patch, it looks like there were
> significant modifications in the controller code to support that, so I
> think you are out of luck.
>
> (https://reviews.apache.org/r/20745)
>
> On Thu, Aug 7, 2014 at 8:18 AM, Jason Rosenberg <jb...@squareup.com> wrote:
>> Since the deletion stuff is now in trunk, would be compatible to issue the
>> command from a jar built from trunk, against a running 0.8.1.1 cluster?  Or
>> does the cluster also have to be running trunk?  (I'm guessing it does :)).
>>
>> I have some topics I'd like to delete, but don't want to wait for 0.8.2
>> (but will probably have to, I'm guessing).
>>
>> Jason
>>
>>
>> On Thu, Aug 7, 2014 at 2:53 AM, Timothy Chen <tn...@gmail.com> wrote:
>>
>>> Hi Gwen,
>>>
>>> That is a very confusing error message for sure, feel free to file a
>>> jira for both the experience cases.
>>>
>>> But in general how delete topic works is that it creates a entry in
>>> the delete_topic zk path, and the leader has a delete topic thread
>>> that watches that path and starts the topic deletion once it receives
>>> the message. It then requires rounds of coordination among all the
>>> brokers that has partitions for the topic to delete all the
>>> partitions, then finally delete the topic from zk.
>>>
>>> Therefore once the deletion finishes it will also deleted from zk. The
>>> topic command can definitely however join the topic list with the
>>> delete topic list and mark the ones being deleted with a special
>>> status.
>>>
>>> Tim
>>>
>>> On Wed, Aug 6, 2014 at 11:20 PM, Gwen Shapira <gs...@cloudera.com>
>>> wrote:
>>> > Hi Timothy,
>>> >
>>> > While we are on the subject, few questions/comments (based on the
>>> > trunk implementation of delete topic command):
>>> >
>>> > * After deleting a topic, I still see it when listing topics. Is the
>>> > expected behavior? Should it disappear after some time?
>>> > * When does the actual deletion gets triggered?
>>> > * If I try to delete a topic twice I get a pretty confusing exception
>>> > (Node exists from zkclient). It will be nice to catch this and say
>>> > "Topic is being deleted" or something to this effect.
>>> > * Even nicer if list topics command will mark topics as "being deleted".
>>> >
>>> > I'll probably open a separate Jira for the "nice" behavior, but
>>> > interested in hearing your thoughts.
>>> >
>>> > Gwen
>>> >
>>> > On Wed, Aug 6, 2014 at 11:01 PM, Timothy Chen <tn...@gmail.com> wrote:
>>> >> Is this the latest master? I've added the delete option in trunk, but
>>> >> it's not in any release yet.
>>> >>
>>> >> We used to have the delete option flag but I believe we removed it
>>> >> that's why the documentation difference.
>>> >>
>>> >> Tim
>>> >>
>>> >> On Wed, Aug 6, 2014 at 10:53 PM, Shlomi Hazan <sh...@viber.com> wrote:
>>> >>> if the answer is pointing out the 'chroot', as a word, it makes no
>>> >>> difference. the result is the same:
>>> >>>
>>> >>> kafka/bin/kafka-topics.sh --zookeeper localhost:2181/chroot --delete
>>> >>> --topic topic-3
>>> >>>
>>> >>> gives the same:
>>> >>>
>>> >>> "Command must include exactly one action: --list, --describe, --create
>>> or
>>> >>> --alter..."
>>> >>>
>>> >>> or should I write something instead of "chroot"?
>>> >>>
>>> >>>
>>> >>>
>>> >>> On Wed, Jun 18, 2014 at 2:06 PM, Shlomi Hazan <sh...@viber.com>
>>> wrote:
>>> >>>
>>> >>>> Hi,
>>> >>>>
>>> >>>> Doing some evaluation testing, and accidently create a queue with
>>> wrong
>>> >>>> replication factor.
>>> >>>>
>>> >>>> Trying to delete as in:
>>> >>>>
>>> >>>> kafka_2.10-0.8.1.1/bin/kafka-topics.sh --zookeeper localhost:2181
>>> --delete
>>> >>>> --topic replicated-topic
>>> >>>>
>>> >>>> Yeilded:
>>> >>>>
>>> >>>> Command must include exactly one action: --list, --describe, --create
>>> or
>>> >>>> –alter
>>> >>>>
>>> >>>> Event though this page (https://kafka.apache.org/documentation.html)
>>> says:
>>> >>>>
>>> >>>>
>>> >>>>
>>> >>>> And finally deleting a topic:
>>> >>>>
>>> >>>>  > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete
>>> --topic my_topic_name
>>> >>>>
>>> >>>> WARNING: Delete topic functionality is beta in 0.8.1. Please report
>>> any
>>> >>>> bugs that you encounter on themailing list <%2...@kafka.apache.org>
>>> or
>>> >>>> JIRA <https://issues.apache.org/jira/browse/KAFKA>.
>>> >>>>
>>> >>>> Kafka does not currently support reducing the number of partitions
>>> for a
>>> >>>> topic or changing the replication factor.
>>> >>>>
>>> >>>> What should I do?
>>> >>>>
>>> >>>> Shlomi
>>> >>>>
>>>

Re: delete topic ?

Posted by Gwen Shapira <gs...@cloudera.com>.
Looking at the delete topic patch, it looks like there were
significant modifications in the controller code to support that, so I
think you are out of luck.

(https://reviews.apache.org/r/20745)

On Thu, Aug 7, 2014 at 8:18 AM, Jason Rosenberg <jb...@squareup.com> wrote:
> Since the deletion stuff is now in trunk, would be compatible to issue the
> command from a jar built from trunk, against a running 0.8.1.1 cluster?  Or
> does the cluster also have to be running trunk?  (I'm guessing it does :)).
>
> I have some topics I'd like to delete, but don't want to wait for 0.8.2
> (but will probably have to, I'm guessing).
>
> Jason
>
>
> On Thu, Aug 7, 2014 at 2:53 AM, Timothy Chen <tn...@gmail.com> wrote:
>
>> Hi Gwen,
>>
>> That is a very confusing error message for sure, feel free to file a
>> jira for both the experience cases.
>>
>> But in general how delete topic works is that it creates a entry in
>> the delete_topic zk path, and the leader has a delete topic thread
>> that watches that path and starts the topic deletion once it receives
>> the message. It then requires rounds of coordination among all the
>> brokers that has partitions for the topic to delete all the
>> partitions, then finally delete the topic from zk.
>>
>> Therefore once the deletion finishes it will also deleted from zk. The
>> topic command can definitely however join the topic list with the
>> delete topic list and mark the ones being deleted with a special
>> status.
>>
>> Tim
>>
>> On Wed, Aug 6, 2014 at 11:20 PM, Gwen Shapira <gs...@cloudera.com>
>> wrote:
>> > Hi Timothy,
>> >
>> > While we are on the subject, few questions/comments (based on the
>> > trunk implementation of delete topic command):
>> >
>> > * After deleting a topic, I still see it when listing topics. Is the
>> > expected behavior? Should it disappear after some time?
>> > * When does the actual deletion gets triggered?
>> > * If I try to delete a topic twice I get a pretty confusing exception
>> > (Node exists from zkclient). It will be nice to catch this and say
>> > "Topic is being deleted" or something to this effect.
>> > * Even nicer if list topics command will mark topics as "being deleted".
>> >
>> > I'll probably open a separate Jira for the "nice" behavior, but
>> > interested in hearing your thoughts.
>> >
>> > Gwen
>> >
>> > On Wed, Aug 6, 2014 at 11:01 PM, Timothy Chen <tn...@gmail.com> wrote:
>> >> Is this the latest master? I've added the delete option in trunk, but
>> >> it's not in any release yet.
>> >>
>> >> We used to have the delete option flag but I believe we removed it
>> >> that's why the documentation difference.
>> >>
>> >> Tim
>> >>
>> >> On Wed, Aug 6, 2014 at 10:53 PM, Shlomi Hazan <sh...@viber.com> wrote:
>> >>> if the answer is pointing out the 'chroot', as a word, it makes no
>> >>> difference. the result is the same:
>> >>>
>> >>> kafka/bin/kafka-topics.sh --zookeeper localhost:2181/chroot --delete
>> >>> --topic topic-3
>> >>>
>> >>> gives the same:
>> >>>
>> >>> "Command must include exactly one action: --list, --describe, --create
>> or
>> >>> --alter..."
>> >>>
>> >>> or should I write something instead of "chroot"?
>> >>>
>> >>>
>> >>>
>> >>> On Wed, Jun 18, 2014 at 2:06 PM, Shlomi Hazan <sh...@viber.com>
>> wrote:
>> >>>
>> >>>> Hi,
>> >>>>
>> >>>> Doing some evaluation testing, and accidently create a queue with
>> wrong
>> >>>> replication factor.
>> >>>>
>> >>>> Trying to delete as in:
>> >>>>
>> >>>> kafka_2.10-0.8.1.1/bin/kafka-topics.sh --zookeeper localhost:2181
>> --delete
>> >>>> --topic replicated-topic
>> >>>>
>> >>>> Yeilded:
>> >>>>
>> >>>> Command must include exactly one action: --list, --describe, --create
>> or
>> >>>> –alter
>> >>>>
>> >>>> Event though this page (https://kafka.apache.org/documentation.html)
>> says:
>> >>>>
>> >>>>
>> >>>>
>> >>>> And finally deleting a topic:
>> >>>>
>> >>>>  > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete
>> --topic my_topic_name
>> >>>>
>> >>>> WARNING: Delete topic functionality is beta in 0.8.1. Please report
>> any
>> >>>> bugs that you encounter on themailing list <%2...@kafka.apache.org>
>> or
>> >>>> JIRA <https://issues.apache.org/jira/browse/KAFKA>.
>> >>>>
>> >>>> Kafka does not currently support reducing the number of partitions
>> for a
>> >>>> topic or changing the replication factor.
>> >>>>
>> >>>> What should I do?
>> >>>>
>> >>>> Shlomi
>> >>>>
>>

Re: delete topic ?

Posted by Jason Rosenberg <jb...@squareup.com>.
Since the deletion stuff is now in trunk, would be compatible to issue the
command from a jar built from trunk, against a running 0.8.1.1 cluster?  Or
does the cluster also have to be running trunk?  (I'm guessing it does :)).

I have some topics I'd like to delete, but don't want to wait for 0.8.2
(but will probably have to, I'm guessing).

Jason


On Thu, Aug 7, 2014 at 2:53 AM, Timothy Chen <tn...@gmail.com> wrote:

> Hi Gwen,
>
> That is a very confusing error message for sure, feel free to file a
> jira for both the experience cases.
>
> But in general how delete topic works is that it creates a entry in
> the delete_topic zk path, and the leader has a delete topic thread
> that watches that path and starts the topic deletion once it receives
> the message. It then requires rounds of coordination among all the
> brokers that has partitions for the topic to delete all the
> partitions, then finally delete the topic from zk.
>
> Therefore once the deletion finishes it will also deleted from zk. The
> topic command can definitely however join the topic list with the
> delete topic list and mark the ones being deleted with a special
> status.
>
> Tim
>
> On Wed, Aug 6, 2014 at 11:20 PM, Gwen Shapira <gs...@cloudera.com>
> wrote:
> > Hi Timothy,
> >
> > While we are on the subject, few questions/comments (based on the
> > trunk implementation of delete topic command):
> >
> > * After deleting a topic, I still see it when listing topics. Is the
> > expected behavior? Should it disappear after some time?
> > * When does the actual deletion gets triggered?
> > * If I try to delete a topic twice I get a pretty confusing exception
> > (Node exists from zkclient). It will be nice to catch this and say
> > "Topic is being deleted" or something to this effect.
> > * Even nicer if list topics command will mark topics as "being deleted".
> >
> > I'll probably open a separate Jira for the "nice" behavior, but
> > interested in hearing your thoughts.
> >
> > Gwen
> >
> > On Wed, Aug 6, 2014 at 11:01 PM, Timothy Chen <tn...@gmail.com> wrote:
> >> Is this the latest master? I've added the delete option in trunk, but
> >> it's not in any release yet.
> >>
> >> We used to have the delete option flag but I believe we removed it
> >> that's why the documentation difference.
> >>
> >> Tim
> >>
> >> On Wed, Aug 6, 2014 at 10:53 PM, Shlomi Hazan <sh...@viber.com> wrote:
> >>> if the answer is pointing out the 'chroot', as a word, it makes no
> >>> difference. the result is the same:
> >>>
> >>> kafka/bin/kafka-topics.sh --zookeeper localhost:2181/chroot --delete
> >>> --topic topic-3
> >>>
> >>> gives the same:
> >>>
> >>> "Command must include exactly one action: --list, --describe, --create
> or
> >>> --alter..."
> >>>
> >>> or should I write something instead of "chroot"?
> >>>
> >>>
> >>>
> >>> On Wed, Jun 18, 2014 at 2:06 PM, Shlomi Hazan <sh...@viber.com>
> wrote:
> >>>
> >>>> Hi,
> >>>>
> >>>> Doing some evaluation testing, and accidently create a queue with
> wrong
> >>>> replication factor.
> >>>>
> >>>> Trying to delete as in:
> >>>>
> >>>> kafka_2.10-0.8.1.1/bin/kafka-topics.sh --zookeeper localhost:2181
> --delete
> >>>> --topic replicated-topic
> >>>>
> >>>> Yeilded:
> >>>>
> >>>> Command must include exactly one action: --list, --describe, --create
> or
> >>>> –alter
> >>>>
> >>>> Event though this page (https://kafka.apache.org/documentation.html)
> says:
> >>>>
> >>>>
> >>>>
> >>>> And finally deleting a topic:
> >>>>
> >>>>  > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete
> --topic my_topic_name
> >>>>
> >>>> WARNING: Delete topic functionality is beta in 0.8.1. Please report
> any
> >>>> bugs that you encounter on themailing list <%2...@kafka.apache.org>
> or
> >>>> JIRA <https://issues.apache.org/jira/browse/KAFKA>.
> >>>>
> >>>> Kafka does not currently support reducing the number of partitions
> for a
> >>>> topic or changing the replication factor.
> >>>>
> >>>> What should I do?
> >>>>
> >>>> Shlomi
> >>>>
>

Re: delete topic ?

Posted by Timothy Chen <tn...@gmail.com>.
Hi Gwen,

That is a very confusing error message for sure, feel free to file a
jira for both the experience cases.

But in general how delete topic works is that it creates a entry in
the delete_topic zk path, and the leader has a delete topic thread
that watches that path and starts the topic deletion once it receives
the message. It then requires rounds of coordination among all the
brokers that has partitions for the topic to delete all the
partitions, then finally delete the topic from zk.

Therefore once the deletion finishes it will also deleted from zk. The
topic command can definitely however join the topic list with the
delete topic list and mark the ones being deleted with a special
status.

Tim

On Wed, Aug 6, 2014 at 11:20 PM, Gwen Shapira <gs...@cloudera.com> wrote:
> Hi Timothy,
>
> While we are on the subject, few questions/comments (based on the
> trunk implementation of delete topic command):
>
> * After deleting a topic, I still see it when listing topics. Is the
> expected behavior? Should it disappear after some time?
> * When does the actual deletion gets triggered?
> * If I try to delete a topic twice I get a pretty confusing exception
> (Node exists from zkclient). It will be nice to catch this and say
> "Topic is being deleted" or something to this effect.
> * Even nicer if list topics command will mark topics as "being deleted".
>
> I'll probably open a separate Jira for the "nice" behavior, but
> interested in hearing your thoughts.
>
> Gwen
>
> On Wed, Aug 6, 2014 at 11:01 PM, Timothy Chen <tn...@gmail.com> wrote:
>> Is this the latest master? I've added the delete option in trunk, but
>> it's not in any release yet.
>>
>> We used to have the delete option flag but I believe we removed it
>> that's why the documentation difference.
>>
>> Tim
>>
>> On Wed, Aug 6, 2014 at 10:53 PM, Shlomi Hazan <sh...@viber.com> wrote:
>>> if the answer is pointing out the 'chroot', as a word, it makes no
>>> difference. the result is the same:
>>>
>>> kafka/bin/kafka-topics.sh --zookeeper localhost:2181/chroot --delete
>>> --topic topic-3
>>>
>>> gives the same:
>>>
>>> "Command must include exactly one action: --list, --describe, --create or
>>> --alter..."
>>>
>>> or should I write something instead of "chroot"?
>>>
>>>
>>>
>>> On Wed, Jun 18, 2014 at 2:06 PM, Shlomi Hazan <sh...@viber.com> wrote:
>>>
>>>> Hi,
>>>>
>>>> Doing some evaluation testing, and accidently create a queue with wrong
>>>> replication factor.
>>>>
>>>> Trying to delete as in:
>>>>
>>>> kafka_2.10-0.8.1.1/bin/kafka-topics.sh --zookeeper localhost:2181 --delete
>>>> --topic replicated-topic
>>>>
>>>> Yeilded:
>>>>
>>>> Command must include exactly one action: --list, --describe, --create or
>>>> –alter
>>>>
>>>> Event though this page (https://kafka.apache.org/documentation.html) says:
>>>>
>>>>
>>>>
>>>> And finally deleting a topic:
>>>>
>>>>  > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete --topic my_topic_name
>>>>
>>>> WARNING: Delete topic functionality is beta in 0.8.1. Please report any
>>>> bugs that you encounter on themailing list <%2...@kafka.apache.org> or
>>>> JIRA <https://issues.apache.org/jira/browse/KAFKA>.
>>>>
>>>> Kafka does not currently support reducing the number of partitions for a
>>>> topic or changing the replication factor.
>>>>
>>>> What should I do?
>>>>
>>>> Shlomi
>>>>

Re: delete topic ?

Posted by Gwen Shapira <gs...@cloudera.com>.
Hi Timothy,

While we are on the subject, few questions/comments (based on the
trunk implementation of delete topic command):

* After deleting a topic, I still see it when listing topics. Is the
expected behavior? Should it disappear after some time?
* When does the actual deletion gets triggered?
* If I try to delete a topic twice I get a pretty confusing exception
(Node exists from zkclient). It will be nice to catch this and say
"Topic is being deleted" or something to this effect.
* Even nicer if list topics command will mark topics as "being deleted".

I'll probably open a separate Jira for the "nice" behavior, but
interested in hearing your thoughts.

Gwen

On Wed, Aug 6, 2014 at 11:01 PM, Timothy Chen <tn...@gmail.com> wrote:
> Is this the latest master? I've added the delete option in trunk, but
> it's not in any release yet.
>
> We used to have the delete option flag but I believe we removed it
> that's why the documentation difference.
>
> Tim
>
> On Wed, Aug 6, 2014 at 10:53 PM, Shlomi Hazan <sh...@viber.com> wrote:
>> if the answer is pointing out the 'chroot', as a word, it makes no
>> difference. the result is the same:
>>
>> kafka/bin/kafka-topics.sh --zookeeper localhost:2181/chroot --delete
>> --topic topic-3
>>
>> gives the same:
>>
>> "Command must include exactly one action: --list, --describe, --create or
>> --alter..."
>>
>> or should I write something instead of "chroot"?
>>
>>
>>
>> On Wed, Jun 18, 2014 at 2:06 PM, Shlomi Hazan <sh...@viber.com> wrote:
>>
>>> Hi,
>>>
>>> Doing some evaluation testing, and accidently create a queue with wrong
>>> replication factor.
>>>
>>> Trying to delete as in:
>>>
>>> kafka_2.10-0.8.1.1/bin/kafka-topics.sh --zookeeper localhost:2181 --delete
>>> --topic replicated-topic
>>>
>>> Yeilded:
>>>
>>> Command must include exactly one action: --list, --describe, --create or
>>> –alter
>>>
>>> Event though this page (https://kafka.apache.org/documentation.html) says:
>>>
>>>
>>>
>>> And finally deleting a topic:
>>>
>>>  > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete --topic my_topic_name
>>>
>>> WARNING: Delete topic functionality is beta in 0.8.1. Please report any
>>> bugs that you encounter on themailing list <%2...@kafka.apache.org> or
>>> JIRA <https://issues.apache.org/jira/browse/KAFKA>.
>>>
>>> Kafka does not currently support reducing the number of partitions for a
>>> topic or changing the replication factor.
>>>
>>> What should I do?
>>>
>>> Shlomi
>>>

Re: delete topic ?

Posted by Timothy Chen <tn...@gmail.com>.
Is this the latest master? I've added the delete option in trunk, but
it's not in any release yet.

We used to have the delete option flag but I believe we removed it
that's why the documentation difference.

Tim

On Wed, Aug 6, 2014 at 10:53 PM, Shlomi Hazan <sh...@viber.com> wrote:
> if the answer is pointing out the 'chroot', as a word, it makes no
> difference. the result is the same:
>
> kafka/bin/kafka-topics.sh --zookeeper localhost:2181/chroot --delete
> --topic topic-3
>
> gives the same:
>
> "Command must include exactly one action: --list, --describe, --create or
> --alter..."
>
> or should I write something instead of "chroot"?
>
>
>
> On Wed, Jun 18, 2014 at 2:06 PM, Shlomi Hazan <sh...@viber.com> wrote:
>
>> Hi,
>>
>> Doing some evaluation testing, and accidently create a queue with wrong
>> replication factor.
>>
>> Trying to delete as in:
>>
>> kafka_2.10-0.8.1.1/bin/kafka-topics.sh --zookeeper localhost:2181 --delete
>> --topic replicated-topic
>>
>> Yeilded:
>>
>> Command must include exactly one action: --list, --describe, --create or
>> –alter
>>
>> Event though this page (https://kafka.apache.org/documentation.html) says:
>>
>>
>>
>> And finally deleting a topic:
>>
>>  > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete --topic my_topic_name
>>
>> WARNING: Delete topic functionality is beta in 0.8.1. Please report any
>> bugs that you encounter on themailing list <%2...@kafka.apache.org> or
>> JIRA <https://issues.apache.org/jira/browse/KAFKA>.
>>
>> Kafka does not currently support reducing the number of partitions for a
>> topic or changing the replication factor.
>>
>> What should I do?
>>
>> Shlomi
>>

Re: delete topic ?

Posted by Shlomi Hazan <sh...@viber.com>.
if the answer is pointing out the 'chroot', as a word, it makes no
difference. the result is the same:

kafka/bin/kafka-topics.sh --zookeeper localhost:2181/chroot --delete
--topic topic-3

gives the same:

"Command must include exactly one action: --list, --describe, --create or
--alter..."

or should I write something instead of "chroot"?



On Wed, Jun 18, 2014 at 2:06 PM, Shlomi Hazan <sh...@viber.com> wrote:

> Hi,
>
> Doing some evaluation testing, and accidently create a queue with wrong
> replication factor.
>
> Trying to delete as in:
>
> kafka_2.10-0.8.1.1/bin/kafka-topics.sh --zookeeper localhost:2181 --delete
> --topic replicated-topic
>
> Yeilded:
>
> Command must include exactly one action: --list, --describe, --create or
> –alter
>
> Event though this page (https://kafka.apache.org/documentation.html) says:
>
>
>
> And finally deleting a topic:
>
>  > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete --topic my_topic_name
>
> WARNING: Delete topic functionality is beta in 0.8.1. Please report any
> bugs that you encounter on themailing list <%2...@kafka.apache.org> or
> JIRA <https://issues.apache.org/jira/browse/KAFKA>.
>
> Kafka does not currently support reducing the number of partitions for a
> topic or changing the replication factor.
>
> What should I do?
>
> Shlomi
>

Re: delete topic ?

Posted by Timothy Chen <tn...@gmail.com>.
Yes the existing delete topic command just cleans up the topic entry in zk, but not really deleting the topic from the cluster.

I have a patch that enables kafka-topics.sh to delete topic but not sure if it's merged to trunk.

Tim

> On Jun 18, 2014, at 1:39 PM, "hsy541@gmail.com" <hs...@gmail.com> wrote:
> 
> I'm using 0.8.1.1
> I use DeleteTopicCommand to delete topic
>    args[0] = "--topic";
>    args[1] = the topic you want to delete
>    args[2] = "--zookeeper";
>    args[3] = kafkaZookeepers;
>    DeleteTopicCommand.main(args);
> 
> You can write your own script to delete the topic, I guess. And I think it
> only deletes the entry in zookeeper
> 
> Best,
> Siyuan
> 
> 
> 
>> On Wed, Jun 18, 2014 at 9:13 AM, Mark Roberts <wi...@gmail.com> wrote:
>> 
>> When we were in testing phase, we would either create a new topic with the
>> correct details or shut the cluster down and hard kill the topic in
>> zookeeper + local disk.  In prod we have the cluster configured via
>> configuration management and auto create turned off.
>> 
>> The ability to delete a topic in a live, running kafka cluster is tricky,
>> and the implementations of it have been subtly incorrect (and therefore
>> dangerous). I know that there is work happening around that, but haven't
>> kept up with the status of it.  Maybe in 8.2? It sounds conceptually
>> simpler to implement with the new metadata API.
>> 
>> -Mark
>> 
>>> On Jun 18, 2014, at 4:06, "Shlomi Hazan" <sh...@viber.com> wrote:
>>> 
>>> Hi,
>>> 
>>> Doing some evaluation testing, and accidently create a queue with wrong
>>> replication factor.
>>> 
>>> Trying to delete as in:
>>> 
>>> kafka_2.10-0.8.1.1/bin/kafka-topics.sh --zookeeper localhost:2181
>> --delete
>>> --topic replicated-topic
>>> 
>>> Yeilded:
>>> 
>>> Command must include exactly one action: --list, --describe, --create or
>>> -alter
>>> 
>>> Event though this page (https://kafka.apache.org/documentation.html)
>> says:
>>> 
>>> 
>>> 
>>> And finally deleting a topic:
>>> 
>>>> bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete --topic
>>> my_topic_name
>>> 
>>> WARNING: Delete topic functionality is beta in 0.8.1. Please report any
>> bugs
>>> that you encounter on the <ma...@kafka.apache.org> mailing
>> list or
>>> <https://issues.apache.org/jira/browse/KAFKA> JIRA.
>>> 
>>> Kafka does not currently support reducing the number of partitions for a
>>> topic or changing the replication factor.
>>> 
>>> What should I do?
>>> 
>>> Shlomi
>> 

Re: delete topic ?

Posted by "hsy541@gmail.com" <hs...@gmail.com>.
I'm using 0.8.1.1
I use DeleteTopicCommand to delete topic
    args[0] = "--topic";
    args[1] = the topic you want to delete
    args[2] = "--zookeeper";
    args[3] = kafkaZookeepers;
    DeleteTopicCommand.main(args);

You can write your own script to delete the topic, I guess. And I think it
only deletes the entry in zookeeper

Best,
Siyuan



On Wed, Jun 18, 2014 at 9:13 AM, Mark Roberts <wi...@gmail.com> wrote:

> When we were in testing phase, we would either create a new topic with the
> correct details or shut the cluster down and hard kill the topic in
> zookeeper + local disk.  In prod we have the cluster configured via
> configuration management and auto create turned off.
>
> The ability to delete a topic in a live, running kafka cluster is tricky,
> and the implementations of it have been subtly incorrect (and therefore
> dangerous). I know that there is work happening around that, but haven't
> kept up with the status of it.  Maybe in 8.2? It sounds conceptually
> simpler to implement with the new metadata API.
>
> -Mark
>
> > On Jun 18, 2014, at 4:06, "Shlomi Hazan" <sh...@viber.com> wrote:
> >
> > Hi,
> >
> > Doing some evaluation testing, and accidently create a queue with wrong
> > replication factor.
> >
> > Trying to delete as in:
> >
> > kafka_2.10-0.8.1.1/bin/kafka-topics.sh --zookeeper localhost:2181
> --delete
> > --topic replicated-topic
> >
> > Yeilded:
> >
> > Command must include exactly one action: --list, --describe, --create or
> > -alter
> >
> > Event though this page (https://kafka.apache.org/documentation.html)
> says:
> >
> >
> >
> > And finally deleting a topic:
> >
> >> bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete --topic
> > my_topic_name
> >
> > WARNING: Delete topic functionality is beta in 0.8.1. Please report any
> bugs
> > that you encounter on the <ma...@kafka.apache.org> mailing
> list or
> > <https://issues.apache.org/jira/browse/KAFKA> JIRA.
> >
> > Kafka does not currently support reducing the number of partitions for a
> > topic or changing the replication factor.
> >
> > What should I do?
> >
> > Shlomi
> >
>

Re: delete topic ?

Posted by Mark Roberts <wi...@gmail.com>.
When we were in testing phase, we would either create a new topic with the correct details or shut the cluster down and hard kill the topic in zookeeper + local disk.  In prod we have the cluster configured via configuration management and auto create turned off.

The ability to delete a topic in a live, running kafka cluster is tricky, and the implementations of it have been subtly incorrect (and therefore dangerous). I know that there is work happening around that, but haven't kept up with the status of it.  Maybe in 8.2? It sounds conceptually simpler to implement with the new metadata API.

-Mark

> On Jun 18, 2014, at 4:06, "Shlomi Hazan" <sh...@viber.com> wrote:
> 
> Hi,
> 
> Doing some evaluation testing, and accidently create a queue with wrong
> replication factor.
> 
> Trying to delete as in:
> 
> kafka_2.10-0.8.1.1/bin/kafka-topics.sh --zookeeper localhost:2181 --delete
> --topic replicated-topic
> 
> Yeilded:
> 
> Command must include exactly one action: --list, --describe, --create or
> -alter
> 
> Event though this page (https://kafka.apache.org/documentation.html) says:
> 
> 
> 
> And finally deleting a topic:
> 
>> bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete --topic
> my_topic_name
> 
> WARNING: Delete topic functionality is beta in 0.8.1. Please report any bugs
> that you encounter on the <ma...@kafka.apache.org> mailing list or
> <https://issues.apache.org/jira/browse/KAFKA> JIRA.
> 
> Kafka does not currently support reducing the number of partitions for a
> topic or changing the replication factor.
> 
> What should I do?
> 
> Shlomi
> 

Re: delete topic ?

Posted by Neha Narkhede <ne...@gmail.com>.
Kafka allows increasing the replication factor of a topic. You can read
about it here
<http://kafka.apache.org/081/documentation.html#basic_ops_increase_replication_factor>.
We do not support reducing the number of partitions, so you either have to
create a new topic or delete the existing one. We fixed a number of bugs in
delete topic but those fixes are on trunk. You can give it a spin.


On Wed, Jun 18, 2014 at 4:06 AM, Shlomi Hazan <sh...@viber.com> wrote:

> Hi,
>
> Doing some evaluation testing, and accidently create a queue with wrong
> replication factor.
>
> Trying to delete as in:
>
> kafka_2.10-0.8.1.1/bin/kafka-topics.sh --zookeeper localhost:2181 --delete
> --topic replicated-topic
>
> Yeilded:
>
> Command must include exactly one action: --list, --describe, --create or
> -alter
>
> Event though this page (https://kafka.apache.org/documentation.html) says:
>
>
>
> And finally deleting a topic:
>
>  > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete --topic
> my_topic_name
>
> WARNING: Delete topic functionality is beta in 0.8.1. Please report any
> bugs
> that you encounter on the <ma...@kafka.apache.org> mailing list
> or
> <https://issues.apache.org/jira/browse/KAFKA> JIRA.
>
> Kafka does not currently support reducing the number of partitions for a
> topic or changing the replication factor.
>
> What should I do?
>
> Shlomi
>
>