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

Joe Stein edited comment on KAFKA-1387 at 8/5/14 4:41 AM:
----------------------------------------------------------

Here is another way to reproduce this issue.  I have seen it a few times now 
with folks getting going with their clusters.

steps to reproduce.  install a 3 node zk ensemble with 3 brokers cluster

e.g. 

git clone https://github.com/stealthly/scala-kafka
git checkout -b zkbk3 origin/zkbk3
vagrant up provider=virtualbox

now setup each node in the cluster as you would broker 1,2,3 and the ensemble

e.g.

vagrant ssh zkbkOne
sudo su
cd /vagrant/vagrant/ && ./up.sh
vagrant ssh zkbkTwo
sudo su
cd /vagrant/vagrant/ && ./up.sh
vagrant ssh zkbkThree
sudo su
cd /vagrant/vagrant/ && ./up.sh

start up zookeeper on all 3 nodes
cd /opt/apache/kafka && bin/zookeeper-server-start.sh 
config/zookeeper.properties 1>>/tmp/zk.log 2>>/tmp/zk.log &

now, start up broker on node 2 only
cd /opt/apache/kafka && bin/kafka-server-start.sh config/server.properties 
1>>/tmp/bk.log 2>>/tmp/bk.log &

ok, now here is where it gets wonky

on server 3 change from broker.id=3 to broker.id=2 
now you need to start up server 1 and 3 (even though it is broker.id=2) at the 
same time

cd /opt/apache/kafka && bin/kafka-server-start.sh config/server.properties 
1>>/tmp/bk.log 2>>/tmp/bk.log &
cd /opt/apache/kafka && bin/kafka-server-start.sh config/server.properties 
1>>/tmp/bk.log 2>>/tmp/bk.log &
( you can have two tabs, hit enter in one switch to other tab and hit enter is 
close enough to same time)

and you get this looping forever

2014-08-05 04:34:38,591] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":2,"timestamp":"1407212148186"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
[2014-08-05 04:34:44,598] INFO conflict in /controller data: 
{"version":1,"brokerid":2,"timestamp":"1407212148186"} stored data: 
{"version":1,"brokerid":2,"timestamp":"1407211911014"} (kafka.utils.ZkUtils$)
[2014-08-05 04:34:44,601] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":2,"timestamp":"1407212148186"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
[2014-08-05 04:34:50,610] INFO conflict in /controller data: 
{"version":1,"brokerid":2,"timestamp":"1407212148186"} stored data: 
{"version":1,"brokerid":2,"timestamp":"1407211911014"} (kafka.utils.ZkUtils$)
[2014-08-05 04:34:50,614] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":2,"timestamp":"1407212148186"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
[2014-08-05 04:34:56,621] INFO conflict in /controller data: 
{"version":1,"brokerid":2,"timestamp":"1407212148186"} stored data: 
{"version":1,"brokerid":2,"timestamp":"1407211911014"} (kafka.utils.ZkUtils$)

the expected result that you get should be

