Now I used GC like this:

-server -Xms1536m -Xmx1536m -XX:NewSize=128m -XX:MaxNewSize=128m
-XX:+UseConcMarkSweepGC -XX:+UseParNewGC
-XX:CMSInitiatingOccupancyFraction=70


But it still happened.  It seems kafka server reconnect with zk, but the
old node was still there. So kafka server stopped.
Can kafka server retry to connect with zk?


2013-03-27 22:15:03,529] INFO Opening socket connection to server localhost/
127.0.0.1:2181 (org.apache.zookeeper.ClientCnxn)
[2013-03-27 22:15:03,529] INFO Socket connection established to localhost/
127.0.0.1:2181, initiating session (org.apache.zookeeper.ClientCnxn)
[2013-03-27 22:15:05,855] INFO Session establishment complete on server
localhost/127.0.0.1:2181, sessionid = 0x13da6d94abf00aa, negotiated timeout
= 6000 (org.apache.zookeeper.ClientCnxn)
[2013-03-27 22:15:05,942] INFO zookeeper state changed (SyncConnected)
(org.I0Itec.zkclient.ZkClient)
[2013-03-27 22:15:14,912] INFO conflict in /brokers/ids/0 data:
127.0.0.1-1364393691770:127.0.0.1:9093 stored data: null
(kafka.utils.ZkUtils$)
[2013-03-27 22:15:14,942] ERROR Error handling event ZkEvent[New session
event sent to kafka.server.KafkaZooKeeper$SessionExpireListener@18f389bc]
(org.I0Itec.zkclient.ZkEventThread)
java.lang.RuntimeException: A broker is already registered on the path
/brokers/ids/0. 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.server.KafkaZooKeeper.registerBrokerInZk(KafkaZooKeeper.scala:57)
    at
kafka.server.KafkaZooKeeper$SessionExpireListener.handleNewSession(KafkaZooKeeper.scala:100)
    at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472)
    at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
[2013-03-27 22:15:33,736] INFO Closing socket connection to /127.0.0.1.
(kafka.network.Processor)





2013/3/27 Neha Narkhede <neha.narkh...@gmail.com>

