kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Marcos Juarez <mjua...@gmail.com>
Subject Re: Kafka 0.8 automatically changing leadership
Date Wed, 06 Aug 2014 16:35:47 GMT
Thank you Joel, will go thorough those docs and make sure our settings are
appropriate on these instances.

Marcos Juarez


On Tue, Aug 5, 2014 at 5:58 PM, Joel Koshy <jjkoshy.w@gmail.com> wrote:

> The session expirations (in the log you pasted) lead to the broker
> losing its registration from zookeeper (which triggers the 'broker
> failure callback' in the controller) - that causes leader election and
> leaders to move. Session expirations are typically due to GC so you
> can take a look at
> https://cwiki.apache.org/confluence/display/KAFKA/Operations to get
> some idea of production settings.
>
> On Tue, Aug 05, 2014 at 04:35:32PM -0600, Marcos Juarez wrote:
> > Joel,
> >
> > Thanks for responding.
> >
> > I see no "Broker failure callback" messages in the logs.   However, I did
> > find this:
> >
> > [2014-08-01 16:47:19,866] INFO Client session timed out, have not heard
> > from server in 4213ms for sessionid 0x143f9e2c9956ee0, closing socket
> > connection and attempting reconnect (org.apache.zookeeper.ClientCnxn)
> > [2014-08-01 16:47:20,531] INFO zookeeper state changed (Disconnected)
> > (org.I0Itec.zkclient.ZkClient)
> > [2014-08-01 16:47:22,305] INFO Opening socket connection to server
> > zookeeper-shared1a.abc.com/10.36.16.157:2181
> > (org.apache.zookeeper.ClientCnxn)
> > [2014-08-01 16:47:22,306] INFO Socket connection established to
> > zookeeper-shared1a.abc.com/10.36.16.157:2181, initiating session
> > (org.apache.zookeeper.ClientCnxn)
> > [2014-08-01 16:47:22,307] INFO zookeeper state changed (Expired)
> > (org.I0Itec.zkclient.ZkClient)
> > [2014-08-01 16:47:22,307] INFO Initiating client connection,
> connectString=
> > zookeeper-shared1a.abc.com:2181/cis/kafka sessionTimeout=6000
> > watcher=org.I0Itec.zkclient.ZkClient@31958905
> > (org.apache.zookeeper.ZooKeeper)
> > [2014-08-01 16:47:22,307] INFO Unable to reconnect to ZooKeeper service,
> > session 0x143f9e2c9956ee0 has expired, closing socket connection
> > (org.apache.zookeeper.ClientCnxn)
> > [2014-08-01 16:47:22,320] INFO Opening socket connection to server
> > zookeeper-shared1a.abc.com/10.36.16.157:2181
> > (org.apache.zookeeper.ClientCnxn)
> > [2014-08-01 16:47:22,322] INFO Socket connection established to
> > zookeeper-shared1a.abc.com/10.36.16.157:2181, initiating session
> > (org.apache.zookeeper.ClientCnxn)
> > [2014-08-01 16:47:22,332] INFO Session establishment complete on server
> > zookeeper-shared1a.abc.com/10.36.16.157:2181, sessionid =
> > 0x143f9e2c9957020, negotiated timeout = 6000
> > (org.apache.zookeeper.ClientCnxn)
> > [2014-08-01 16:47:24,152] INFO re-registering broker info in ZK for
> broker
> > 31268 (kafka.server.KafkaZooKeeper)
> > [2014-08-01 16:47:24,155] INFO Registered broker 31268 at path
> > /brokers/ids/31268 with address kafka-cis2a.abc.com:9092.
> > (kafka.utils.ZkUtils$)
> > [2014-08-01 16:47:24,156] INFO EventThread shut down
> > (org.apache.zookeeper.ClientCnxn)
> > [2014-08-01 16:47:24,156] INFO zookeeper state changed (SyncConnected)
> > (org.I0Itec.zkclient.ZkClient)
> > [2014-08-01 16:47:24,388] INFO done re-registering broker
> > (kafka.server.KafkaZooKeeper)
> > [2014-08-01 16:47:24,389] INFO Subscribing to /brokers/topics path to
> watch
> > for new topics (kafka.server.KafkaZooKeeper)
> > [2014-08-01 16:47:24,391] INFO conflict in /controller data: {
> > "brokerid":31268, "timestamp":"1406911644389", "version":1 } stored
> data: {
> > "brokerid":32391, "timestamp":"1406674545486", "version":1 }
> > (kafka.utils.ZkUtils$)
> > [2014-08-01 16:47:24,469] INFO New leader is 32391
> >
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)25832,ISR:25832,LeaderEpoch:51,ControllerEpoch:330),ReplicationFactor:2),AllReplicas:31268,25832),(enriched_clean,1)
> > ->
> >
> (LeaderAndIsrInfo:(Leader:17977,ISR:17977,32391,LeaderEpoch:30,ControllerEpoch:330),ReplicationFactor:3),AllReplicas:31268,32391,17977)......
> >
> > So, it's complaining about not being able to talk to a zookeeper node, so
> > maybe a brief network partition?  That could be, since these nodes are
> > currently running on a cloud provider, since this a test environment.
> The
> > last line was too large to post here, so I just truncated it.  That line
> >  basically moved leadership on all the topics on that node to the other
> > three Kafka nodes.  At the same time this happened, the other three nodes
> > are reporting "Handling LeaderAndIsr requests".
> >
> > If this was indeed a problem with a brief network partition, should I
> have
> > seen that "broker failure callback" message somewhere in the logs?  And
> > does that mean that Kafka can't withstand network partitions at all, and
> > shouldn't be used on unreliable cloud infrastructure?
> >
> > Thanks for your help.
> >
> > Marcos Juarez
> >
> >
> > On Fri, Aug 1, 2014 at 4:53 PM, Joel Koshy <jjkoshy.w@gmail.com> wrote:
> >
> > > >  Leadership moves automatically for at least a few of the topics,
> which
> > > > never happens when we run them on our prod, non-AWS hardware.  This
> > > causes
> > >
> > > Under normal operation (i.e., without broker failures) leadership
> > > should not move. Leader changes occur when brokers fail - due to GC,
> > > controlled shutdowns/bounces, or network partitioning.
> > >
> > > Do you see any "Broker failure callback" message on the controller
> > > log? Can you also check your broker logs to see if you see any
> > > zookeeper session expirations?
> > >
> > > Joel
> > >
> > > On Fri, Aug 01, 2014 at 04:01:59PM -0600, Marcos Juarez wrote:
> > > > Hi,
> > > >
> > > > We have a Kafka 0.8 cluster in a test environment (in this case, on
> AWS
> > > EC2
> > > > nodes).  Even though we've tried to run very little load on this test
> > > > cluster, it seems like the instances can't even keep up with that.
> > > >  Leadership moves automatically for at least a few of the topics,
> which
> > > > never happens when we run them on our prod, non-AWS hardware.  This
> > > causes
> > > > us to eventually have to rebalance the topics on those test clusters,
> > > which
> > > > is annoying.
> > > >
> > > > Can any of you point me to the set of conditions/thresholds that
> have to
> > > be
> > > > met for the Kafka cluster to decide to automatically move leadership
> of a
> > > > topic/partition to another replica in the ISR?  I'd like to
> understand
> > > how
> > > > exactly Kafka does this, to see if we can provision an instance type
> for
> > > > those test Kakfa clusters that can handle the load without moving
> > > > leadership around.
> > > >
> > > > Thanks,
> > > >
> > > > Marcos Juarez
> > >
> > >
>
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message