What is the best practice for doing so?

Thanks
Rob 

On May 16, 2014, at 12:55 PM, Neha Narkhede <neha.narkh...@gmail.com> wrote:

>>> Am I right with the above?
> 
> That is the hacky way of deleting a topic and has not been well tested and
> hence is not certified to work. There are also several race conditions in
> the steps above where the controller can keep overwriting the data deleted
> in zookeeper. So again, delete topic requires first class support which is
> being tested right now. We think it will be available in 0.8.2. Until then,
> your cluster may or may not end up in a bad state if you try deleting in a
> hacky way. But you can try and test that if you want :)
> 
> Thanks,
> Neha
> 
> 
> 
> On Fri, May 16, 2014 at 8:57 AM, Kashyap Mhaisekar <kashya...@gmail.com>wrote:
> 
>> Hi Neha/Jun,
>> A kafa topic deletion involves -
>> 1. Zookeeper clean up (Nodes in zookeeper clusters to be removed - one
>> under brokers/ and other where the offset info is stored)
>> 2. kafka data logs to cleared on each server in cluster for the topic to be
>> deleted
>> 3. partition and replication data to be removed on each server in cluster
>> for the topic to be deleted
>> 
>> So to delete a topic, if we can use kafka-delete-topic.sh and manually
>> ensure the above 3, we should be able to restart the kafka servers and
>> confirm that topic is indeed deleted and can be recreated.
>> 
>> Am I right with the above? Am still on 0.8.0. and trying to understand what
>> is in 0.8.1 that is missing in 0.8.0
>> 
>> Regards,
>> Kashyap
>> 
>> 
>> On Tue, May 13, 2014 at 11:09 AM, Neha Narkhede <neha.narkh...@gmail.com
>>> wrote:
>> 
>>>>> 3. Deleted the topic. Checked only the zookeeper to see if the
>>> /brokers/topics DOES NOT have the topic
>>> 
>>> You are seeing this problem since delete topic is not supported in Kafka.
>>> Any attempt to delete a topic may leave your cluster in an unstable
>> state.
>>> We plan to release 0.8.2 with delete topic support or you could try the
>>> latest trunk.
>>> 
>>> 
>>> On Mon, May 12, 2014 at 9:39 AM, Kashyap Mhaisekar <kashya...@gmail.com
>>>> wrote:
>>> 
>>>> Hi,
>>>> I am hitting a strange exception while creating a topic in Kafka -
>>>> Steps to generate this-
>>>> 1. Created a topic multipartition_test with 2 partitions and 2 replicas
>>>> 2. Added some data to this topics and verified data is coming up for
>> both
>>>> partitions
>>>> 3. Deleted the topic. Checked only the zookeeper to see if the
>>>> /brokers/topics DOES NOT have the topic
>>>> 4. Recreated the topic in exactly the same way as in point 1.
>>>> 
>>>> After this, when I list topics using ./kafka-list-topic.sh, i see that
>>>> *leader:* none and *isr:* for this topic. State change logs give the
>>>> following exception.
>>>> 
>>>> kafka.common.StateChangeFailedException: encountered error while
>> electing
>>>> leader for partition [multipartition_test,1] due to: LeaderAndIsr
>>>> information doesn't exist for partition [multipartition_test,1] in
>>>> OnlinePartition state.
>>>>        at
>> kafka.controller.PartitionStateMachine.electLeaderForPartition(PartitionStateMachine.scala:327)
>>>>        at
>> kafka.controller.PartitionStateMachine.kafka$controller$PartitionStateMachine$$handleStateChange(PartitionStateMachine.scala:154)
>>>>        at
>> kafka.controller.PartitionStateMachine$$anonfun$handleStateChanges$2.apply(PartitionStateMachine.scala:110)
>>>>        at
>> kafka.controller.PartitionStateMachine$$anonfun$handleStateChanges$2.apply(PartitionStateMachine.scala:109)
>>>>        at scala.collection.immutable.Set$Set2.foreach(Set.scala:101)
>>>>        at
>> kafka.controller.PartitionStateMachine.handleStateChanges(PartitionStateMachine.scala:109)
>>>>        at
>> kafka.controller.KafkaController.onNewPartitionCreation(KafkaController.scala:325)
>>>>        at
>> kafka.controller.KafkaController.onNewTopicCreation(KafkaController.scala:312)
>>>>        at
>> kafka.controller.PartitionStateMachine$TopicChangeListener.liftedTree1$1(PartitionStateMachine.scala:376)
>>>>        at
>> kafka.controller.PartitionStateMachine$TopicChangeListener.handleChildChange(PartitionStateMachine.scala:361)
>>>>        at org.I0Itec.zkclient.ZkClient$7.run(ZkClient.java:568)
>>>>        at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
>>>> Caused by: kafka.common.StateChangeFailedException: LeaderAndIsr
>>>> information doesn't exist for partition [multipartition_test,1] in
>>>> OnlinePartition state
>>>>        at
>> kafka.controller.PartitionStateMachine.getLeaderIsrAndEpochOrThrowException(PartitionStateMachine.scala:347)
>>>>        at
>> kafka.controller.PartitionStateMachine.electLeaderForPartition(PartitionStateMachine.scala:291)
>>>>        ... 11 more
>>>> 
>>>> Can you please help what am I doing wrong?
>>>> 
>>>> Regards,
>>>> kashyap
>> 

Reply via email to