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

Joao Santos commented on KAFKA-4834:
------------------------------------

We are experiencing exactly the same issue on 0.11.0.0.

Here is what we do to fix it:
 - Force controller re-election

 - If the topic is gone, cleanup is done. Otherwise, continue to next step.

 - Check that the data directories for the problematic topic do not exist on 
any of the Kafka nodes. For example, for topic "testTopic", and a Kafka data 
base directory of "/opt/kafka/logs", check there are no directories named 
"/opt/kafka/logs/testTopic-*"

 - If directories do exist, stop the node where they do exist, remove the 
directories from the filesystem, restart the node and wait for all topics to be 
in sync again ("ISR" equals "Replicas").  Repeat this step for each node 
containing those directories. It is important to wait for sync, to guarantee 
cluster availability.

 - If the topic is gone, cleanup is done. Otherwise, continue to next step.

 - Login to zookeeper using zookeeper shell

 - Delete the topic from the zookeeper tree by issuing the command "rmr 
/brokers/topics/testTopic"

 - Check that the topic is gone.



To guarantee it's in a good state, you can now create the topic manually on the 
command line and delete it again. It should delete properly this time around.

> Kafka cannot delete topic with ReplicaStateMachine went wrong
> -------------------------------------------------------------
>
>                 Key: KAFKA-4834
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4834
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.10.1.1
>            Reporter: Dan
>              Labels: reliability
>
> It happened several times that some topics can not be deleted in our 
> production environment. By analyzing the log, we found ReplicaStateMachine 
> went wrong. Here are the error messages:
> In state-change.log:
> ERROR Controller 2 epoch 201 initiated state change of replica 1 for 
> partition [test_create_topic1,1] from OnlineReplica to ReplicaDeletionStarted 
> failed (state.change.logger)
> java.lang.AssertionError: assertion failed: Replica 
> [Topic=test_create_topic1,Partition=1,Replica=1] should be in the 
> OfflineReplica states before moving to ReplicaDeletionStarted state. Instead 
> it is in OnlineReplica state
>         at scala.Predef$.assert(Predef.scala:179)
>         at 
> kafka.controller.ReplicaStateMachine.assertValidPreviousStates(ReplicaStateMachine.scala:309)
>         at 
> kafka.controller.ReplicaStateMachine.handleStateChange(ReplicaStateMachine.scala:190)
>         at 
> kafka.controller.ReplicaStateMachine$$anonfun$handleStateChanges$2.apply(ReplicaStateMachine.scala:114)
>         at 
> kafka.controller.ReplicaStateMachine$$anonfun$handleStateChanges$2.apply(ReplicaStateMachine.scala:114)
>         at 
> scala.collection.immutable.HashSet$HashSet1.foreach(HashSet.scala:153)
>         at 
> scala.collection.immutable.HashSet$HashTrieSet.foreach(HashSet.scala:306)
>         at 
> kafka.controller.ReplicaStateMachine.handleStateChanges(ReplicaStateMachine.scala:114)
>         at 
> kafka.controller.TopicDeletionManager$$anonfun$startReplicaDeletion$2.apply(TopicDeletionManager.scala:344)
>         at 
> kafka.controller.TopicDeletionManager$$anonfun$startReplicaDeletion$2.apply(TopicDeletionManager.scala:334)
>         at scala.collection.immutable.Map$Map1.foreach(Map.scala:109)
>         at 
> kafka.controller.TopicDeletionManager.startReplicaDeletion(TopicDeletionManager.scala:334)
>         at 
> kafka.controller.TopicDeletionManager.kafka$controller$TopicDeletionManager$$onPartitionDeletion(TopicDeletionManager.scala:367)
>         at 
> kafka.controller.TopicDeletionManager$$anonfun$kafka$controller$TopicDeletionManager$$onTopicDeletion$2.apply(TopicDeletionManager.scala:313)
>         at 
> kafka.controller.TopicDeletionManager$$anonfun$kafka$controller$TopicDeletionManager$$onTopicDeletion$2.apply(TopicDeletionManager.scala:312)
>         at scala.collection.immutable.Set$Set1.foreach(Set.scala:74)
>         at 
> kafka.controller.TopicDeletionManager.kafka$controller$TopicDeletionManager$$onTopicDeletion(TopicDeletionManager.scala:312)
>         at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1$$anonfun$apply$mcV$sp$4.apply(TopicDeletionManager.scala:431)
>         at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1$$anonfun$apply$mcV$sp$4.apply(TopicDeletionManager.scala:403)
>         at 
> scala.collection.immutable.HashSet$HashSet1.foreach(HashSet.scala:153)
>         at 
> scala.collection.immutable.HashSet$HashTrieSet.foreach(HashSet.scala:306)
>         at 
> scala.collection.immutable.HashSet$HashTrieSet.foreach(HashSet.scala:306)
>         at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1.apply$mcV$sp(TopicDeletionManager.scala:403)
>         at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1.apply(TopicDeletionManager.scala:397)
>         at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1.apply(TopicDeletionManager.scala:397)
>         at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234)
>         at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread.doWork(TopicDeletionManager.scala:397)
>         at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)
> In controller.log:
> INFO Leader not yet assigned for partition [test_create_topic1,1]. Skip 
> sending UpdateMetadataRequest. (kafka.controller.ControllerBrokerRequestBatch)
> There may exist two controllers in the cluster because creating a new topic 
> may trigger two machines to change the state of same partition, eg. 
> NonExistentPartition -> NewPartition.
> On the other controller, we found following messages in controller.log of 
> several days earlier:
> [2017-02-25 16:51:22,353] INFO [Topic Deletion Manager 0], Topic deletion 
> callback for simple_topic_090 (kafka.controller.TopicDeletionManager)
> [2017-02-25 16:51:22,354] ERROR [delete-topics-thread-0], Error due to  
> (kafka.controller.TopicDeletionManager$DeleteTopicsThread)
> java.util.NoSuchElementException: key not found: [test_create_topic_55,2]
>     at scala.collection.MapLike$class.default(MapLike.scala:228)
>     at scala.collection.AbstractMap.default(Map.scala:58)
>     at scala.collection.mutable.HashMap.apply(HashMap.scala:64)
>     at 
> kafka.controller.ControllerBrokerRequestBatch.kafka$controller$ControllerBrokerRequestBatch$$updateMetadataRequestMapFor$1(ControllerChannelManager.scala:310)
>     at 
> kafka.controller.ControllerBrokerRequestBatch$$anonfun$addUpdateMetadataRequestForBrokers$4.apply(ControllerChannelManager.scala:343)
>     at 
> kafka.controller.ControllerBrokerRequestBatch$$anonfun$addUpdateMetadataRequestForBrokers$4.apply(ControllerChannelManager.scala:343)
>     at scala.collection.mutable.HashSet.foreach(HashSet.scala:79)
>     at 
> kafka.controller.ControllerBrokerRequestBatch.addUpdateMetadataRequestForBrokers(ControllerChannelManager.scala:343)
>     at 
> kafka.controller.KafkaController.sendUpdateMetadataRequest(KafkaController.scala:1023)
>     at 
> kafka.controller.TopicDeletionManager.kafka$controller$TopicDeletionManager$$onTopicDeletion(TopicDeletionManager.scala:310)
>     at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1$$anonfun$apply$mcV$sp$4.apply(TopicDeletionManager.scala:431)
>     at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1$$anonfun$apply$mcV$sp$4.apply(TopicDeletionManager.scala:403)
>     at scala.collection.immutable.HashSet$HashSet1.foreach(HashSet.scala:153)
>     at 
> scala.collection.immutable.HashSet$HashTrieSet.foreach(HashSet.scala:306)
>     at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1.apply$mcV$sp(TopicDeletionManager.scala:403)
>     at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1.apply(TopicDeletionManager.scala:397)
>     at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread$$anonfun$doWork$1.apply(TopicDeletionManager.scala:397)
>     at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234)
>     at 
> kafka.controller.TopicDeletionManager$DeleteTopicsThread.doWork(TopicDeletionManager.scala:397)
>     at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)
> [2017-02-25 16:51:22,354] INFO [delete-topics-thread-0], Stopped  
> (kafka.controller.TopicDeletionManager$DeleteTopicsThread)
> [2017-02-25 16:53:58,973] DEBUG [IsrChangeNotificationListener] Fired!!! 
> (kafka.controller.IsrChangeNotificationListener)
> [2017-02-25 16:53:59,014] DEBUG Sending MetadataRequest to 
> Brokers:ArrayBuffer(0, 1, 3, 4) for TopicAndPartitions:Set(…) 
> (kafka.controller.IsrChangeNotificationListener)
> [2017-02-25 16:53:59,015] ERROR [Controller 0]: Forcing the controller to 
> resign (kafka.controller.KafkaController)



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to