[
https://issues.apache.org/jira/browse/KAFKA-1310?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13942939#comment-13942939
]
Michael Noll commented on KAFKA-1310:
-------------------------------------
I can confirm this issue, using Kafka 0.8.1.
Here are the error messages when trying to create a topic:
{code}
$ bin/kafka-topics.sh --create --zookeeper zookeeper1:2181 --topic testing
--partitions 1 --replication-factor 1
Error while executing topic command
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode
for /brokers/ids
org.I0Itec.zkclient.exception.ZkNoNodeException:
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode
for /brokers/ids
at org.I0Itec.zkclient.exception.ZkException.create(ZkException.java:47)
at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:685)
at org.I0Itec.zkclient.ZkClient.getChildren(ZkClient.java:413)
at org.I0Itec.zkclient.ZkClient.getChildren(ZkClient.java:409)
at kafka.utils.ZkUtils$.getChildren(ZkUtils.scala:480)
at kafka.utils.ZkUtils$.getSortedBrokerList(ZkUtils.scala:81)
at kafka.admin.AdminUtils$.createTopic(AdminUtils.scala:154)
at kafka.admin.TopicCommand$.createTopic(TopicCommand.scala:88)
at kafka.admin.TopicCommand$.main(TopicCommand.scala:50)
at kafka.admin.TopicCommand.main(TopicCommand.scala)
Caused by: org.apache.zookeeper.KeeperException$NoNodeException:
KeeperErrorCode = NoNode for /brokers/ids
at org.apache.zookeeper.KeeperException.create(KeeperException.java:102)
at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1249)
at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:1277)
at org.I0Itec.zkclient.ZkConnection.getChildren(ZkConnection.java:99)
at org.I0Itec.zkclient.ZkClient$2.call(ZkClient.java:416)
at org.I0Itec.zkclient.ZkClient$2.call(ZkClient.java:413)
at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
... 8 more
{code}
If you use the ZK CLI you will sometimes see a znode under {{/brokers/ids}},
sometimes not. In my limited testing I could, for instance, create a topic
(partitions=1, replicas=1) and then list/describe it. But at least when I
reached the point to try sending messages to it, it would fail. See next
example.
When trying to use the console producer to sent a test message "foo" (String)
to the topic/broker:
{code}
$ bin/kafka-console-producer.sh --topic testing --broker-list localhost:9092
foo <<< This is the test message, manually entered in the console/terminal
[2014-03-20 09:45:32,223] WARN Error while fetching metadata [{TopicMetadata
for topic testing ->
No partition metadata for topic testing due to
kafka.common.LeaderNotAvailableException}] for topic [testing]: class
kafka.common.LeaderNotAvailableException (kafka.producer.BrokerPartitionInfo)
[2014-03-20 09:45:32,233] WARN Error while fetching metadata [{TopicMetadata
for topic testing ->
No partition metadata for topic testing due to
kafka.common.LeaderNotAvailableException}] for topic [testing]: class
kafka.common.LeaderNotAvailableException (kafka.producer.BrokerPartitionInfo)
[2014-03-20 09:45:32,234] ERROR Failed to collate messages by topic, partition
due to: Failed to fetch topic metadata for topic: testing
(kafka.producer.async.DefaultEventHandler)
{code}
*How to reproduce*
Using Wirbelsturm you can reproduce this error as follow. This assumes you
have Vagrant 1.4.x and VirtualBox already installed on your host machine.
{code}
$ git clone https://github.com/miguno/wirbelsturm.git
$ cd wirbelsturm
$ ./bootstrap # <<< May take a while depending on how fast your Internet
connection is.
# Then uncomment the `kafka_broker` section in `wirbelsturm.yaml`.
# Only remove the leading `#` character in each line -- the remaining leading
whitespace is significant.
$ vagrant up zookeeper1 kafka1 # <<< May take a while (boots VMs, downloads
RPMs from the Internet to provision the VMs, etc.)
{code}
Now you can ssh into the VM {{kafka1}} via {{vagrant ssh kafka1}} and run the
commands above from within the {{/opt/kafka}} directory.
> Zookeeper timeout causes deadlock in Controller
> -----------------------------------------------
>
> Key: KAFKA-1310
> URL: https://issues.apache.org/jira/browse/KAFKA-1310
> Project: Kafka
> Issue Type: Bug
> Affects Versions: 0.8.1
> Reporter: Fedor Korotkiy
> Assignee: Neha Narkhede
> Priority: Blocker
>
> Steps to reproduce:
> 1. Checkout and build 0.8.1 branch from github:
> git clone [email protected]:apache/kafka.git && cd kafka && git checkout
> origin/0.8.1 && ./gradlew jar
> 2. Start zookeeper server:
> ./bin/zookeeper-server-start.sh config/zookeeper.properties
> 3. Start kafka server:
> ./bin/kafka-server-start.sh config/server.properties
> 4. Suspend zookeeper process for 10 seconds (ctrl-Z, then %1).
> 5. And kafka hasn't been re-registered in zookeeper.
> ./bin/zookeeper-shell.sh
> ls /brokers/ids
> >> []
> Root cause of the problem seems to be the deadlock between DeleteTopicsThread
> and SessionExpirationListener in KafkaController.
> 1. DeleteTopicsThread acquires controllerLock and await()-s on
> deleteTopicsCond in awaitTopicDeletionNotification()
> 2. SessionExpirationListener fires. It acquires controllerLock and tries to
> shutdown deleteTopicManager(in onControllerResignation()). This interrupts
> DeleteTopicsThread.
> 3. DeleteTopicsThread can't return from deleteTopicsCond.await() because
> controllerLock is taken. We got a deadlock.
--
This message was sent by Atlassian JIRA
(v6.2#6252)