[
https://issues.apache.org/jira/browse/KAFKA-1019?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14084641#comment-14084641
]
Mickael Hemri commented on KAFKA-1019:
--
I tried with zookeeper 3.3.6 and we have the same issue.
To reproduce:
Create a topic named testid
{code}bin/kafka-topics.sh --topic testid --replication-factor 3 --partition 3
--zookeeper 127.0.0.1:2181/kafka --create
Created topic "testid".{code}
{code}./bin/kafka-topics.sh --topic testid --zookeeper 127.0.0.1:2181/kafka
--describe
Topic:testidPartitionCount:3ReplicationFactor:3 Configs:
Topic: testid Partition: 0Leader: 31985 Replicas:
31985,9920,4580 Isr: 31985,9920,4580
Topic: testid Partition: 1Leader: 4580Replicas:
4580,31985,9920 Isr: 4580,31985,9920
Topic: testid Partition: 2Leader: 9920Replicas:
9920,4580,31985 Isr: 9920,4580,31985
{code}
Ok great, we have leaders and /brokers/topics/testid/partitions in zookeeper
Delete testid topic
{code}bin/kafka-run-class.sh kafka.admin.DeleteTopicCommand --topic testid
--zookeeper 127.0.0.1:2181/kafka
deletion succeeded!
{code}
Create again a topic named testid
{code}bin/kafka-topics.sh --topic testid --replication-factor 3 --partition 3
--zookeeper 127.0.0.1:2181/kafka --create
Created topic "testid".{code}
Now check:
{code}./bin/kafka-topics.sh --topic testid --zookeeper 127.0.0.1:2181/kafka
--describe
Topic:testidPartitionCount:3ReplicationFactor:3 Configs:
Topic: testid Partition: 0Leader: noneReplicas:
31985,4580,9920 Isr:
Topic: testid Partition: 1Leader: noneReplicas:
4580,9920,31985 Isr:
Topic: testid Partition: 2Leader: noneReplicas:
9920,31985,4580 Isr:{code}
As you can see we have no leader when we create the topic after a deletion. And
there is no /brokers/topics/testid/partitions in zookeeper
It works again with a different topic name, so it seems that something is not
properly deleted with DeleteTopicCommand command.
We reproduced it on 3 differents zookeeper chroot: 127.0.0.1:2181/kafka,
127.0.0.1:2181/kafka2 and 127.0.0.1:2181/kafka3
Thanks
> 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.8.2
>
>
> 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.2#6252)