[2014-08-05 04:07:20,917] INFO conflict in /brokers/ids/2 data: 
{"jmx_port":-1,"timestamp":"1407211640900","host":"192.168.30.3","version":1,"port":9092}
 stored data: {"jmx_port":-1,"timestamp":"140721119
9464","host":"192.168.30.2","version":1,"port":9092} (kafka.utils.ZkUtils$)
[2014-08-05 04:07:20,949] FATAL Fatal error during KafkaServerStable startup. 
Prepare to shutdown (kafka.server.KafkaServerStartable)
java.lang.RuntimeException: A broker is already registered on the path 
/brokers/ids/2. This probably indicates that you either have configured a 
brokerid that is already in use, or else you have shutdown 
this broker and restarted it faster than the zookeeper timeout so it appears to 
be re-registering.
        at kafka.utils.ZkUtils$.registerBrokerInZk(ZkUtils.scala:205)
        at kafka.server.KafkaHealthcheck.register(KafkaHealthcheck.scala:57)
        at kafka.server.KafkaHealthcheck.startup(KafkaHealthcheck.scala:44)
        at kafka.server.KafkaServer.startup(KafkaServer.scala:103)
        at 
kafka.server.KafkaServerStartable.startup(KafkaServerStartable.scala:34)
        at kafka.Kafka$.main(Kafka.scala:46)
        at kafka.Kafka.main(Kafka.scala)
[2014-08-05 04:07:20,952] INFO [Kafka Server 2], shutting down 
(kafka.server.KafkaServer)
[2014-08-05 04:07:20,954] INFO [Socket Server on Broker 2], Shutting down 
(kafka.network.SocketServer)
[2014-08-05 04:07:20,959] INFO [Socket Server on Broker 2], Shutdown completed 
(kafka.network.SocketServer)
[2014-08-05 04:07:20,960] INFO [Kafka Request Handler on Broker 2], shutting 
down (kafka.server.KafkaRequestHandlerPool)
[2014-08-05 04:07:20,992] INFO [Kafka Request Handler on Broker 2], shut down 
completely (kafka.server.KafkaRequestHandlerPool)
[2014-08-05 04:07:21,263] INFO [Replica Manager on Broker 2]: Shut down 
(kafka.server.ReplicaManager)
[2014-08-05 04:07:21,263] INFO [ReplicaFetcherManager on broker 2] shutting 
down (kafka.server.ReplicaFetcherManager)

which is what you get if you just start server 3 on its one configured with 
broker.id=2

I originally saw this on a 12 node AWS cluster same kafka 0.8.1.1 zk 3.3.4

I don't know how very critical this is, someone just brought up something 
similar but with /brokers/ids/x I don't know if they are related but something 
wonky is going on with the ZkUtils.createEphemeralPathExpectConflictHandleZKBug 
code paths.





was (Author: joestein):
Here is another way to reproduce this issue.  I have seen it a few times now 
with folks getting going with their clusters.

steps to reproduce.  install a 3 node zk ensemble with 3 brokers cluster

e.g. 

git clone https://github.com/stealthly/scala-kafka
git checkout -b zkbk3 origin/zkbk3
vagrant up provider=virtualbox

now setup each node in the cluster as you would broker 1,2,3 and the ensemble

e.g.

vagrant ssh zkbkOne
sudo su
cd /vagrant/vagrant/ && ./up.sh
vagrant ssh zkbkTwo
sudo su
cd /vagrant/vagrant/ && ./up.sh
vagrant ssh zkbkThree
sudo su
cd /vagrant/vagrant/ && ./up.sh

start up zookeeper on all 3 nodes
cd /opt/apache/kafka && bin/zookeeper-server-start.sh 
config/zookeeper.properties 1>>/tmp/zk.log 2>>/tmp/zk.log &

now, start up broker on node 2 only
cd /opt/apache/kafka && bin/kafka-server-start.sh config/server.properties 
1>>/tmp/bk.log 2>>/tmp/bk.log &

ok, now here is where it gets wonky

- change the broker.id int server 3 to = 2 
now you need to start up server 1 and 3 (even though it is 2) at the same time

cd /opt/apache/kafka && bin/kafka-server-start.sh config/server.properties 
1>>/tmp/bk.log 2>>/tmp/bk.log &
cd /opt/apache/kafka && bin/kafka-server-start.sh config/server.properties 
1>>/tmp/bk.log 2>>/tmp/bk.log &
( you can have two tabs, hit enter in one switch to other tab and hit enter is 
close enough to same time)

and you get this looping forever

2014-08-05 04:34:38,591] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":2,"timestamp":"1407212148186"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
[2014-08-05 04:34:44,598] INFO conflict in /controller data: 
{"version":1,"brokerid":2,"timestamp":"1407212148186"} stored data: 
{"version":1,"brokerid":2,"timestamp":"1407211911014"} (kafka.utils.ZkUtils$)
[2014-08-05 04:34:44,601] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":2,"timestamp":"1407212148186"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
[2014-08-05 04:34:50,610] INFO conflict in /controller data: 
{"version":1,"brokerid":2,"timestamp":"1407212148186"} stored data: 
{"version":1,"brokerid":2,"timestamp":"1407211911014"} (kafka.utils.ZkUtils$)
[2014-08-05 04:34:50,614] INFO I wrote this conflicted ephemeral node 
[{"version":1,"brokerid":2,"timestamp":"1407212148186"}] at /controller a while 
back in a different session, hence I will backoff for this node to be deleted 
by Zookeeper and retry (kafka.utils.ZkUtils$)
[2014-08-05 04:34:56,621] INFO conflict in /controller data: 
{"version":1,"brokerid":2,"timestamp":"1407212148186"} stored data: 
{"version":1,"brokerid":2,"timestamp":"1407211911014"} (kafka.utils.ZkUtils$)

