[ https://issues.apache.org/jira/browse/KAFKA-1019?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14144507#comment-14144507 ]
hongyu bi edited comment on KAFKA-1019 at 9/23/14 9:10 AM: ----------------------------------------------------------- Thanks @Guozhang. Yes, followed the same process to re-produce this issue. Sorry I don't quit understand why KAFKA-1558 lead to this issue? >From the method below I don't see any process on topic delete: PartitionStateMachine$TopicChangeListener handleChildChange: val newTopics = currentChildren -- controllerContext.allTopics val deletedTopics = controllerContext.allTopics -- currentChildren controllerContext.allTopics = currentChildren val addedPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, newTopics.toSeq) controllerContext.partitionReplicaAssignment = controllerContext.partitionReplicaAssignment.filter(p => !deletedTopics.contains(p._1.topic)) controllerContext.partitionReplicaAssignment.++=(addedPartitionReplicaAssignment) if(newTopics.size > 0) controller.onNewTopicCreation(newTopics, addedPartitionReplicaAssignment.keySet.toSet) plus1: From the stat-change.log: when I create the same topic after deleting it , PartitionStateMachine report: ERROR Controller 20225 epoch 1 initiated state change for partition [hobi1,1] from OnlinePartition to NewPartition failed (state.change.logger) java.lang.IllegalStateException: Partition [hobi1,1] should be in the NonExistentPartition states before moving to NewPartition state. Instead it is in OnlinePartition state at kafka.controller.PartitionStateMachine.assertValidPreviousStates(PartitionStateMachine.scala:243) It seems when deleting topic , kafka doesn't sync some internal data structure? plus2: >From TopicDeleteManager , it seems deleting topic doesn't enter this block: if(controller.replicaStateMachine.areAllReplicasForTopicDeleted(topic)) { // clear up all state for this topic from controller cache and zookeeper completeDeleteTopic(topic) info("Deletion of topic %s successfully completed".format(topic)) } was (Author: hongyu.bi): Thanks @Guozhang. Yes, followed the same process to re-produce this issue. Sorry I don't quit understand why KAFKA-1558 lead to this issue? >From the method below I don't see any process on topic delete: PartitionStateMachine$TopicChangeListener handleChildChange: val newTopics = currentChildren -- controllerContext.allTopics val deletedTopics = controllerContext.allTopics -- currentChildren controllerContext.allTopics = currentChildren val addedPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, newTopics.toSeq) controllerContext.partitionReplicaAssignment = controllerContext.partitionReplicaAssignment.filter(p => !deletedTopics.contains(p._1.topic)) controllerContext.partitionReplicaAssignment.++=(addedPartitionReplicaAssignment) if(newTopics.size > 0) controller.onNewTopicCreation(newTopics, addedPartitionReplicaAssignment.keySet.toSet) plus: From the stat-change.log: when I create the same topic after deleting it , PartitionStateMachine report: ERROR Controller 20225 epoch 1 initiated state change for partition [hobi1,1] from OnlinePartition to NewPartition failed (state.change.logger) java.lang.IllegalStateException: Partition [hobi1,1] should be in the NonExistentPartition states before moving to NewPartition state. Instead it is in OnlinePartition state at kafka.controller.PartitionStateMachine.assertValidPreviousStates(PartitionStateMachine.scala:243) It seems when deleting topic , kafka doesn't sync some internal data structure? > kafka-preferred-replica-election.sh will fail without clear error message if > /brokers/topics/[topic]/partitions does not exist > ------------------------------------------------------------------------------------------------------------------------------ > > Key: KAFKA-1019 > URL: https://issues.apache.org/jira/browse/KAFKA-1019 > Project: Kafka > Issue Type: Bug > Affects Versions: 0.8.1 > Reporter: Guozhang Wang > Labels: newbie > Fix For: 0.9.0 > > > From Libo Yu: > I tried to run kafka-preferred-replica-election.sh on our kafka cluster. > But I got this expection: > Failed to start preferred replica election > org.I0Itec.zkclient.exception.ZkNoNodeException: > org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = > NoNode for /brokers/topics/uattoqaaa.default/partitions > I checked zookeeper and there is no > /brokers/topics/uattoqaaa.default/partitions. All I found is > /brokers/topics/uattoqaaa.default. -- This message was sent by Atlassian JIRA (v6.3.4#6332)