> The kafka-server-start.sh script doesn't have the mentioned GC
> settings and heap size configured. However, probably doing that is a
> good idea.
>
> Thanks,
> Neha
>
> On Tue, Mar 26, 2013 at 9:47 AM, Yonghui Zhao <zhaoyong...@gmail.com>
> wrote:
> > kafka server is started by bin/kafka-server-start.sh.  No gc setting.
> > 在 2013-3-26 下午11:40,"Neha Narkhede" <neha.narkh...@gmail.com>写道:
> >
> >> Did you have a gc pause around that time on the server ? What are your
> >> server's current gc settings ?
> >>
> >> Thanks,
> >> Neha
> >>
> >> On Mon, Mar 25, 2013 at 8:48 PM, Yonghui Zhao <zhaoyong...@gmail.com>
> >> wrote:
> >> > Thanks Neha, btw have you seen this exception.  We didn't restart any
> >> > service it happens in deep night.
> >> >
> >> > java.lang.RuntimeException: A broker is already registered on the path
> >> > /brokers/ids/0. 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.server.KafkaZooKeeper.registerBrokerInZk(KafkaZooKeeper.scala:57)
> >> >     at
> >> >
> >>
> kafka.server.KafkaZooKeeper$SessionExpireListener.handleNewSession(KafkaZooKeeper.scala:100)
> >> >     at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472)
> >> >     at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
> >> > [2013-03-26 02:07:19,155] INFO re-registering broker info in ZK for
> >> broker
> >> > 0 (kafka.server.KafkaZooKeeper)
> >> > [2013-03-26 02:07:19,155] INFO Registering broker /brokers/ids/0
> >> > (kafka.server.KafkaZooKeeper)
> >> > [2013-03-26 02:07:19,611] INFO conflict in /brokers/ids/0 data:
> >> > 127.0.0.1-1364234839275:127.0.0.1:9093 stored data:
> >> 127.0.0.1-1364227372971:
> >> > 127.0.0.1:9093 (kafka.utils.ZkUtils$)
> >> > [2013-03-26 02:07:19,611] ERROR Error handling event ZkEvent[New
> session
> >> > event sent to
> kafka.server.KafkaZooKeeper$SessionExpireListener@40f8c9bf
> >> ]
> >> > (org.I0Itec.zkclient.ZkEventThread)
> >> > java.lang.RuntimeException: A broker is already registered on the path
> >> > /brokers/ids/0. 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.server.KafkaZooKeeper.registerBrokerInZk(KafkaZooKeeper.scala:57)
> >> >     at
> >> >
> >>
> kafka.server.KafkaZooKeeper$SessionExpireListener.handleNewSession(KafkaZooKeeper.scala:100)
> >> >     at org.I0Itec.zkclient.ZkClient$4.run(ZkClient.java:472)
> >> >     at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
> >> >
> >> >
> >> >
> >> > 2013/3/26 Neha Narkhede <neha.narkh...@gmail.com>
> >> >
> >> >> That really depends on your consumer application's memory allocation
> >> >> patterns. If it is a thin wrapper over a Kafka consumer, I would
> imagine
> >> >> you can get away with using CMS for the tenured generation and
> parallel
> >> >> collector for the new generation with a small heap like 1gb or so.
> >> >>
> >> >> Thanks,
> >> >> Neha
> >> >>
> >> >> On Monday, March 25, 2013, Yonghui Zhao wrote:
> >> >>
> >> >> > Any suggestion on consumer side?
> >> >> > 在 2013-3-25 下午9:49,"Neha Narkhede" <neha.narkh...@gmail.com
> >> <javascript:;>
> >> >> > >写道:
> >> >> >
> >> >> > > For Kafka 0.7 in production at Linkedin, we use a heap of size
> 3G,
> >> new
> >> >> > gen
> >> >> > > 256 MB, CMS collector with occupancy of 70%.
> >> >> > >
> >> >> > > Thanks,
> >> >> > > Neha
> >> >> > >
> >> >> > > On Sunday, March 24, 2013, Yonghui Zhao wrote:
> >> >> > >
> >> >> > > > Hi Jun,
> >> >> > > >
> >> >> > > > I used kafka-server-start.sh to start kafka, there is only one
> jvm
> >> >> > > setting
> >> >> > > > "-Xmx512M“
> >> >> > > >
> >> >> > > > Do you have some recommend GC setting?   Usually our sever has
> >> 32GB
> >> >> or
> >> >> > > 64GB
> >> >> > > > RAM.
> >> >> > > >
> >> >> > > > 2013/3/22 Jun Rao <jun...@gmail.com>
> >> >> > > >
> >> >> > > > > A typical reason for many rebalancing is the consumer side
> GC.
> >> If
> >> >> so,
> >> >> > > you
> >> >> > > > > will see logs in the consume saying sth like "expired
> session"
> >> for
> >> >> > ZK.
> >> >> > > > > Occasional rebalances are fine. Too many rebalances can slow
> >> down
> >> >> the
> >> >> > > > > consumption and you will need to tune your GC setting.
> >> >> > > > >
> >> >> > > > > Thanks,
> >> >> > > > >
> >> >> > > > > Jun
> >> >> > > > >
> >> >> > > > > On Thu, Mar 21, 2013 at 11:07 PM, Yonghui Zhao <
> >> >> > zhaoyong...@gmail.com
> >> >> > > > > >wrote:
> >> >> > > > >
> >> >> > > > > > Yes, before consumer exception:
> >> >> > > > > >
> >> >> > > > > > 2013/03/21 12:07:17.909 INFO [ZookeeperConsumerConnector]
> []
> >> >> > > > > > 0_lg-mc-db01.bj-1363784482043-f98c7868 *end rebalancing
> >> >> > > > > > consumer*0_lg-mc-db01.bj-1363784482043-f98c7868 try #0
> >> >> > > > > > 2013/03/21 12:07:17.911 INFO [ZookeeperConsumerConnector]
> []
> >> >> > > > > > 0_lg-mc-db01.bj-1363784482043-f98c7868 *begin rebalancing
> >> >> > > > > > consumer*0_lg-mc-db01.bj-1363784482043-f98c7868 try #0
> >> >> > > > > > 2013/03/21 12:07:17.934 INFO [FetcherRunnable] []
> >> FetchRunnable-0
> >> >> > > start
> >> >> > > > > > fetching topic: sms part: 0 offset: 43667888259 from
> >> >> > 127.0.0.1:9093
> >> >> > > > > > 2013/03/21 12:07:17.940 INFO [SimpleConsumer] [] Reconnect
> in
> >> >> > > > multifetch
> >> >> > > > > > due to socket error:
> >> >> > > > > > java.nio.channels.*ClosedByInterruptException*
> >> >> > > > > >         at
> >> java.nio.channels.spi.*AbstractInterruptibleChannel*
> >> >> > > > > > .end(AbstractInterruptibleChannel.java:201)
> >> >> > > > > >
> >> >> > > > > >
> >> >> > > > > > 2013/03/21 12:07:17.978 INFO [ZookeeperConsumerConnector]
> []
> >> >> > > > > > 0_lg-mc-db01.bj-1363784482043-f98c7868 *end rebalancing
> >> >> > > > > > consumer*0_lg-mc-db01.bj-1363784482043-f98c7868 try #0
> >> >> > > > > > 2013/03/21 12:07:18.004 INFO [FetcherRunnable] []
> >> FetchRunnable-0
> >> >> > > start
> >> >> > > > > > fetching topic: sms part: 0 offset: 43667888259 from
> >> >> > 127.0.0.1:9093
> >> >> > > > > > 2013/03/21 12:07:18.066 INFO [ZookeeperConsumerConnector]
> []
> >> >> > > > > > 0_lg-mc-db01.bj-1363784482043-f98c7868 *begin rebalancing
> >> >> consume*r
> >> >> > > > > > 0_lg-mc-db01.bj-1363784482043-f98c7868 try #0
> >> >> > > > > > 2013/03/21 12:07:18.176 INFO [SimpleConsumer] [] Reconnect
> in
> >> >> > > > multifetch
> >> >> > > > > > due to socket error:
> >> >> > > > > > java.nio.channels.*ClosedByInterruptException*
> >> >> > > > > >         at
> >> java.nio.channels.spi.*AbstractInterruptibleChannel*
> >> >> > > > > > .end(AbstractInterruptibleChannel.java:201)
> >> >> > > > > >
> >> >> > > > > >
> >> >> > > > > > So you think it is normal? How can we avoid this exception?
> >> >> > > > > >
> >> >> > > > > > I used 4 partitions in kafka,  use only 1 partition?
> >> >> > > > > >
> >> >> > > > > >
> >> >> > > > > >
> >> >> > > > > > 2013/3/22 Jun Rao <jun...@gmail.com>
> >> >> > > > > >
> >> >> > > > > > > Do you see any rebalances in the consumer? Each rebalance
> >> will
> >> >> > > > > interrupt
> >> >> > > > > > > existing fetcher threads first.
> >> >> > > > > > >
> >> >> > > > > > > Thanks,
> >> >> > > > > > >
> >> >> > > > > > > Jun
> >> >> > > > > > >
> >> >> > > > > > > On Thu, Mar 21, 2013 at 9:40 PM, Yonghui Zhao <
> >> >> > > zhaoyong...@gmail.com
> >> >> >
> >> >>
> >>
>

Reply via email to