the expected result that you get should be

[2014-08-05 04:07:20,917] INFO conflict in /brokers/ids/2 data: 
{"jmx_port":-1,"timestamp":"1407211640900","host":"192.168.30.3","version":1,"port":9092}
 stored data: {"jmx_port":-1,"timestamp":"140721119
9464","host":"192.168.30.2","version":1,"port":9092} (kafka.utils.ZkUtils$)
[2014-08-05 04:07:20,949] FATAL Fatal error during KafkaServerStable startup. 
Prepare to shutdown (kafka.server.KafkaServerStartable)
java.lang.RuntimeException: A broker is already registered on the path 
/brokers/ids/2. This probably indicates that you either have configured a 
brokerid that is already in use, or else you have shutdown 
this broker and restarted it faster than the zookeeper timeout so it appears to 
be re-registering.
        at kafka.utils.ZkUtils$.registerBrokerInZk(ZkUtils.scala:205)
        at kafka.server.KafkaHealthcheck.register(KafkaHealthcheck.scala:57)
        at kafka.server.KafkaHealthcheck.startup(KafkaHealthcheck.scala:44)
        at kafka.server.KafkaServer.startup(KafkaServer.scala:103)
        at 
kafka.server.KafkaServerStartable.startup(KafkaServerStartable.scala:34)
        at kafka.Kafka$.main(Kafka.scala:46)
        at kafka.Kafka.main(Kafka.scala)
[2014-08-05 04:07:20,952] INFO [Kafka Server 2], shutting down 
(kafka.server.KafkaServer)
[2014-08-05 04:07:20,954] INFO [Socket Server on Broker 2], Shutting down 
(kafka.network.SocketServer)
[2014-08-05 04:07:20,959] INFO [Socket Server on Broker 2], Shutdown completed 
(kafka.network.SocketServer)
[2014-08-05 04:07:20,960] INFO [Kafka Request Handler on Broker 2], shutting 
down (kafka.server.KafkaRequestHandlerPool)
[2014-08-05 04:07:20,992] INFO [Kafka Request Handler on Broker 2], shut down 
completely (kafka.server.KafkaRequestHandlerPool)
[2014-08-05 04:07:21,263] INFO [Replica Manager on Broker 2]: Shut down 
(kafka.server.ReplicaManager)
[2014-08-05 04:07:21,263] INFO [ReplicaFetcherManager on broker 2] shutting 
down (kafka.server.ReplicaFetcherManager)

which is what you get if you just start server 3 on its one configured with 
broker.id=2

I originally saw this on a 12 node AWS cluster same kafka 0.8.1.1 zk 3.3.4

I don't know how very critical this is, someone just brought up something 
similar but with /brokers/ids/x I don't know if they are related but something 
wonky is going on with the ZkUtils.createEphemeralPathExpectConflictHandleZKBug 
code paths.




> Kafka getting stuck creating ephemeral node it has already created when two 
> zookeeper sessions are established in a very short period of time
> ---------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-1387
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1387
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Fedor Korotkiy
>
> Kafka broker re-registers itself in zookeeper every time handleNewSession() 
> callback is invoked.
> https://github.com/apache/kafka/blob/0.8.1/core/src/main/scala/kafka/server/KafkaHealthcheck.scala
>  
> Now imagine the following sequence of events.
> 1) Zookeeper session reestablishes. handleNewSession() callback is queued by 
> the zkClient, but not invoked yet.
> 2) Zookeeper session reestablishes again, queueing callback second time.
> 3) First callback is invoked, creating /broker/[id] ephemeral path.
> 4) Second callback is invoked and it tries to create /broker/[id] path using 
> createEphemeralPathExpectConflictHandleZKBug() function. But the path is 
> already exists, so createEphemeralPathExpectConflictHandleZKBug() is getting 
> stuck in the infinite loop.
> Seems like controller election code have the same issue.
> I'am able to reproduce this issue on the 0.8.1 branch from github using the 
> following configs.
> # zookeeper
> tickTime=10
> dataDir=/tmp/zk/
> clientPort=2101
> maxClientCnxns=0
> # kafka
> broker.id=1
> log.dir=/tmp/kafka
> zookeeper.connect=localhost:2101
> zookeeper.connection.timeout.ms=100
> zookeeper.sessiontimeout.ms=100
> Just start kafka and zookeeper and then pause zookeeper several times using 
> Ctrl-Z.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to