You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Andrew Otto <ot...@wikimedia.org> on 2014/03/20 23:26:35 UTC

Zookeeper reconnect failed due to 'state changed (Expired)'

Hm, just saw something a little fishy.  

(In the following logs, analytics1021 (id 21) and analytics1022 (id 22) are Brokers and analytics1023,analytics1024,analytics1025 are Zookeepers.)

At 2014-03-20 21:12:26, analytics1021 lost its connection to zookeeper. It reconnected to analytics1023, but as it was doing so, it logged a 'zookeeper state changed (Expired)’ message, which apparently caused the newly opened socket to close again.  Am I reading that right?

In the time that it took analytics1021 to finally re-establish a zookeeper connection, analytics1022 had become the leader for all partitions.  I can rebalance the leaders, but I’d prefer if this didn’t happen in the first place.

1.  What does 'zookeeper state changed (Expired)’ mean?
2.  Has anyone seen issues like this before?  Where zookeeper connections are flaky enough to cause leader elections?

Thanks!
-Andrew Otto



[2014-03-20 21:12:26,427] 685698013 [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO org.apache.zookeeper.ClientCnxn - Client session timed out, have not heard from server in 10067ms for sessionid 0x1425d903a3f1f89, closing socket connection and attempting reconnect
[2014-03-20 21:12:26,528] 685698114 [main-EventThread] INFO org.I0Itec.zkclient.ZkClient - zookeeper state changed (Disconnected)
[2014-03-20 21:12:27,252] 685698838 [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO org.apache.zookeeper.ClientCnxn - Opening socket connection to server analytics1023.eqiad.wmnet/10.64.5.15:2181
[2014-03-20 21:12:27,271] 685698857 [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO org.apache.zookeeper.ClientCnxn - Socket connection established to analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
[2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO org.I0Itec.zkclient.ZkClient - zookeeper state changed (Expired)
[2014-03-20 21:12:27,273] 685698859 [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO org.apache.zookeeper.ClientCnxn - Unable to reconnect to ZooKeeper service, session 0x1425d903a3f1f89 has expired, closing socket connection
[2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO org.apache.zookeeper.ZooKeeper - Initiating client connection, connectString=analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad sessionTimeout=6000 watcher=org.I0Itec.zkclient.ZkClient@63203b59
[2014-03-20 21:12:27,290] 685698876 [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad] INFO kafka.server.KafkaZooKeeper - re-registering broker info in ZK for broker 21
[2014-03-20 21:12:27,290] 685698876 [main-EventThread] INFO org.apache.zookeeper.ClientCnxn - EventThread shut down
[2014-03-20 21:12:27,290] 685698876 [main-SendThread()] INFO org.apache.zookeeper.ClientCnxn - Opening socket connection to server analytics1023.eqiad.wmnet/10.64.5.15:2181
[2014-03-20 21:12:27,292] 685698878 [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO org.apache.zookeeper.ClientCnxn - Socket connection established to analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
[2014-03-20 21:12:27,313] 685698899 [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO org.apache.zookeeper.ClientCnxn - Session establishment complete on server analytics1023.eqiad.wmnet/10.64.5.15:2181, sessionid = 0xff429b4201dd2352, negotiated timeout = 6000
[2014-03-20 21:12:27,314] 685698900 [main-EventThread] INFO org.I0Itec.zkclient.ZkClient - zookeeper state changed (SyncConnected)
[2014-03-20 21:12:27,324] 685698910 [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad] INFO kafka.server.KafkaZooKeeper - done re-registering broker
[2014-03-20 21:12:27,324] 685698910 [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad] INFO kafka.server.KafkaZooKeeper - Subscribing to /brokers/topics path to watch for new topics
[2014-03-20 21:12:27,370] 685698956 [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad] INFO kafka.server.ZookeeperLeaderElector$LeaderChangeListener - New leader is 22



Re: Zookeeper reconnect failed due to 'state changed (Expired)'

Posted by Neha Narkhede <ne...@gmail.com>.
Kafka
internally only handles session expired event but there are many bad
situations of zookeepers where Kafka can go haywire.

Interesting. Would you mind giving more details about this? Kafka does
handle events other than session expirations through the zkclient library.
Also, Zookeeper only loses ephemeral nodes on session expirations. It does
lose ephemeral nodes in other circumstances, when the cluster starves on
I/O, but those expose zookeeper corner case bugs :-)

Thanks,
Neha


On Fri, Mar 21, 2014 at 7:51 AM, Jun Rao <ju...@gmail.com> wrote:

> Hi, Jae,
>
> My understanding is that ephemeral nodes can only be lost on ZK session
> expiration. Are you saying that there are other cases when ephemeral nodes
> can be lost?
>
> Thanks,
>
> Jun
>
>
> On Thu, Mar 20, 2014 at 9:52 PM, Bae, Jae Hyeon <me...@gmail.com>
> wrote:
>
> > This issue is zookeeper resiliency.
> >
> > What I have done is, ephemeral node creation is replaced by Apache
> > Curator's PersistentEphemeralNode recipe, to reinstate ephemeral nodes
> > after zookeeper blip. Also, all watchers also should be reinstated. Kafka
> > internally only handles session expired event but there are many bad
> > situations of zookeepers where Kafka can go haywire.
> >
> >
> >
> >
> > On Thu, Mar 20, 2014 at 9:23 PM, Neha Narkhede <neha.narkhede@gmail.com
> > >wrote:
> >
> > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > >
> > > 2.  Has anyone seen issues like this before?  Where zookeeper
> connections
> > > are flaky enough to cause leader elections?
> > >
> > > It means zookeeper expired the session. The most common reason for this
> > is
> > > client side GC (in your case, client is the Kafka broker) though if
> your
> > > zookeeper setup is flaky, this might even point to a server side I/O
> > > bottleneck or GC. Could you inspect the broker and zookeeper's gc logs?
> > >
> > >
> > > On Thu, Mar 20, 2014 at 3:26 PM, Andrew Otto <ot...@wikimedia.org>
> wrote:
> > >
> > > > Hm, just saw something a little fishy.
> > > >
> > > > (In the following logs, analytics1021 (id 21) and analytics1022 (id
> 22)
> > > > are Brokers and analytics1023,analytics1024,analytics1025 are
> > > Zookeepers.)
> > > >
> > > > At 2014-03-20 21:12:26, analytics1021 lost its connection to
> zookeeper.
> > > It
> > > > reconnected to analytics1023, but as it was doing so, it logged a
> > > > 'zookeeper state changed (Expired)' message, which apparently caused
> > the
> > > > newly opened socket to close again.  Am I reading that right?
> > > >
> > > > In the time that it took analytics1021 to finally re-establish a
> > > zookeeper
> > > > connection, analytics1022 had become the leader for all partitions.
>  I
> > > can
> > > > rebalance the leaders, but I'd prefer if this didn't happen in the
> > first
> > > > place.
> > > >
> > > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > > > 2.  Has anyone seen issues like this before?  Where zookeeper
> > connections
> > > > are flaky enough to cause leader elections?
> > > >
> > > > Thanks!
> > > > -Andrew Otto
> > > >
> > > >
> > > >
> > > > [2014-03-20 21:12:26,427] 685698013
> > > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > > org.apache.zookeeper.ClientCnxn - Client session timed out, have not
> > > heard
> > > > from server in 10067ms for sessionid 0x1425d903a3f1f89, closing
> socket
> > > > connection and attempting reconnect
> > > > [2014-03-20 21:12:26,528] 685698114 [main-EventThread] INFO
> > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed (Disconnected)
> > > > [2014-03-20 21:12:27,252] 685698838
> > > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > > org.apache.zookeeper.ClientCnxn - Opening socket connection to server
> > > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > > [2014-03-20 21:12:27,271] 685698857
> > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > org.apache.zookeeper.ClientCnxn - Socket connection established to
> > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> > > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed (Expired)
> > > > [2014-03-20 21:12:27,273] 685698859
> > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > org.apache.zookeeper.ClientCnxn - Unable to reconnect to ZooKeeper
> > > service,
> > > > session 0x1425d903a3f1f89 has expired, closing socket connection
> > > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > > org.apache.zookeeper.ZooKeeper - Initiating client connection,
> > > >
> > >
> >
> connectString=analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad
> > > > sessionTimeout=6000 watcher=org.I0Itec.zkclient.ZkClient@63203b59
> > > > [2014-03-20 21:12:27,290] 685698876
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > INFO kafka.server.KafkaZooKeeper - re-registering broker info in ZK
> for
> > > > broker 21
> > > > [2014-03-20 21:12:27,290] 685698876 [main-EventThread] INFO
> > > > org.apache.zookeeper.ClientCnxn - EventThread shut down
> > > > [2014-03-20 21:12:27,290] 685698876 [main-SendThread()] INFO
> > > > org.apache.zookeeper.ClientCnxn - Opening socket connection to server
> > > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > > [2014-03-20 21:12:27,292] 685698878
> > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > org.apache.zookeeper.ClientCnxn - Socket connection established to
> > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> > > > [2014-03-20 21:12:27,313] 685698899
> > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > org.apache.zookeeper.ClientCnxn - Session establishment complete on
> > > server
> > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, sessionid =
> > > > 0xff429b4201dd2352, negotiated timeout = 6000
> > > > [2014-03-20 21:12:27,314] 685698900 [main-EventThread] INFO
> > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> (SyncConnected)
> > > > [2014-03-20 21:12:27,324] 685698910
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > INFO kafka.server.KafkaZooKeeper - done re-registering broker
> > > > [2014-03-20 21:12:27,324] 685698910
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > INFO kafka.server.KafkaZooKeeper - Subscribing to /brokers/topics
> path
> > to
> > > > watch for new topics
> > > > [2014-03-20 21:12:27,370] 685698956
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > INFO kafka.server.ZookeeperLeaderElector$LeaderChangeListener - New
> > > leader
> > > > is 22
> > > >
> > > >
> > > >
> > >
> >
>

Re: Zookeeper reconnect failed due to 'state changed (Expired)'

Posted by Andrew Otto <ao...@wikimedia.org>.
Ah!  That makes so much sense, and is likely the cause of our lost messages.

Thanks, I am now experimenting with other ack values.  I’m a little worried
about latency, especially since some of our producers send traffic across
the Atlantic (we don’t use MirrorMaker :/ ).




On Oct 1, 2014, at 10:14 AM, Neha Narkhede <ne...@gmail.com> wrote:

> But, as long as broker with the unreplicated messages comes back into the
> ISR, those messages should be replicated, no?
> 
> Or, would the fact that the a new leader has been chosen for a partition,
> and new messages have been sent to that new leader, cause the offending
> broker to drop the unreplicated messages when it comes back into the ISR?
> 
> The moment the leader moves to another broker, the partition's source of
> truth is the new broker's log and other followers truncate their logs to
> follow the new leader. So, any unreplicated messages that didn't reach the
> new leader are lost. If the old leader rejoins ISR, it will also truncate
> it's log to follow the new leader's log.
> 
> Thanks,
> Neha
> 
> On Wed, Oct 1, 2014 at 5:48 AM, Andrew Otto <ao...@wikimedia.org> wrote:
> 
>> I understand that, but even if the leader quickly (within a few seconds)
>> rejoins the cluster?  I had thought that ack=1 meant that messages not yet
>> replicated from a broker that has a serious fatal failure (disk corruption,
>> etc.) would be lost forever.  But, as long as broker with the unreplicated
>> messages comes back into the ISR, those messages should be replicated, no?
>> 
>> Or, would the fact that the a new leader has been chosen for a partition,
>> and new messages have been sent to that new leader, cause the offending
>> broker to drop the unreplicated messages when it comes back into the ISR?
>> 
>> 
>> On Sep 30, 2014, at 7:17 PM, Jun Rao <ju...@gmail.com> wrote:
>> 
>>> With ack=1, acked messages could be lost when the leader fails.
>>> 
>>> Thanks,
>>> 
>>> Jun
>>> 
>>> On Mon, Sep 29, 2014 at 8:04 AM, Andrew Otto <ot...@wikimedia.org> wrote:
>>> 
>>>> This happened again to me this weekend.  I've done some sleuthing, and
>>>> definitely can see some crazy paging stats when this lock up happens.
>> For
>>>> the curious, more info can be found here:
>>>> https://bugzilla.wikimedia.org/show_bug.cgi?id=69667.  I had tuned
>>>> dirty_expire_centisecs from 30 seconds to 10, but this still happened
>>>> again.  I'll continue to troubleshoot and tune.  This slow going
>> because it
>>>> is not regularly reproducible.  I have to make a single change and then
>>>> wait a week or two for the timeout to occur.
>>>> 
>>>> Here's a related question.  When the timeout happens, we lose some
>>>> messages.  Our producer is varnishkafka, which uses the librdkafka
>> producer
>>>> client.  librdkafka keeps track of produce errors.  We
>>>> have kafka.topic.request.required.acks = 1.  According to librdkafka,
>> all
>>>> messages sent have been ACKed by the leader of the partition to which
>> the
>>>> messages are sent.  Also, when we lose messages due to this timeout, the
>>>> broker that times out is always the controller.  When it attempts to
>>>> reconnect to Zookeeper, we see:
>>>> 
>>>> INFO  kafka.utils.ZkUtils$  - conflict in /controller data:
>>>> {"version":1,"brokerid":21,"timestamp":"1411879981756"} stored data:
>>>> {"version":1,"brokerid":22,"timestamp":"1407187809296"}
>>>> 
>>>> In the case when a controller drops out of the ISR for a few seconds,
>> is it
>>>> possible for this confused broker to drop ACKed messages?
>>>> 
>>>> 
>>>> 
>>>> On Thu, Jul 3, 2014 at 12:48 AM, Jun Rao <ju...@gmail.com> wrote:
>>>> 
>>>>> Are you on Linux? We have seen this pattern (user/sys time low and real
>>>>> time high in GC time) before. In our case, the problem was due to disk
>>>>> I/Os. When there are lots of dirty pages (in our case, this is caused
>> by
>>>>> log4j logging), Linux can draft user threads (in this case GC threads)
>> to
>>>>> flush the dirty pages. So, all those time in real was spent on disk
>> I/Os,
>>>>> rather than real GCs. The fix is to tune dirty_expire_centisecs and
>>>>> dirty_writeback_centisecs
>>>>> to flush dirty pages more frequently to avoid such drafting.
>>>>> 
>>>>> Thanks,
>>>>> 
>>>>> Jun
>>>>> 
>>>>> 
>>>>> On Wed, Jul 2, 2014 at 1:32 PM, Andrew Otto <ot...@wikimedia.org>
>> wrote:
>>>>> 
>>>>>> Hi again!
>>>>>> 
>>>>>> I've been having this issue consistently since I first started this
>>>>> thread,
>>>>>> but it was happening infrequently enough for me to brush it aside and
>>>>> just
>>>>>> run an election to rebalance brokers.
>>>>>> 
>>>>>> I recently expanded (and reinstalled) our Kafka cluster so that it now
>>>>> has
>>>>>> 4 brokers with a default replication factor of 3 for each partition.
>> I
>>>>>> also switched over to the G1GC as recommended here:
>>>>>> https://kafka.apache.org/081/ops.html (even though we are still
>>>> running
>>>>>> Kafka 0.8.0; we hope to upgrade soon).
>>>>>> 
>>>>>> Now, only one of the 4 brokers (analytics1021, the same problem broker
>>>> we
>>>>>> saw before) gets its ZK connection expired even more frequently.
>>>>>> Previously it was less than once a week, now I am seeing this happen
>>>>>> multiple times a day.
>>>>>> 
>>>>>> I've posted all the relevant logs from a recent event here:
>>>>>> https://gist.github.com/ottomata/e42480446c627ea0af22
>>>>>> 
>>>>>> This includes the GC log on the offending Kafka broker during the time
>>>>> this
>>>>>> happened.  I am pretty green when it comes to GC tuning, but I do see
>>>>> this
>>>>>> interesting stat:
>>>>>> 
>>>>>> 
>>>>>> [Times: user=0.14 sys=0.00, real=11.47 secs]
>>>>>> 
>>>>>> Did Kafka's JVM really just take 11.47 secs to do a GC there?  I'm
>>>>>> probably missing something, but I don't see which part of that real
>>>>>> time summary makes up the bulk of that GC time
>>>>>> 
>>>>>> This is strange, riight?  This broker is identically configured to all
>>>>>> its peers, and should be handling on average the exact same amount and
>>>>>> type of traffic.  Anyone have any advice?
>>>>>> 
>>>>>> Thanks!
>>>>>> -Andrew Otto
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> On Fri, Mar 21, 2014 at 6:48 PM, Neha Narkhede <
>>>> neha.narkhede@gmail.com>
>>>>>> wrote:
>>>>>> 
>>>>>>> I see, that makes sense. Thanks a lot for clarifying!
>>>>>>> 
>>>>>>> -Neha
>>>>>>> 
>>>>>>> 
>>>>>>> On Fri, Mar 21, 2014 at 11:01 AM, Bae, Jae Hyeon <metacret@gmail.com
>>>>> 
>>>>>>> wrote:
>>>>>>> 
>>>>>>>> Let me clarify the situation. I forgot to mention that my case
>>>> might
>>>>>> not
>>>>>>> be
>>>>>>>> general one because Netflix is using Apache Curator as the main
>>>>>> zookeeper
>>>>>>>> client and ZkClient in Kafka should be bridged to Apache Curator,
>>>> so
>>>>>> the
>>>>>>>> behavior I have seen might not be general one.
>>>>>>>> 
>>>>>>>> Kafka's ZKSessionExpireListener.handleNewSession() is reinstating
>>>> all
>>>>>>>> ephemeral nodes and watchers but handleNewSession() was not kicked
>>>> in
>>>>>> my
>>>>>>>> case. So, I created Netflix internal version of ZkClient to replace
>>>>>>>> ephemeral node creation and watcher reinstating.
>>>>>>>> 
>>>>>>>> I have a plan to remove all external dependency from Kafka soon
>>>> and I
>>>>>>> might
>>>>>>>> be able to mention more about Kafka's zk resiliency.
>>>>>>>> 
>>>>>>>> 
>>>>>>>> On Fri, Mar 21, 2014 at 7:51 AM, Jun Rao <ju...@gmail.com> wrote:
>>>>>>>> 
>>>>>>>>> Hi, Jae,
>>>>>>>>> 
>>>>>>>>> My understanding is that ephemeral nodes can only be lost on ZK
>>>>>> session
>>>>>>>>> expiration. Are you saying that there are other cases when
>>>>> ephemeral
>>>>>>>> nodes
>>>>>>>>> can be lost?
>>>>>>>>> 
>>>>>>>>> Thanks,
>>>>>>>>> 
>>>>>>>>> Jun
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> On Thu, Mar 20, 2014 at 9:52 PM, Bae, Jae Hyeon <
>>>>> metacret@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> This issue is zookeeper resiliency.
>>>>>>>>>> 
>>>>>>>>>> What I have done is, ephemeral node creation is replaced by
>>>>> Apache
>>>>>>>>>> Curator's PersistentEphemeralNode recipe, to reinstate
>>>> ephemeral
>>>>>>> nodes
>>>>>>>>>> after zookeeper blip. Also, all watchers also should be
>>>>> reinstated.
>>>>>>>> Kafka
>>>>>>>>>> internally only handles session expired event but there are
>>>> many
>>>>>> bad
>>>>>>>>>> situations of zookeepers where Kafka can go haywire.
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> On Thu, Mar 20, 2014 at 9:23 PM, Neha Narkhede <
>>>>>>>> neha.narkhede@gmail.com
>>>>>>>>>>> wrote:
>>>>>>>>>> 
>>>>>>>>>>> 1.  What does 'zookeeper state changed (Expired)' mean?
>>>>>>>>>>> 
>>>>>>>>>>> 2.  Has anyone seen issues like this before?  Where zookeeper
>>>>>>>>> connections
>>>>>>>>>>> are flaky enough to cause leader elections?
>>>>>>>>>>> 
>>>>>>>>>>> It means zookeeper expired the session. The most common
>>>> reason
>>>>>> for
>>>>>>>> this
>>>>>>>>>> is
>>>>>>>>>>> client side GC (in your case, client is the Kafka broker)
>>>>> though
>>>>>> if
>>>>>>>>> your
>>>>>>>>>>> zookeeper setup is flaky, this might even point to a server
>>>>> side
>>>>>>> I/O
>>>>>>>>>>> bottleneck or GC. Could you inspect the broker and
>>>> zookeeper's
>>>>> gc
>>>>>>>> logs?
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> On Thu, Mar 20, 2014 at 3:26 PM, Andrew Otto <
>>>>> otto@wikimedia.org
>>>>>>> 
>>>>>>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>>> Hm, just saw something a little fishy.
>>>>>>>>>>>> 
>>>>>>>>>>>> (In the following logs, analytics1021 (id 21) and
>>>>> analytics1022
>>>>>>> (id
>>>>>>>>> 22)
>>>>>>>>>>>> are Brokers and analytics1023,analytics1024,analytics1025
>>>> are
>>>>>>>>>>> Zookeepers.)
>>>>>>>>>>>> 
>>>>>>>>>>>> At 2014-03-20 21:12:26, analytics1021 lost its connection
>>>> to
>>>>>>>>> zookeeper.
>>>>>>>>>>> It
>>>>>>>>>>>> reconnected to analytics1023, but as it was doing so, it
>>>>>> logged a
>>>>>>>>>>>> 'zookeeper state changed (Expired)' message, which
>>>> apparently
>>>>>>>> caused
>>>>>>>>>> the
>>>>>>>>>>>> newly opened socket to close again.  Am I reading that
>>>> right?
>>>>>>>>>>>> 
>>>>>>>>>>>> In the time that it took analytics1021 to finally
>>>>> re-establish
>>>>>> a
>>>>>>>>>>> zookeeper
>>>>>>>>>>>> connection, analytics1022 had become the leader for all
>>>>>>> partitions.
>>>>>>>>> I
>>>>>>>>>>> can
>>>>>>>>>>>> rebalance the leaders, but I'd prefer if this didn't happen
>>>>> in
>>>>>>> the
>>>>>>>>>> first
>>>>>>>>>>>> place.
>>>>>>>>>>>> 
>>>>>>>>>>>> 1.  What does 'zookeeper state changed (Expired)' mean?
>>>>>>>>>>>> 2.  Has anyone seen issues like this before?  Where
>>>> zookeeper
>>>>>>>>>> connections
>>>>>>>>>>>> are flaky enough to cause leader elections?
>>>>>>>>>>>> 
>>>>>>>>>>>> Thanks!
>>>>>>>>>>>> -Andrew Otto
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> [2014-03-20 21:12:26,427] 685698013
>>>>>>>>>>>> [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
>>>>>>>>>>>> org.apache.zookeeper.ClientCnxn - Client session timed out,
>>>>>> have
>>>>>>>> not
>>>>>>>>>>> heard
>>>>>>>>>>>> from server in 10067ms for sessionid 0x1425d903a3f1f89,
>>>>> closing
>>>>>>>>> socket
>>>>>>>>>>>> connection and attempting reconnect
>>>>>>>>>>>> [2014-03-20 21:12:26,528] 685698114 [main-EventThread] INFO
>>>>>>>>>>>> org.I0Itec.zkclient.ZkClient - zookeeper state changed
>>>>>>>> (Disconnected)
>>>>>>>>>>>> [2014-03-20 21:12:27,252] 685698838
>>>>>>>>>>>> [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
>>>>>>>>>>>> org.apache.zookeeper.ClientCnxn - Opening socket connection
>>>>> to
>>>>>>>> server
>>>>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181
>>>>>>>>>>>> [2014-03-20 21:12:27,271] 685698857
>>>>>>>>>>>> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
>>>>>>>>>>>> org.apache.zookeeper.ClientCnxn - Socket connection
>>>>> established
>>>>>>> to
>>>>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating
>>>>> session
>>>>>>>>>>>> [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
>>>>>>>>>>>> org.I0Itec.zkclient.ZkClient - zookeeper state changed
>>>>>> (Expired)
>>>>>>>>>>>> [2014-03-20 21:12:27,273] 685698859
>>>>>>>>>>>> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
>>>>>>>>>>>> org.apache.zookeeper.ClientCnxn - Unable to reconnect to
>>>>>>> ZooKeeper
>>>>>>>>>>> service,
>>>>>>>>>>>> session 0x1425d903a3f1f89 has expired, closing socket
>>>>>> connection
>>>>>>>>>>>> [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
>>>>>>>>>>>> org.apache.zookeeper.ZooKeeper - Initiating client
>>>>> connection,
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>> connectString=analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad
>>>>>>>>>>>> sessionTimeout=6000
>>>>>> watcher=org.I0Itec.zkclient.ZkClient@63203b59
>>>>>>>>>>>> [2014-03-20 21:12:27,290] 685698876
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
>>>>>>>>>>>> INFO kafka.server.KafkaZooKeeper - re-registering broker
>>>> info
>>>>>> in
>>>>>>> ZK
>>>>>>>>> for
>>>>>>>>>>>> broker 21
>>>>>>>>>>>> [2014-03-20 21:12:27,290] 685698876 [main-EventThread] INFO
>>>>>>>>>>>> org.apache.zookeeper.ClientCnxn - EventThread shut down
>>>>>>>>>>>> [2014-03-20 21:12:27,290] 685698876 [main-SendThread()]
>>>> INFO
>>>>>>>>>>>> org.apache.zookeeper.ClientCnxn - Opening socket connection
>>>>> to
>>>>>>>> server
>>>>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181
>>>>>>>>>>>> [2014-03-20 21:12:27,292] 685698878
>>>>>>>>>>>> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
>>>>>>>>>>>> org.apache.zookeeper.ClientCnxn - Socket connection
>>>>> established
>>>>>>> to
>>>>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating
>>>>> session
>>>>>>>>>>>> [2014-03-20 21:12:27,313] 685698899
>>>>>>>>>>>> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
>>>>>>>>>>>> org.apache.zookeeper.ClientCnxn - Session establishment
>>>>>> complete
>>>>>>> on
>>>>>>>>>>> server
>>>>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181, sessionid =
>>>>>>>>>>>> 0xff429b4201dd2352, negotiated timeout = 6000
>>>>>>>>>>>> [2014-03-20 21:12:27,314] 685698900 [main-EventThread] INFO
>>>>>>>>>>>> org.I0Itec.zkclient.ZkClient - zookeeper state changed
>>>>>>>>> (SyncConnected)
>>>>>>>>>>>> [2014-03-20 21:12:27,324] 685698910
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
>>>>>>>>>>>> INFO kafka.server.KafkaZooKeeper - done re-registering
>>>> broker
>>>>>>>>>>>> [2014-03-20 21:12:27,324] 685698910
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
>>>>>>>>>>>> INFO kafka.server.KafkaZooKeeper - Subscribing to
>>>>>> /brokers/topics
>>>>>>>>> path
>>>>>>>>>> to
>>>>>>>>>>>> watch for new topics
>>>>>>>>>>>> [2014-03-20 21:12:27,370] 685698956
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
>>>>>>>>>>>> INFO
>>>>> kafka.server.ZookeeperLeaderElector$LeaderChangeListener -
>>>>>>> New
>>>>>>>>>>> leader
>>>>>>>>>>>> is 22
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>> 
>> 


Re: Zookeeper reconnect failed due to 'state changed (Expired)'

Posted by Neha Narkhede <ne...@gmail.com>.
 But, as long as broker with the unreplicated messages comes back into the
ISR, those messages should be replicated, no?

Or, would the fact that the a new leader has been chosen for a partition,
and new messages have been sent to that new leader, cause the offending
broker to drop the unreplicated messages when it comes back into the ISR?

The moment the leader moves to another broker, the partition's source of
truth is the new broker's log and other followers truncate their logs to
follow the new leader. So, any unreplicated messages that didn't reach the
new leader are lost. If the old leader rejoins ISR, it will also truncate
it's log to follow the new leader's log.

Thanks,
Neha

On Wed, Oct 1, 2014 at 5:48 AM, Andrew Otto <ao...@wikimedia.org> wrote:

> I understand that, but even if the leader quickly (within a few seconds)
> rejoins the cluster?  I had thought that ack=1 meant that messages not yet
> replicated from a broker that has a serious fatal failure (disk corruption,
> etc.) would be lost forever.  But, as long as broker with the unreplicated
> messages comes back into the ISR, those messages should be replicated, no?
>
> Or, would the fact that the a new leader has been chosen for a partition,
> and new messages have been sent to that new leader, cause the offending
> broker to drop the unreplicated messages when it comes back into the ISR?
>
>
> On Sep 30, 2014, at 7:17 PM, Jun Rao <ju...@gmail.com> wrote:
>
> > With ack=1, acked messages could be lost when the leader fails.
> >
> > Thanks,
> >
> > Jun
> >
> > On Mon, Sep 29, 2014 at 8:04 AM, Andrew Otto <ot...@wikimedia.org> wrote:
> >
> >> This happened again to me this weekend.  I've done some sleuthing, and
> >> definitely can see some crazy paging stats when this lock up happens.
> For
> >> the curious, more info can be found here:
> >> https://bugzilla.wikimedia.org/show_bug.cgi?id=69667.  I had tuned
> >> dirty_expire_centisecs from 30 seconds to 10, but this still happened
> >> again.  I'll continue to troubleshoot and tune.  This slow going
> because it
> >> is not regularly reproducible.  I have to make a single change and then
> >> wait a week or two for the timeout to occur.
> >>
> >> Here's a related question.  When the timeout happens, we lose some
> >> messages.  Our producer is varnishkafka, which uses the librdkafka
> producer
> >> client.  librdkafka keeps track of produce errors.  We
> >> have kafka.topic.request.required.acks = 1.  According to librdkafka,
> all
> >> messages sent have been ACKed by the leader of the partition to which
> the
> >> messages are sent.  Also, when we lose messages due to this timeout, the
> >> broker that times out is always the controller.  When it attempts to
> >> reconnect to Zookeeper, we see:
> >>
> >>  INFO  kafka.utils.ZkUtils$  - conflict in /controller data:
> >> {"version":1,"brokerid":21,"timestamp":"1411879981756"} stored data:
> >> {"version":1,"brokerid":22,"timestamp":"1407187809296"}
> >>
> >> In the case when a controller drops out of the ISR for a few seconds,
> is it
> >> possible for this confused broker to drop ACKed messages?
> >>
> >>
> >>
> >> On Thu, Jul 3, 2014 at 12:48 AM, Jun Rao <ju...@gmail.com> wrote:
> >>
> >>> Are you on Linux? We have seen this pattern (user/sys time low and real
> >>> time high in GC time) before. In our case, the problem was due to disk
> >>> I/Os. When there are lots of dirty pages (in our case, this is caused
> by
> >>> log4j logging), Linux can draft user threads (in this case GC threads)
> to
> >>> flush the dirty pages. So, all those time in real was spent on disk
> I/Os,
> >>> rather than real GCs. The fix is to tune dirty_expire_centisecs and
> >>> dirty_writeback_centisecs
> >>> to flush dirty pages more frequently to avoid such drafting.
> >>>
> >>> Thanks,
> >>>
> >>> Jun
> >>>
> >>>
> >>> On Wed, Jul 2, 2014 at 1:32 PM, Andrew Otto <ot...@wikimedia.org>
> wrote:
> >>>
> >>>> Hi again!
> >>>>
> >>>> I've been having this issue consistently since I first started this
> >>> thread,
> >>>> but it was happening infrequently enough for me to brush it aside and
> >>> just
> >>>> run an election to rebalance brokers.
> >>>>
> >>>> I recently expanded (and reinstalled) our Kafka cluster so that it now
> >>> has
> >>>> 4 brokers with a default replication factor of 3 for each partition.
> I
> >>>> also switched over to the G1GC as recommended here:
> >>>> https://kafka.apache.org/081/ops.html (even though we are still
> >> running
> >>>> Kafka 0.8.0; we hope to upgrade soon).
> >>>>
> >>>> Now, only one of the 4 brokers (analytics1021, the same problem broker
> >> we
> >>>> saw before) gets its ZK connection expired even more frequently.
> >>>> Previously it was less than once a week, now I am seeing this happen
> >>>> multiple times a day.
> >>>>
> >>>> I've posted all the relevant logs from a recent event here:
> >>>> https://gist.github.com/ottomata/e42480446c627ea0af22
> >>>>
> >>>> This includes the GC log on the offending Kafka broker during the time
> >>> this
> >>>> happened.  I am pretty green when it comes to GC tuning, but I do see
> >>> this
> >>>> interesting stat:
> >>>>
> >>>>
> >>>> [Times: user=0.14 sys=0.00, real=11.47 secs]
> >>>>
> >>>> Did Kafka's JVM really just take 11.47 secs to do a GC there?  I'm
> >>>> probably missing something, but I don't see which part of that real
> >>>> time summary makes up the bulk of that GC time
> >>>>
> >>>> This is strange, riight?  This broker is identically configured to all
> >>>> its peers, and should be handling on average the exact same amount and
> >>>> type of traffic.  Anyone have any advice?
> >>>>
> >>>> Thanks!
> >>>> -Andrew Otto
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> On Fri, Mar 21, 2014 at 6:48 PM, Neha Narkhede <
> >> neha.narkhede@gmail.com>
> >>>> wrote:
> >>>>
> >>>>> I see, that makes sense. Thanks a lot for clarifying!
> >>>>>
> >>>>> -Neha
> >>>>>
> >>>>>
> >>>>> On Fri, Mar 21, 2014 at 11:01 AM, Bae, Jae Hyeon <metacret@gmail.com
> >>>
> >>>>> wrote:
> >>>>>
> >>>>>> Let me clarify the situation. I forgot to mention that my case
> >> might
> >>>> not
> >>>>> be
> >>>>>> general one because Netflix is using Apache Curator as the main
> >>>> zookeeper
> >>>>>> client and ZkClient in Kafka should be bridged to Apache Curator,
> >> so
> >>>> the
> >>>>>> behavior I have seen might not be general one.
> >>>>>>
> >>>>>> Kafka's ZKSessionExpireListener.handleNewSession() is reinstating
> >> all
> >>>>>> ephemeral nodes and watchers but handleNewSession() was not kicked
> >> in
> >>>> my
> >>>>>> case. So, I created Netflix internal version of ZkClient to replace
> >>>>>> ephemeral node creation and watcher reinstating.
> >>>>>>
> >>>>>> I have a plan to remove all external dependency from Kafka soon
> >> and I
> >>>>> might
> >>>>>> be able to mention more about Kafka's zk resiliency.
> >>>>>>
> >>>>>>
> >>>>>> On Fri, Mar 21, 2014 at 7:51 AM, Jun Rao <ju...@gmail.com> wrote:
> >>>>>>
> >>>>>>> Hi, Jae,
> >>>>>>>
> >>>>>>> My understanding is that ephemeral nodes can only be lost on ZK
> >>>> session
> >>>>>>> expiration. Are you saying that there are other cases when
> >>> ephemeral
> >>>>>> nodes
> >>>>>>> can be lost?
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>>
> >>>>>>> Jun
> >>>>>>>
> >>>>>>>
> >>>>>>> On Thu, Mar 20, 2014 at 9:52 PM, Bae, Jae Hyeon <
> >>> metacret@gmail.com>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> This issue is zookeeper resiliency.
> >>>>>>>>
> >>>>>>>> What I have done is, ephemeral node creation is replaced by
> >>> Apache
> >>>>>>>> Curator's PersistentEphemeralNode recipe, to reinstate
> >> ephemeral
> >>>>> nodes
> >>>>>>>> after zookeeper blip. Also, all watchers also should be
> >>> reinstated.
> >>>>>> Kafka
> >>>>>>>> internally only handles session expired event but there are
> >> many
> >>>> bad
> >>>>>>>> situations of zookeepers where Kafka can go haywire.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Thu, Mar 20, 2014 at 9:23 PM, Neha Narkhede <
> >>>>>> neha.narkhede@gmail.com
> >>>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> 1.  What does 'zookeeper state changed (Expired)' mean?
> >>>>>>>>>
> >>>>>>>>> 2.  Has anyone seen issues like this before?  Where zookeeper
> >>>>>>> connections
> >>>>>>>>> are flaky enough to cause leader elections?
> >>>>>>>>>
> >>>>>>>>> It means zookeeper expired the session. The most common
> >> reason
> >>>> for
> >>>>>> this
> >>>>>>>> is
> >>>>>>>>> client side GC (in your case, client is the Kafka broker)
> >>> though
> >>>> if
> >>>>>>> your
> >>>>>>>>> zookeeper setup is flaky, this might even point to a server
> >>> side
> >>>>> I/O
> >>>>>>>>> bottleneck or GC. Could you inspect the broker and
> >> zookeeper's
> >>> gc
> >>>>>> logs?
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Thu, Mar 20, 2014 at 3:26 PM, Andrew Otto <
> >>> otto@wikimedia.org
> >>>>>
> >>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hm, just saw something a little fishy.
> >>>>>>>>>>
> >>>>>>>>>> (In the following logs, analytics1021 (id 21) and
> >>> analytics1022
> >>>>> (id
> >>>>>>> 22)
> >>>>>>>>>> are Brokers and analytics1023,analytics1024,analytics1025
> >> are
> >>>>>>>>> Zookeepers.)
> >>>>>>>>>>
> >>>>>>>>>> At 2014-03-20 21:12:26, analytics1021 lost its connection
> >> to
> >>>>>>> zookeeper.
> >>>>>>>>> It
> >>>>>>>>>> reconnected to analytics1023, but as it was doing so, it
> >>>> logged a
> >>>>>>>>>> 'zookeeper state changed (Expired)' message, which
> >> apparently
> >>>>>> caused
> >>>>>>>> the
> >>>>>>>>>> newly opened socket to close again.  Am I reading that
> >> right?
> >>>>>>>>>>
> >>>>>>>>>> In the time that it took analytics1021 to finally
> >>> re-establish
> >>>> a
> >>>>>>>>> zookeeper
> >>>>>>>>>> connection, analytics1022 had become the leader for all
> >>>>> partitions.
> >>>>>>> I
> >>>>>>>>> can
> >>>>>>>>>> rebalance the leaders, but I'd prefer if this didn't happen
> >>> in
> >>>>> the
> >>>>>>>> first
> >>>>>>>>>> place.
> >>>>>>>>>>
> >>>>>>>>>> 1.  What does 'zookeeper state changed (Expired)' mean?
> >>>>>>>>>> 2.  Has anyone seen issues like this before?  Where
> >> zookeeper
> >>>>>>>> connections
> >>>>>>>>>> are flaky enough to cause leader elections?
> >>>>>>>>>>
> >>>>>>>>>> Thanks!
> >>>>>>>>>> -Andrew Otto
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> [2014-03-20 21:12:26,427] 685698013
> >>>>>>>>>> [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> >>>>>>>>>> org.apache.zookeeper.ClientCnxn - Client session timed out,
> >>>> have
> >>>>>> not
> >>>>>>>>> heard
> >>>>>>>>>> from server in 10067ms for sessionid 0x1425d903a3f1f89,
> >>> closing
> >>>>>>> socket
> >>>>>>>>>> connection and attempting reconnect
> >>>>>>>>>> [2014-03-20 21:12:26,528] 685698114 [main-EventThread] INFO
> >>>>>>>>>> org.I0Itec.zkclient.ZkClient - zookeeper state changed
> >>>>>> (Disconnected)
> >>>>>>>>>> [2014-03-20 21:12:27,252] 685698838
> >>>>>>>>>> [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> >>>>>>>>>> org.apache.zookeeper.ClientCnxn - Opening socket connection
> >>> to
> >>>>>> server
> >>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181
> >>>>>>>>>> [2014-03-20 21:12:27,271] 685698857
> >>>>>>>>>> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> >>>>>>>>>> org.apache.zookeeper.ClientCnxn - Socket connection
> >>> established
> >>>>> to
> >>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating
> >>> session
> >>>>>>>>>> [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> >>>>>>>>>> org.I0Itec.zkclient.ZkClient - zookeeper state changed
> >>>> (Expired)
> >>>>>>>>>> [2014-03-20 21:12:27,273] 685698859
> >>>>>>>>>> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> >>>>>>>>>> org.apache.zookeeper.ClientCnxn - Unable to reconnect to
> >>>>> ZooKeeper
> >>>>>>>>> service,
> >>>>>>>>>> session 0x1425d903a3f1f89 has expired, closing socket
> >>>> connection
> >>>>>>>>>> [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> >>>>>>>>>> org.apache.zookeeper.ZooKeeper - Initiating client
> >>> connection,
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> connectString=analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad
> >>>>>>>>>> sessionTimeout=6000
> >>>> watcher=org.I0Itec.zkclient.ZkClient@63203b59
> >>>>>>>>>> [2014-03-20 21:12:27,290] 685698876
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> >>>>>>>>>> INFO kafka.server.KafkaZooKeeper - re-registering broker
> >> info
> >>>> in
> >>>>> ZK
> >>>>>>> for
> >>>>>>>>>> broker 21
> >>>>>>>>>> [2014-03-20 21:12:27,290] 685698876 [main-EventThread] INFO
> >>>>>>>>>> org.apache.zookeeper.ClientCnxn - EventThread shut down
> >>>>>>>>>> [2014-03-20 21:12:27,290] 685698876 [main-SendThread()]
> >> INFO
> >>>>>>>>>> org.apache.zookeeper.ClientCnxn - Opening socket connection
> >>> to
> >>>>>> server
> >>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181
> >>>>>>>>>> [2014-03-20 21:12:27,292] 685698878
> >>>>>>>>>> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> >>>>>>>>>> org.apache.zookeeper.ClientCnxn - Socket connection
> >>> established
> >>>>> to
> >>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating
> >>> session
> >>>>>>>>>> [2014-03-20 21:12:27,313] 685698899
> >>>>>>>>>> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> >>>>>>>>>> org.apache.zookeeper.ClientCnxn - Session establishment
> >>>> complete
> >>>>> on
> >>>>>>>>> server
> >>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181, sessionid =
> >>>>>>>>>> 0xff429b4201dd2352, negotiated timeout = 6000
> >>>>>>>>>> [2014-03-20 21:12:27,314] 685698900 [main-EventThread] INFO
> >>>>>>>>>> org.I0Itec.zkclient.ZkClient - zookeeper state changed
> >>>>>>> (SyncConnected)
> >>>>>>>>>> [2014-03-20 21:12:27,324] 685698910
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> >>>>>>>>>> INFO kafka.server.KafkaZooKeeper - done re-registering
> >> broker
> >>>>>>>>>> [2014-03-20 21:12:27,324] 685698910
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> >>>>>>>>>> INFO kafka.server.KafkaZooKeeper - Subscribing to
> >>>> /brokers/topics
> >>>>>>> path
> >>>>>>>> to
> >>>>>>>>>> watch for new topics
> >>>>>>>>>> [2014-03-20 21:12:27,370] 685698956
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> >>>>>>>>>> INFO
> >>> kafka.server.ZookeeperLeaderElector$LeaderChangeListener -
> >>>>> New
> >>>>>>>>> leader
> >>>>>>>>>> is 22
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
>
>

Re: Zookeeper reconnect failed due to 'state changed (Expired)'

Posted by Andrew Otto <ao...@wikimedia.org>.
I understand that, but even if the leader quickly (within a few seconds) rejoins the cluster?  I had thought that ack=1 meant that messages not yet replicated from a broker that has a serious fatal failure (disk corruption, etc.) would be lost forever.  But, as long as broker with the unreplicated messages comes back into the ISR, those messages should be replicated, no?

Or, would the fact that the a new leader has been chosen for a partition, and new messages have been sent to that new leader, cause the offending broker to drop the unreplicated messages when it comes back into the ISR?


On Sep 30, 2014, at 7:17 PM, Jun Rao <ju...@gmail.com> wrote:

> With ack=1, acked messages could be lost when the leader fails.
> 
> Thanks,
> 
> Jun
> 
> On Mon, Sep 29, 2014 at 8:04 AM, Andrew Otto <ot...@wikimedia.org> wrote:
> 
>> This happened again to me this weekend.  I've done some sleuthing, and
>> definitely can see some crazy paging stats when this lock up happens.  For
>> the curious, more info can be found here:
>> https://bugzilla.wikimedia.org/show_bug.cgi?id=69667.  I had tuned
>> dirty_expire_centisecs from 30 seconds to 10, but this still happened
>> again.  I'll continue to troubleshoot and tune.  This slow going because it
>> is not regularly reproducible.  I have to make a single change and then
>> wait a week or two for the timeout to occur.
>> 
>> Here's a related question.  When the timeout happens, we lose some
>> messages.  Our producer is varnishkafka, which uses the librdkafka producer
>> client.  librdkafka keeps track of produce errors.  We
>> have kafka.topic.request.required.acks = 1.  According to librdkafka, all
>> messages sent have been ACKed by the leader of the partition to which the
>> messages are sent.  Also, when we lose messages due to this timeout, the
>> broker that times out is always the controller.  When it attempts to
>> reconnect to Zookeeper, we see:
>> 
>>  INFO  kafka.utils.ZkUtils$  - conflict in /controller data:
>> {"version":1,"brokerid":21,"timestamp":"1411879981756"} stored data:
>> {"version":1,"brokerid":22,"timestamp":"1407187809296"}
>> 
>> In the case when a controller drops out of the ISR for a few seconds, is it
>> possible for this confused broker to drop ACKed messages?
>> 
>> 
>> 
>> On Thu, Jul 3, 2014 at 12:48 AM, Jun Rao <ju...@gmail.com> wrote:
>> 
>>> Are you on Linux? We have seen this pattern (user/sys time low and real
>>> time high in GC time) before. In our case, the problem was due to disk
>>> I/Os. When there are lots of dirty pages (in our case, this is caused by
>>> log4j logging), Linux can draft user threads (in this case GC threads) to
>>> flush the dirty pages. So, all those time in real was spent on disk I/Os,
>>> rather than real GCs. The fix is to tune dirty_expire_centisecs and
>>> dirty_writeback_centisecs
>>> to flush dirty pages more frequently to avoid such drafting.
>>> 
>>> Thanks,
>>> 
>>> Jun
>>> 
>>> 
>>> On Wed, Jul 2, 2014 at 1:32 PM, Andrew Otto <ot...@wikimedia.org> wrote:
>>> 
>>>> Hi again!
>>>> 
>>>> I've been having this issue consistently since I first started this
>>> thread,
>>>> but it was happening infrequently enough for me to brush it aside and
>>> just
>>>> run an election to rebalance brokers.
>>>> 
>>>> I recently expanded (and reinstalled) our Kafka cluster so that it now
>>> has
>>>> 4 brokers with a default replication factor of 3 for each partition.  I
>>>> also switched over to the G1GC as recommended here:
>>>> https://kafka.apache.org/081/ops.html (even though we are still
>> running
>>>> Kafka 0.8.0; we hope to upgrade soon).
>>>> 
>>>> Now, only one of the 4 brokers (analytics1021, the same problem broker
>> we
>>>> saw before) gets its ZK connection expired even more frequently.
>>>> Previously it was less than once a week, now I am seeing this happen
>>>> multiple times a day.
>>>> 
>>>> I've posted all the relevant logs from a recent event here:
>>>> https://gist.github.com/ottomata/e42480446c627ea0af22
>>>> 
>>>> This includes the GC log on the offending Kafka broker during the time
>>> this
>>>> happened.  I am pretty green when it comes to GC tuning, but I do see
>>> this
>>>> interesting stat:
>>>> 
>>>> 
>>>> [Times: user=0.14 sys=0.00, real=11.47 secs]
>>>> 
>>>> Did Kafka's JVM really just take 11.47 secs to do a GC there?  I'm
>>>> probably missing something, but I don't see which part of that real
>>>> time summary makes up the bulk of that GC time
>>>> 
>>>> This is strange, riight?  This broker is identically configured to all
>>>> its peers, and should be handling on average the exact same amount and
>>>> type of traffic.  Anyone have any advice?
>>>> 
>>>> Thanks!
>>>> -Andrew Otto
>>>> 
>>>> 
>>>> 
>>>> 
>>>> On Fri, Mar 21, 2014 at 6:48 PM, Neha Narkhede <
>> neha.narkhede@gmail.com>
>>>> wrote:
>>>> 
>>>>> I see, that makes sense. Thanks a lot for clarifying!
>>>>> 
>>>>> -Neha
>>>>> 
>>>>> 
>>>>> On Fri, Mar 21, 2014 at 11:01 AM, Bae, Jae Hyeon <metacret@gmail.com
>>> 
>>>>> wrote:
>>>>> 
>>>>>> Let me clarify the situation. I forgot to mention that my case
>> might
>>>> not
>>>>> be
>>>>>> general one because Netflix is using Apache Curator as the main
>>>> zookeeper
>>>>>> client and ZkClient in Kafka should be bridged to Apache Curator,
>> so
>>>> the
>>>>>> behavior I have seen might not be general one.
>>>>>> 
>>>>>> Kafka's ZKSessionExpireListener.handleNewSession() is reinstating
>> all
>>>>>> ephemeral nodes and watchers but handleNewSession() was not kicked
>> in
>>>> my
>>>>>> case. So, I created Netflix internal version of ZkClient to replace
>>>>>> ephemeral node creation and watcher reinstating.
>>>>>> 
>>>>>> I have a plan to remove all external dependency from Kafka soon
>> and I
>>>>> might
>>>>>> be able to mention more about Kafka's zk resiliency.
>>>>>> 
>>>>>> 
>>>>>> On Fri, Mar 21, 2014 at 7:51 AM, Jun Rao <ju...@gmail.com> wrote:
>>>>>> 
>>>>>>> Hi, Jae,
>>>>>>> 
>>>>>>> My understanding is that ephemeral nodes can only be lost on ZK
>>>> session
>>>>>>> expiration. Are you saying that there are other cases when
>>> ephemeral
>>>>>> nodes
>>>>>>> can be lost?
>>>>>>> 
>>>>>>> Thanks,
>>>>>>> 
>>>>>>> Jun
>>>>>>> 
>>>>>>> 
>>>>>>> On Thu, Mar 20, 2014 at 9:52 PM, Bae, Jae Hyeon <
>>> metacret@gmail.com>
>>>>>>> wrote:
>>>>>>> 
>>>>>>>> This issue is zookeeper resiliency.
>>>>>>>> 
>>>>>>>> What I have done is, ephemeral node creation is replaced by
>>> Apache
>>>>>>>> Curator's PersistentEphemeralNode recipe, to reinstate
>> ephemeral
>>>>> nodes
>>>>>>>> after zookeeper blip. Also, all watchers also should be
>>> reinstated.
>>>>>> Kafka
>>>>>>>> internally only handles session expired event but there are
>> many
>>>> bad
>>>>>>>> situations of zookeepers where Kafka can go haywire.
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> On Thu, Mar 20, 2014 at 9:23 PM, Neha Narkhede <
>>>>>> neha.narkhede@gmail.com
>>>>>>>>> wrote:
>>>>>>>> 
>>>>>>>>> 1.  What does 'zookeeper state changed (Expired)' mean?
>>>>>>>>> 
>>>>>>>>> 2.  Has anyone seen issues like this before?  Where zookeeper
>>>>>>> connections
>>>>>>>>> are flaky enough to cause leader elections?
>>>>>>>>> 
>>>>>>>>> It means zookeeper expired the session. The most common
>> reason
>>>> for
>>>>>> this
>>>>>>>> is
>>>>>>>>> client side GC (in your case, client is the Kafka broker)
>>> though
>>>> if
>>>>>>> your
>>>>>>>>> zookeeper setup is flaky, this might even point to a server
>>> side
>>>>> I/O
>>>>>>>>> bottleneck or GC. Could you inspect the broker and
>> zookeeper's
>>> gc
>>>>>> logs?
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> On Thu, Mar 20, 2014 at 3:26 PM, Andrew Otto <
>>> otto@wikimedia.org
>>>>> 
>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> Hm, just saw something a little fishy.
>>>>>>>>>> 
>>>>>>>>>> (In the following logs, analytics1021 (id 21) and
>>> analytics1022
>>>>> (id
>>>>>>> 22)
>>>>>>>>>> are Brokers and analytics1023,analytics1024,analytics1025
>> are
>>>>>>>>> Zookeepers.)
>>>>>>>>>> 
>>>>>>>>>> At 2014-03-20 21:12:26, analytics1021 lost its connection
>> to
>>>>>>> zookeeper.
>>>>>>>>> It
>>>>>>>>>> reconnected to analytics1023, but as it was doing so, it
>>>> logged a
>>>>>>>>>> 'zookeeper state changed (Expired)' message, which
>> apparently
>>>>>> caused
>>>>>>>> the
>>>>>>>>>> newly opened socket to close again.  Am I reading that
>> right?
>>>>>>>>>> 
>>>>>>>>>> In the time that it took analytics1021 to finally
>>> re-establish
>>>> a
>>>>>>>>> zookeeper
>>>>>>>>>> connection, analytics1022 had become the leader for all
>>>>> partitions.
>>>>>>> I
>>>>>>>>> can
>>>>>>>>>> rebalance the leaders, but I'd prefer if this didn't happen
>>> in
>>>>> the
>>>>>>>> first
>>>>>>>>>> place.
>>>>>>>>>> 
>>>>>>>>>> 1.  What does 'zookeeper state changed (Expired)' mean?
>>>>>>>>>> 2.  Has anyone seen issues like this before?  Where
>> zookeeper
>>>>>>>> connections
>>>>>>>>>> are flaky enough to cause leader elections?
>>>>>>>>>> 
>>>>>>>>>> Thanks!
>>>>>>>>>> -Andrew Otto
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> [2014-03-20 21:12:26,427] 685698013
>>>>>>>>>> [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
>>>>>>>>>> org.apache.zookeeper.ClientCnxn - Client session timed out,
>>>> have
>>>>>> not
>>>>>>>>> heard
>>>>>>>>>> from server in 10067ms for sessionid 0x1425d903a3f1f89,
>>> closing
>>>>>>> socket
>>>>>>>>>> connection and attempting reconnect
>>>>>>>>>> [2014-03-20 21:12:26,528] 685698114 [main-EventThread] INFO
>>>>>>>>>> org.I0Itec.zkclient.ZkClient - zookeeper state changed
>>>>>> (Disconnected)
>>>>>>>>>> [2014-03-20 21:12:27,252] 685698838
>>>>>>>>>> [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
>>>>>>>>>> org.apache.zookeeper.ClientCnxn - Opening socket connection
>>> to
>>>>>> server
>>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181
>>>>>>>>>> [2014-03-20 21:12:27,271] 685698857
>>>>>>>>>> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
>>>>>>>>>> org.apache.zookeeper.ClientCnxn - Socket connection
>>> established
>>>>> to
>>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating
>>> session
>>>>>>>>>> [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
>>>>>>>>>> org.I0Itec.zkclient.ZkClient - zookeeper state changed
>>>> (Expired)
>>>>>>>>>> [2014-03-20 21:12:27,273] 685698859
>>>>>>>>>> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
>>>>>>>>>> org.apache.zookeeper.ClientCnxn - Unable to reconnect to
>>>>> ZooKeeper
>>>>>>>>> service,
>>>>>>>>>> session 0x1425d903a3f1f89 has expired, closing socket
>>>> connection
>>>>>>>>>> [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
>>>>>>>>>> org.apache.zookeeper.ZooKeeper - Initiating client
>>> connection,
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> 
>> connectString=analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad
>>>>>>>>>> sessionTimeout=6000
>>>> watcher=org.I0Itec.zkclient.ZkClient@63203b59
>>>>>>>>>> [2014-03-20 21:12:27,290] 685698876
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> 
>> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
>>>>>>>>>> INFO kafka.server.KafkaZooKeeper - re-registering broker
>> info
>>>> in
>>>>> ZK
>>>>>>> for
>>>>>>>>>> broker 21
>>>>>>>>>> [2014-03-20 21:12:27,290] 685698876 [main-EventThread] INFO
>>>>>>>>>> org.apache.zookeeper.ClientCnxn - EventThread shut down
>>>>>>>>>> [2014-03-20 21:12:27,290] 685698876 [main-SendThread()]
>> INFO
>>>>>>>>>> org.apache.zookeeper.ClientCnxn - Opening socket connection
>>> to
>>>>>> server
>>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181
>>>>>>>>>> [2014-03-20 21:12:27,292] 685698878
>>>>>>>>>> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
>>>>>>>>>> org.apache.zookeeper.ClientCnxn - Socket connection
>>> established
>>>>> to
>>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating
>>> session
>>>>>>>>>> [2014-03-20 21:12:27,313] 685698899
>>>>>>>>>> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
>>>>>>>>>> org.apache.zookeeper.ClientCnxn - Session establishment
>>>> complete
>>>>> on
>>>>>>>>> server
>>>>>>>>>> analytics1023.eqiad.wmnet/10.64.5.15:2181, sessionid =
>>>>>>>>>> 0xff429b4201dd2352, negotiated timeout = 6000
>>>>>>>>>> [2014-03-20 21:12:27,314] 685698900 [main-EventThread] INFO
>>>>>>>>>> org.I0Itec.zkclient.ZkClient - zookeeper state changed
>>>>>>> (SyncConnected)
>>>>>>>>>> [2014-03-20 21:12:27,324] 685698910
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> 
>> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
>>>>>>>>>> INFO kafka.server.KafkaZooKeeper - done re-registering
>> broker
>>>>>>>>>> [2014-03-20 21:12:27,324] 685698910
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> 
>> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
>>>>>>>>>> INFO kafka.server.KafkaZooKeeper - Subscribing to
>>>> /brokers/topics
>>>>>>> path
>>>>>>>> to
>>>>>>>>>> watch for new topics
>>>>>>>>>> [2014-03-20 21:12:27,370] 685698956
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> 
>> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
>>>>>>>>>> INFO
>>> kafka.server.ZookeeperLeaderElector$LeaderChangeListener -
>>>>> New
>>>>>>>>> leader
>>>>>>>>>> is 22
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> 
>> 


Re: Zookeeper reconnect failed due to 'state changed (Expired)'

Posted by Jun Rao <ju...@gmail.com>.
With ack=1, acked messages could be lost when the leader fails.

Thanks,

Jun

On Mon, Sep 29, 2014 at 8:04 AM, Andrew Otto <ot...@wikimedia.org> wrote:

> This happened again to me this weekend.  I've done some sleuthing, and
> definitely can see some crazy paging stats when this lock up happens.  For
> the curious, more info can be found here:
> https://bugzilla.wikimedia.org/show_bug.cgi?id=69667.  I had tuned
> dirty_expire_centisecs from 30 seconds to 10, but this still happened
> again.  I'll continue to troubleshoot and tune.  This slow going because it
> is not regularly reproducible.  I have to make a single change and then
> wait a week or two for the timeout to occur.
>
> Here's a related question.  When the timeout happens, we lose some
> messages.  Our producer is varnishkafka, which uses the librdkafka producer
> client.  librdkafka keeps track of produce errors.  We
> have kafka.topic.request.required.acks = 1.  According to librdkafka, all
> messages sent have been ACKed by the leader of the partition to which the
> messages are sent.  Also, when we lose messages due to this timeout, the
> broker that times out is always the controller.  When it attempts to
> reconnect to Zookeeper, we see:
>
>   INFO  kafka.utils.ZkUtils$  - conflict in /controller data:
> {"version":1,"brokerid":21,"timestamp":"1411879981756"} stored data:
> {"version":1,"brokerid":22,"timestamp":"1407187809296"}
>
> In the case when a controller drops out of the ISR for a few seconds, is it
> possible for this confused broker to drop ACKed messages?
>
>
>
> On Thu, Jul 3, 2014 at 12:48 AM, Jun Rao <ju...@gmail.com> wrote:
>
> > Are you on Linux? We have seen this pattern (user/sys time low and real
> > time high in GC time) before. In our case, the problem was due to disk
> > I/Os. When there are lots of dirty pages (in our case, this is caused by
> > log4j logging), Linux can draft user threads (in this case GC threads) to
> > flush the dirty pages. So, all those time in real was spent on disk I/Os,
> > rather than real GCs. The fix is to tune dirty_expire_centisecs and
> > dirty_writeback_centisecs
> > to flush dirty pages more frequently to avoid such drafting.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Wed, Jul 2, 2014 at 1:32 PM, Andrew Otto <ot...@wikimedia.org> wrote:
> >
> > > Hi again!
> > >
> > > I've been having this issue consistently since I first started this
> > thread,
> > > but it was happening infrequently enough for me to brush it aside and
> > just
> > > run an election to rebalance brokers.
> > >
> > > I recently expanded (and reinstalled) our Kafka cluster so that it now
> > has
> > > 4 brokers with a default replication factor of 3 for each partition.  I
> > > also switched over to the G1GC as recommended here:
> > > https://kafka.apache.org/081/ops.html (even though we are still
> running
> > > Kafka 0.8.0; we hope to upgrade soon).
> > >
> > > Now, only one of the 4 brokers (analytics1021, the same problem broker
> we
> > > saw before) gets its ZK connection expired even more frequently.
> > >  Previously it was less than once a week, now I am seeing this happen
> > > multiple times a day.
> > >
> > > I've posted all the relevant logs from a recent event here:
> > > https://gist.github.com/ottomata/e42480446c627ea0af22
> > >
> > > This includes the GC log on the offending Kafka broker during the time
> > this
> > > happened.  I am pretty green when it comes to GC tuning, but I do see
> > this
> > > interesting stat:
> > >
> > >
> > >  [Times: user=0.14 sys=0.00, real=11.47 secs]
> > >
> > > Did Kafka's JVM really just take 11.47 secs to do a GC there?  I'm
> > > probably missing something, but I don't see which part of that real
> > > time summary makes up the bulk of that GC time
> > >
> > > This is strange, riight?  This broker is identically configured to all
> > > its peers, and should be handling on average the exact same amount and
> > > type of traffic.  Anyone have any advice?
> > >
> > > Thanks!
> > > -Andrew Otto
> > >
> > >
> > >
> > >
> > > On Fri, Mar 21, 2014 at 6:48 PM, Neha Narkhede <
> neha.narkhede@gmail.com>
> > > wrote:
> > >
> > > > I see, that makes sense. Thanks a lot for clarifying!
> > > >
> > > > -Neha
> > > >
> > > >
> > > > On Fri, Mar 21, 2014 at 11:01 AM, Bae, Jae Hyeon <metacret@gmail.com
> >
> > > > wrote:
> > > >
> > > > > Let me clarify the situation. I forgot to mention that my case
> might
> > > not
> > > > be
> > > > > general one because Netflix is using Apache Curator as the main
> > > zookeeper
> > > > > client and ZkClient in Kafka should be bridged to Apache Curator,
> so
> > > the
> > > > > behavior I have seen might not be general one.
> > > > >
> > > > > Kafka's ZKSessionExpireListener.handleNewSession() is reinstating
> all
> > > > > ephemeral nodes and watchers but handleNewSession() was not kicked
> in
> > > my
> > > > > case. So, I created Netflix internal version of ZkClient to replace
> > > > > ephemeral node creation and watcher reinstating.
> > > > >
> > > > > I have a plan to remove all external dependency from Kafka soon
> and I
> > > > might
> > > > > be able to mention more about Kafka's zk resiliency.
> > > > >
> > > > >
> > > > > On Fri, Mar 21, 2014 at 7:51 AM, Jun Rao <ju...@gmail.com> wrote:
> > > > >
> > > > > > Hi, Jae,
> > > > > >
> > > > > > My understanding is that ephemeral nodes can only be lost on ZK
> > > session
> > > > > > expiration. Are you saying that there are other cases when
> > ephemeral
> > > > > nodes
> > > > > > can be lost?
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > >
> > > > > > On Thu, Mar 20, 2014 at 9:52 PM, Bae, Jae Hyeon <
> > metacret@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > This issue is zookeeper resiliency.
> > > > > > >
> > > > > > > What I have done is, ephemeral node creation is replaced by
> > Apache
> > > > > > > Curator's PersistentEphemeralNode recipe, to reinstate
> ephemeral
> > > > nodes
> > > > > > > after zookeeper blip. Also, all watchers also should be
> > reinstated.
> > > > > Kafka
> > > > > > > internally only handles session expired event but there are
> many
> > > bad
> > > > > > > situations of zookeepers where Kafka can go haywire.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Thu, Mar 20, 2014 at 9:23 PM, Neha Narkhede <
> > > > > neha.narkhede@gmail.com
> > > > > > > >wrote:
> > > > > > >
> > > > > > > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > > > > > > >
> > > > > > > > 2.  Has anyone seen issues like this before?  Where zookeeper
> > > > > > connections
> > > > > > > > are flaky enough to cause leader elections?
> > > > > > > >
> > > > > > > > It means zookeeper expired the session. The most common
> reason
> > > for
> > > > > this
> > > > > > > is
> > > > > > > > client side GC (in your case, client is the Kafka broker)
> > though
> > > if
> > > > > > your
> > > > > > > > zookeeper setup is flaky, this might even point to a server
> > side
> > > > I/O
> > > > > > > > bottleneck or GC. Could you inspect the broker and
> zookeeper's
> > gc
> > > > > logs?
> > > > > > > >
> > > > > > > >
> > > > > > > > On Thu, Mar 20, 2014 at 3:26 PM, Andrew Otto <
> > otto@wikimedia.org
> > > >
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hm, just saw something a little fishy.
> > > > > > > > >
> > > > > > > > > (In the following logs, analytics1021 (id 21) and
> > analytics1022
> > > > (id
> > > > > > 22)
> > > > > > > > > are Brokers and analytics1023,analytics1024,analytics1025
> are
> > > > > > > > Zookeepers.)
> > > > > > > > >
> > > > > > > > > At 2014-03-20 21:12:26, analytics1021 lost its connection
> to
> > > > > > zookeeper.
> > > > > > > > It
> > > > > > > > > reconnected to analytics1023, but as it was doing so, it
> > > logged a
> > > > > > > > > 'zookeeper state changed (Expired)' message, which
> apparently
> > > > > caused
> > > > > > > the
> > > > > > > > > newly opened socket to close again.  Am I reading that
> right?
> > > > > > > > >
> > > > > > > > > In the time that it took analytics1021 to finally
> > re-establish
> > > a
> > > > > > > > zookeeper
> > > > > > > > > connection, analytics1022 had become the leader for all
> > > > partitions.
> > > > > >  I
> > > > > > > > can
> > > > > > > > > rebalance the leaders, but I'd prefer if this didn't happen
> > in
> > > > the
> > > > > > > first
> > > > > > > > > place.
> > > > > > > > >
> > > > > > > > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > > > > > > > > 2.  Has anyone seen issues like this before?  Where
> zookeeper
> > > > > > > connections
> > > > > > > > > are flaky enough to cause leader elections?
> > > > > > > > >
> > > > > > > > > Thanks!
> > > > > > > > > -Andrew Otto
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > [2014-03-20 21:12:26,427] 685698013
> > > > > > > > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > > > > > > > org.apache.zookeeper.ClientCnxn - Client session timed out,
> > > have
> > > > > not
> > > > > > > > heard
> > > > > > > > > from server in 10067ms for sessionid 0x1425d903a3f1f89,
> > closing
> > > > > > socket
> > > > > > > > > connection and attempting reconnect
> > > > > > > > > [2014-03-20 21:12:26,528] 685698114 [main-EventThread] INFO
> > > > > > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> > > > > (Disconnected)
> > > > > > > > > [2014-03-20 21:12:27,252] 685698838
> > > > > > > > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > > > > > > > org.apache.zookeeper.ClientCnxn - Opening socket connection
> > to
> > > > > server
> > > > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > > > > > > > [2014-03-20 21:12:27,271] 685698857
> > > > > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > > > > org.apache.zookeeper.ClientCnxn - Socket connection
> > established
> > > > to
> > > > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating
> > session
> > > > > > > > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > > > > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> > > (Expired)
> > > > > > > > > [2014-03-20 21:12:27,273] 685698859
> > > > > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > > > > org.apache.zookeeper.ClientCnxn - Unable to reconnect to
> > > > ZooKeeper
> > > > > > > > service,
> > > > > > > > > session 0x1425d903a3f1f89 has expired, closing socket
> > > connection
> > > > > > > > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > > > > > > > org.apache.zookeeper.ZooKeeper - Initiating client
> > connection,
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> connectString=analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad
> > > > > > > > > sessionTimeout=6000
> > > watcher=org.I0Itec.zkclient.ZkClient@63203b59
> > > > > > > > > [2014-03-20 21:12:27,290] 685698876
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > > > > INFO kafka.server.KafkaZooKeeper - re-registering broker
> info
> > > in
> > > > ZK
> > > > > > for
> > > > > > > > > broker 21
> > > > > > > > > [2014-03-20 21:12:27,290] 685698876 [main-EventThread] INFO
> > > > > > > > > org.apache.zookeeper.ClientCnxn - EventThread shut down
> > > > > > > > > [2014-03-20 21:12:27,290] 685698876 [main-SendThread()]
> INFO
> > > > > > > > > org.apache.zookeeper.ClientCnxn - Opening socket connection
> > to
> > > > > server
> > > > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > > > > > > > [2014-03-20 21:12:27,292] 685698878
> > > > > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > > > > org.apache.zookeeper.ClientCnxn - Socket connection
> > established
> > > > to
> > > > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating
> > session
> > > > > > > > > [2014-03-20 21:12:27,313] 685698899
> > > > > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > > > > org.apache.zookeeper.ClientCnxn - Session establishment
> > > complete
> > > > on
> > > > > > > > server
> > > > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, sessionid =
> > > > > > > > > 0xff429b4201dd2352, negotiated timeout = 6000
> > > > > > > > > [2014-03-20 21:12:27,314] 685698900 [main-EventThread] INFO
> > > > > > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> > > > > > (SyncConnected)
> > > > > > > > > [2014-03-20 21:12:27,324] 685698910
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > > > > INFO kafka.server.KafkaZooKeeper - done re-registering
> broker
> > > > > > > > > [2014-03-20 21:12:27,324] 685698910
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > > > > INFO kafka.server.KafkaZooKeeper - Subscribing to
> > > /brokers/topics
> > > > > > path
> > > > > > > to
> > > > > > > > > watch for new topics
> > > > > > > > > [2014-03-20 21:12:27,370] 685698956
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > > > > INFO
> > kafka.server.ZookeeperLeaderElector$LeaderChangeListener -
> > > > New
> > > > > > > > leader
> > > > > > > > > is 22
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Zookeeper reconnect failed due to 'state changed (Expired)'

Posted by Andrew Otto <ot...@wikimedia.org>.
This happened again to me this weekend.  I've done some sleuthing, and
definitely can see some crazy paging stats when this lock up happens.  For
the curious, more info can be found here:
https://bugzilla.wikimedia.org/show_bug.cgi?id=69667.  I had tuned
dirty_expire_centisecs from 30 seconds to 10, but this still happened
again.  I'll continue to troubleshoot and tune.  This slow going because it
is not regularly reproducible.  I have to make a single change and then
wait a week or two for the timeout to occur.

Here's a related question.  When the timeout happens, we lose some
messages.  Our producer is varnishkafka, which uses the librdkafka producer
client.  librdkafka keeps track of produce errors.  We
have kafka.topic.request.required.acks = 1.  According to librdkafka, all
messages sent have been ACKed by the leader of the partition to which the
messages are sent.  Also, when we lose messages due to this timeout, the
broker that times out is always the controller.  When it attempts to
reconnect to Zookeeper, we see:

  INFO  kafka.utils.ZkUtils$  - conflict in /controller data:
{"version":1,"brokerid":21,"timestamp":"1411879981756"} stored data:
{"version":1,"brokerid":22,"timestamp":"1407187809296"}

In the case when a controller drops out of the ISR for a few seconds, is it
possible for this confused broker to drop ACKed messages?



On Thu, Jul 3, 2014 at 12:48 AM, Jun Rao <ju...@gmail.com> wrote:

> Are you on Linux? We have seen this pattern (user/sys time low and real
> time high in GC time) before. In our case, the problem was due to disk
> I/Os. When there are lots of dirty pages (in our case, this is caused by
> log4j logging), Linux can draft user threads (in this case GC threads) to
> flush the dirty pages. So, all those time in real was spent on disk I/Os,
> rather than real GCs. The fix is to tune dirty_expire_centisecs and
> dirty_writeback_centisecs
> to flush dirty pages more frequently to avoid such drafting.
>
> Thanks,
>
> Jun
>
>
> On Wed, Jul 2, 2014 at 1:32 PM, Andrew Otto <ot...@wikimedia.org> wrote:
>
> > Hi again!
> >
> > I've been having this issue consistently since I first started this
> thread,
> > but it was happening infrequently enough for me to brush it aside and
> just
> > run an election to rebalance brokers.
> >
> > I recently expanded (and reinstalled) our Kafka cluster so that it now
> has
> > 4 brokers with a default replication factor of 3 for each partition.  I
> > also switched over to the G1GC as recommended here:
> > https://kafka.apache.org/081/ops.html (even though we are still running
> > Kafka 0.8.0; we hope to upgrade soon).
> >
> > Now, only one of the 4 brokers (analytics1021, the same problem broker we
> > saw before) gets its ZK connection expired even more frequently.
> >  Previously it was less than once a week, now I am seeing this happen
> > multiple times a day.
> >
> > I've posted all the relevant logs from a recent event here:
> > https://gist.github.com/ottomata/e42480446c627ea0af22
> >
> > This includes the GC log on the offending Kafka broker during the time
> this
> > happened.  I am pretty green when it comes to GC tuning, but I do see
> this
> > interesting stat:
> >
> >
> >  [Times: user=0.14 sys=0.00, real=11.47 secs]
> >
> > Did Kafka's JVM really just take 11.47 secs to do a GC there?  I'm
> > probably missing something, but I don't see which part of that real
> > time summary makes up the bulk of that GC time
> >
> > This is strange, riight?  This broker is identically configured to all
> > its peers, and should be handling on average the exact same amount and
> > type of traffic.  Anyone have any advice?
> >
> > Thanks!
> > -Andrew Otto
> >
> >
> >
> >
> > On Fri, Mar 21, 2014 at 6:48 PM, Neha Narkhede <ne...@gmail.com>
> > wrote:
> >
> > > I see, that makes sense. Thanks a lot for clarifying!
> > >
> > > -Neha
> > >
> > >
> > > On Fri, Mar 21, 2014 at 11:01 AM, Bae, Jae Hyeon <me...@gmail.com>
> > > wrote:
> > >
> > > > Let me clarify the situation. I forgot to mention that my case might
> > not
> > > be
> > > > general one because Netflix is using Apache Curator as the main
> > zookeeper
> > > > client and ZkClient in Kafka should be bridged to Apache Curator, so
> > the
> > > > behavior I have seen might not be general one.
> > > >
> > > > Kafka's ZKSessionExpireListener.handleNewSession() is reinstating all
> > > > ephemeral nodes and watchers but handleNewSession() was not kicked in
> > my
> > > > case. So, I created Netflix internal version of ZkClient to replace
> > > > ephemeral node creation and watcher reinstating.
> > > >
> > > > I have a plan to remove all external dependency from Kafka soon and I
> > > might
> > > > be able to mention more about Kafka's zk resiliency.
> > > >
> > > >
> > > > On Fri, Mar 21, 2014 at 7:51 AM, Jun Rao <ju...@gmail.com> wrote:
> > > >
> > > > > Hi, Jae,
> > > > >
> > > > > My understanding is that ephemeral nodes can only be lost on ZK
> > session
> > > > > expiration. Are you saying that there are other cases when
> ephemeral
> > > > nodes
> > > > > can be lost?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > > On Thu, Mar 20, 2014 at 9:52 PM, Bae, Jae Hyeon <
> metacret@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > This issue is zookeeper resiliency.
> > > > > >
> > > > > > What I have done is, ephemeral node creation is replaced by
> Apache
> > > > > > Curator's PersistentEphemeralNode recipe, to reinstate ephemeral
> > > nodes
> > > > > > after zookeeper blip. Also, all watchers also should be
> reinstated.
> > > > Kafka
> > > > > > internally only handles session expired event but there are many
> > bad
> > > > > > situations of zookeepers where Kafka can go haywire.
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Thu, Mar 20, 2014 at 9:23 PM, Neha Narkhede <
> > > > neha.narkhede@gmail.com
> > > > > > >wrote:
> > > > > >
> > > > > > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > > > > > >
> > > > > > > 2.  Has anyone seen issues like this before?  Where zookeeper
> > > > > connections
> > > > > > > are flaky enough to cause leader elections?
> > > > > > >
> > > > > > > It means zookeeper expired the session. The most common reason
> > for
> > > > this
> > > > > > is
> > > > > > > client side GC (in your case, client is the Kafka broker)
> though
> > if
> > > > > your
> > > > > > > zookeeper setup is flaky, this might even point to a server
> side
> > > I/O
> > > > > > > bottleneck or GC. Could you inspect the broker and zookeeper's
> gc
> > > > logs?
> > > > > > >
> > > > > > >
> > > > > > > On Thu, Mar 20, 2014 at 3:26 PM, Andrew Otto <
> otto@wikimedia.org
> > >
> > > > > wrote:
> > > > > > >
> > > > > > > > Hm, just saw something a little fishy.
> > > > > > > >
> > > > > > > > (In the following logs, analytics1021 (id 21) and
> analytics1022
> > > (id
> > > > > 22)
> > > > > > > > are Brokers and analytics1023,analytics1024,analytics1025 are
> > > > > > > Zookeepers.)
> > > > > > > >
> > > > > > > > At 2014-03-20 21:12:26, analytics1021 lost its connection to
> > > > > zookeeper.
> > > > > > > It
> > > > > > > > reconnected to analytics1023, but as it was doing so, it
> > logged a
> > > > > > > > 'zookeeper state changed (Expired)' message, which apparently
> > > > caused
> > > > > > the
> > > > > > > > newly opened socket to close again.  Am I reading that right?
> > > > > > > >
> > > > > > > > In the time that it took analytics1021 to finally
> re-establish
> > a
> > > > > > > zookeeper
> > > > > > > > connection, analytics1022 had become the leader for all
> > > partitions.
> > > > >  I
> > > > > > > can
> > > > > > > > rebalance the leaders, but I'd prefer if this didn't happen
> in
> > > the
> > > > > > first
> > > > > > > > place.
> > > > > > > >
> > > > > > > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > > > > > > > 2.  Has anyone seen issues like this before?  Where zookeeper
> > > > > > connections
> > > > > > > > are flaky enough to cause leader elections?
> > > > > > > >
> > > > > > > > Thanks!
> > > > > > > > -Andrew Otto
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > [2014-03-20 21:12:26,427] 685698013
> > > > > > > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > > > > > > org.apache.zookeeper.ClientCnxn - Client session timed out,
> > have
> > > > not
> > > > > > > heard
> > > > > > > > from server in 10067ms for sessionid 0x1425d903a3f1f89,
> closing
> > > > > socket
> > > > > > > > connection and attempting reconnect
> > > > > > > > [2014-03-20 21:12:26,528] 685698114 [main-EventThread] INFO
> > > > > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> > > > (Disconnected)
> > > > > > > > [2014-03-20 21:12:27,252] 685698838
> > > > > > > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > > > > > > org.apache.zookeeper.ClientCnxn - Opening socket connection
> to
> > > > server
> > > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > > > > > > [2014-03-20 21:12:27,271] 685698857
> > > > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > > > org.apache.zookeeper.ClientCnxn - Socket connection
> established
> > > to
> > > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating
> session
> > > > > > > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > > > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> > (Expired)
> > > > > > > > [2014-03-20 21:12:27,273] 685698859
> > > > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > > > org.apache.zookeeper.ClientCnxn - Unable to reconnect to
> > > ZooKeeper
> > > > > > > service,
> > > > > > > > session 0x1425d903a3f1f89 has expired, closing socket
> > connection
> > > > > > > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > > > > > > org.apache.zookeeper.ZooKeeper - Initiating client
> connection,
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> connectString=analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad
> > > > > > > > sessionTimeout=6000
> > watcher=org.I0Itec.zkclient.ZkClient@63203b59
> > > > > > > > [2014-03-20 21:12:27,290] 685698876
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > > > INFO kafka.server.KafkaZooKeeper - re-registering broker info
> > in
> > > ZK
> > > > > for
> > > > > > > > broker 21
> > > > > > > > [2014-03-20 21:12:27,290] 685698876 [main-EventThread] INFO
> > > > > > > > org.apache.zookeeper.ClientCnxn - EventThread shut down
> > > > > > > > [2014-03-20 21:12:27,290] 685698876 [main-SendThread()] INFO
> > > > > > > > org.apache.zookeeper.ClientCnxn - Opening socket connection
> to
> > > > server
> > > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > > > > > > [2014-03-20 21:12:27,292] 685698878
> > > > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > > > org.apache.zookeeper.ClientCnxn - Socket connection
> established
> > > to
> > > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating
> session
> > > > > > > > [2014-03-20 21:12:27,313] 685698899
> > > > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > > > org.apache.zookeeper.ClientCnxn - Session establishment
> > complete
> > > on
> > > > > > > server
> > > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, sessionid =
> > > > > > > > 0xff429b4201dd2352, negotiated timeout = 6000
> > > > > > > > [2014-03-20 21:12:27,314] 685698900 [main-EventThread] INFO
> > > > > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> > > > > (SyncConnected)
> > > > > > > > [2014-03-20 21:12:27,324] 685698910
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > > > INFO kafka.server.KafkaZooKeeper - done re-registering broker
> > > > > > > > [2014-03-20 21:12:27,324] 685698910
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > > > INFO kafka.server.KafkaZooKeeper - Subscribing to
> > /brokers/topics
> > > > > path
> > > > > > to
> > > > > > > > watch for new topics
> > > > > > > > [2014-03-20 21:12:27,370] 685698956
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > > > INFO
> kafka.server.ZookeeperLeaderElector$LeaderChangeListener -
> > > New
> > > > > > > leader
> > > > > > > > is 22
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Zookeeper reconnect failed due to 'state changed (Expired)'

Posted by Jun Rao <ju...@gmail.com>.
Are you on Linux? We have seen this pattern (user/sys time low and real
time high in GC time) before. In our case, the problem was due to disk
I/Os. When there are lots of dirty pages (in our case, this is caused by
log4j logging), Linux can draft user threads (in this case GC threads) to
flush the dirty pages. So, all those time in real was spent on disk I/Os,
rather than real GCs. The fix is to tune dirty_expire_centisecs and
dirty_writeback_centisecs
to flush dirty pages more frequently to avoid such drafting.

Thanks,

Jun


On Wed, Jul 2, 2014 at 1:32 PM, Andrew Otto <ot...@wikimedia.org> wrote:

> Hi again!
>
> I've been having this issue consistently since I first started this thread,
> but it was happening infrequently enough for me to brush it aside and just
> run an election to rebalance brokers.
>
> I recently expanded (and reinstalled) our Kafka cluster so that it now has
> 4 brokers with a default replication factor of 3 for each partition.  I
> also switched over to the G1GC as recommended here:
> https://kafka.apache.org/081/ops.html (even though we are still running
> Kafka 0.8.0; we hope to upgrade soon).
>
> Now, only one of the 4 brokers (analytics1021, the same problem broker we
> saw before) gets its ZK connection expired even more frequently.
>  Previously it was less than once a week, now I am seeing this happen
> multiple times a day.
>
> I've posted all the relevant logs from a recent event here:
> https://gist.github.com/ottomata/e42480446c627ea0af22
>
> This includes the GC log on the offending Kafka broker during the time this
> happened.  I am pretty green when it comes to GC tuning, but I do see this
> interesting stat:
>
>
>  [Times: user=0.14 sys=0.00, real=11.47 secs]
>
> Did Kafka's JVM really just take 11.47 secs to do a GC there?  I'm
> probably missing something, but I don't see which part of that real
> time summary makes up the bulk of that GC time
>
> This is strange, riight?  This broker is identically configured to all
> its peers, and should be handling on average the exact same amount and
> type of traffic.  Anyone have any advice?
>
> Thanks!
> -Andrew Otto
>
>
>
>
> On Fri, Mar 21, 2014 at 6:48 PM, Neha Narkhede <ne...@gmail.com>
> wrote:
>
> > I see, that makes sense. Thanks a lot for clarifying!
> >
> > -Neha
> >
> >
> > On Fri, Mar 21, 2014 at 11:01 AM, Bae, Jae Hyeon <me...@gmail.com>
> > wrote:
> >
> > > Let me clarify the situation. I forgot to mention that my case might
> not
> > be
> > > general one because Netflix is using Apache Curator as the main
> zookeeper
> > > client and ZkClient in Kafka should be bridged to Apache Curator, so
> the
> > > behavior I have seen might not be general one.
> > >
> > > Kafka's ZKSessionExpireListener.handleNewSession() is reinstating all
> > > ephemeral nodes and watchers but handleNewSession() was not kicked in
> my
> > > case. So, I created Netflix internal version of ZkClient to replace
> > > ephemeral node creation and watcher reinstating.
> > >
> > > I have a plan to remove all external dependency from Kafka soon and I
> > might
> > > be able to mention more about Kafka's zk resiliency.
> > >
> > >
> > > On Fri, Mar 21, 2014 at 7:51 AM, Jun Rao <ju...@gmail.com> wrote:
> > >
> > > > Hi, Jae,
> > > >
> > > > My understanding is that ephemeral nodes can only be lost on ZK
> session
> > > > expiration. Are you saying that there are other cases when ephemeral
> > > nodes
> > > > can be lost?
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Thu, Mar 20, 2014 at 9:52 PM, Bae, Jae Hyeon <me...@gmail.com>
> > > > wrote:
> > > >
> > > > > This issue is zookeeper resiliency.
> > > > >
> > > > > What I have done is, ephemeral node creation is replaced by Apache
> > > > > Curator's PersistentEphemeralNode recipe, to reinstate ephemeral
> > nodes
> > > > > after zookeeper blip. Also, all watchers also should be reinstated.
> > > Kafka
> > > > > internally only handles session expired event but there are many
> bad
> > > > > situations of zookeepers where Kafka can go haywire.
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On Thu, Mar 20, 2014 at 9:23 PM, Neha Narkhede <
> > > neha.narkhede@gmail.com
> > > > > >wrote:
> > > > >
> > > > > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > > > > >
> > > > > > 2.  Has anyone seen issues like this before?  Where zookeeper
> > > > connections
> > > > > > are flaky enough to cause leader elections?
> > > > > >
> > > > > > It means zookeeper expired the session. The most common reason
> for
> > > this
> > > > > is
> > > > > > client side GC (in your case, client is the Kafka broker) though
> if
> > > > your
> > > > > > zookeeper setup is flaky, this might even point to a server side
> > I/O
> > > > > > bottleneck or GC. Could you inspect the broker and zookeeper's gc
> > > logs?
> > > > > >
> > > > > >
> > > > > > On Thu, Mar 20, 2014 at 3:26 PM, Andrew Otto <otto@wikimedia.org
> >
> > > > wrote:
> > > > > >
> > > > > > > Hm, just saw something a little fishy.
> > > > > > >
> > > > > > > (In the following logs, analytics1021 (id 21) and analytics1022
> > (id
> > > > 22)
> > > > > > > are Brokers and analytics1023,analytics1024,analytics1025 are
> > > > > > Zookeepers.)
> > > > > > >
> > > > > > > At 2014-03-20 21:12:26, analytics1021 lost its connection to
> > > > zookeeper.
> > > > > > It
> > > > > > > reconnected to analytics1023, but as it was doing so, it
> logged a
> > > > > > > 'zookeeper state changed (Expired)' message, which apparently
> > > caused
> > > > > the
> > > > > > > newly opened socket to close again.  Am I reading that right?
> > > > > > >
> > > > > > > In the time that it took analytics1021 to finally re-establish
> a
> > > > > > zookeeper
> > > > > > > connection, analytics1022 had become the leader for all
> > partitions.
> > > >  I
> > > > > > can
> > > > > > > rebalance the leaders, but I'd prefer if this didn't happen in
> > the
> > > > > first
> > > > > > > place.
> > > > > > >
> > > > > > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > > > > > > 2.  Has anyone seen issues like this before?  Where zookeeper
> > > > > connections
> > > > > > > are flaky enough to cause leader elections?
> > > > > > >
> > > > > > > Thanks!
> > > > > > > -Andrew Otto
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > [2014-03-20 21:12:26,427] 685698013
> > > > > > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > > > > > org.apache.zookeeper.ClientCnxn - Client session timed out,
> have
> > > not
> > > > > > heard
> > > > > > > from server in 10067ms for sessionid 0x1425d903a3f1f89, closing
> > > > socket
> > > > > > > connection and attempting reconnect
> > > > > > > [2014-03-20 21:12:26,528] 685698114 [main-EventThread] INFO
> > > > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> > > (Disconnected)
> > > > > > > [2014-03-20 21:12:27,252] 685698838
> > > > > > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > > > > > org.apache.zookeeper.ClientCnxn - Opening socket connection to
> > > server
> > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > > > > > [2014-03-20 21:12:27,271] 685698857
> > > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > > org.apache.zookeeper.ClientCnxn - Socket connection established
> > to
> > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> > > > > > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> (Expired)
> > > > > > > [2014-03-20 21:12:27,273] 685698859
> > > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > > org.apache.zookeeper.ClientCnxn - Unable to reconnect to
> > ZooKeeper
> > > > > > service,
> > > > > > > session 0x1425d903a3f1f89 has expired, closing socket
> connection
> > > > > > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > > > > > org.apache.zookeeper.ZooKeeper - Initiating client connection,
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> connectString=analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad
> > > > > > > sessionTimeout=6000
> watcher=org.I0Itec.zkclient.ZkClient@63203b59
> > > > > > > [2014-03-20 21:12:27,290] 685698876
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > > INFO kafka.server.KafkaZooKeeper - re-registering broker info
> in
> > ZK
> > > > for
> > > > > > > broker 21
> > > > > > > [2014-03-20 21:12:27,290] 685698876 [main-EventThread] INFO
> > > > > > > org.apache.zookeeper.ClientCnxn - EventThread shut down
> > > > > > > [2014-03-20 21:12:27,290] 685698876 [main-SendThread()] INFO
> > > > > > > org.apache.zookeeper.ClientCnxn - Opening socket connection to
> > > server
> > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > > > > > [2014-03-20 21:12:27,292] 685698878
> > > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > > org.apache.zookeeper.ClientCnxn - Socket connection established
> > to
> > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> > > > > > > [2014-03-20 21:12:27,313] 685698899
> > > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > > org.apache.zookeeper.ClientCnxn - Session establishment
> complete
> > on
> > > > > > server
> > > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, sessionid =
> > > > > > > 0xff429b4201dd2352, negotiated timeout = 6000
> > > > > > > [2014-03-20 21:12:27,314] 685698900 [main-EventThread] INFO
> > > > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> > > > (SyncConnected)
> > > > > > > [2014-03-20 21:12:27,324] 685698910
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > > INFO kafka.server.KafkaZooKeeper - done re-registering broker
> > > > > > > [2014-03-20 21:12:27,324] 685698910
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > > INFO kafka.server.KafkaZooKeeper - Subscribing to
> /brokers/topics
> > > > path
> > > > > to
> > > > > > > watch for new topics
> > > > > > > [2014-03-20 21:12:27,370] 685698956
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > > INFO kafka.server.ZookeeperLeaderElector$LeaderChangeListener -
> > New
> > > > > > leader
> > > > > > > is 22
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Zookeeper reconnect failed due to 'state changed (Expired)'

Posted by Andrew Otto <ot...@wikimedia.org>.
Hi again!

I've been having this issue consistently since I first started this thread,
but it was happening infrequently enough for me to brush it aside and just
run an election to rebalance brokers.

I recently expanded (and reinstalled) our Kafka cluster so that it now has
4 brokers with a default replication factor of 3 for each partition.  I
also switched over to the G1GC as recommended here:
https://kafka.apache.org/081/ops.html (even though we are still running
Kafka 0.8.0; we hope to upgrade soon).

Now, only one of the 4 brokers (analytics1021, the same problem broker we
saw before) gets its ZK connection expired even more frequently.
 Previously it was less than once a week, now I am seeing this happen
multiple times a day.

I've posted all the relevant logs from a recent event here:
https://gist.github.com/ottomata/e42480446c627ea0af22

This includes the GC log on the offending Kafka broker during the time this
happened.  I am pretty green when it comes to GC tuning, but I do see this
interesting stat:


 [Times: user=0.14 sys=0.00, real=11.47 secs]

Did Kafka's JVM really just take 11.47 secs to do a GC there?  I'm
probably missing something, but I don't see which part of that real
time summary makes up the bulk of that GC time

This is strange, riight?  This broker is identically configured to all
its peers, and should be handling on average the exact same amount and
type of traffic.  Anyone have any advice?

Thanks!
-Andrew Otto




On Fri, Mar 21, 2014 at 6:48 PM, Neha Narkhede <ne...@gmail.com>
wrote:

> I see, that makes sense. Thanks a lot for clarifying!
>
> -Neha
>
>
> On Fri, Mar 21, 2014 at 11:01 AM, Bae, Jae Hyeon <me...@gmail.com>
> wrote:
>
> > Let me clarify the situation. I forgot to mention that my case might not
> be
> > general one because Netflix is using Apache Curator as the main zookeeper
> > client and ZkClient in Kafka should be bridged to Apache Curator, so the
> > behavior I have seen might not be general one.
> >
> > Kafka's ZKSessionExpireListener.handleNewSession() is reinstating all
> > ephemeral nodes and watchers but handleNewSession() was not kicked in my
> > case. So, I created Netflix internal version of ZkClient to replace
> > ephemeral node creation and watcher reinstating.
> >
> > I have a plan to remove all external dependency from Kafka soon and I
> might
> > be able to mention more about Kafka's zk resiliency.
> >
> >
> > On Fri, Mar 21, 2014 at 7:51 AM, Jun Rao <ju...@gmail.com> wrote:
> >
> > > Hi, Jae,
> > >
> > > My understanding is that ephemeral nodes can only be lost on ZK session
> > > expiration. Are you saying that there are other cases when ephemeral
> > nodes
> > > can be lost?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Thu, Mar 20, 2014 at 9:52 PM, Bae, Jae Hyeon <me...@gmail.com>
> > > wrote:
> > >
> > > > This issue is zookeeper resiliency.
> > > >
> > > > What I have done is, ephemeral node creation is replaced by Apache
> > > > Curator's PersistentEphemeralNode recipe, to reinstate ephemeral
> nodes
> > > > after zookeeper blip. Also, all watchers also should be reinstated.
> > Kafka
> > > > internally only handles session expired event but there are many bad
> > > > situations of zookeepers where Kafka can go haywire.
> > > >
> > > >
> > > >
> > > >
> > > > On Thu, Mar 20, 2014 at 9:23 PM, Neha Narkhede <
> > neha.narkhede@gmail.com
> > > > >wrote:
> > > >
> > > > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > > > >
> > > > > 2.  Has anyone seen issues like this before?  Where zookeeper
> > > connections
> > > > > are flaky enough to cause leader elections?
> > > > >
> > > > > It means zookeeper expired the session. The most common reason for
> > this
> > > > is
> > > > > client side GC (in your case, client is the Kafka broker) though if
> > > your
> > > > > zookeeper setup is flaky, this might even point to a server side
> I/O
> > > > > bottleneck or GC. Could you inspect the broker and zookeeper's gc
> > logs?
> > > > >
> > > > >
> > > > > On Thu, Mar 20, 2014 at 3:26 PM, Andrew Otto <ot...@wikimedia.org>
> > > wrote:
> > > > >
> > > > > > Hm, just saw something a little fishy.
> > > > > >
> > > > > > (In the following logs, analytics1021 (id 21) and analytics1022
> (id
> > > 22)
> > > > > > are Brokers and analytics1023,analytics1024,analytics1025 are
> > > > > Zookeepers.)
> > > > > >
> > > > > > At 2014-03-20 21:12:26, analytics1021 lost its connection to
> > > zookeeper.
> > > > > It
> > > > > > reconnected to analytics1023, but as it was doing so, it logged a
> > > > > > 'zookeeper state changed (Expired)' message, which apparently
> > caused
> > > > the
> > > > > > newly opened socket to close again.  Am I reading that right?
> > > > > >
> > > > > > In the time that it took analytics1021 to finally re-establish a
> > > > > zookeeper
> > > > > > connection, analytics1022 had become the leader for all
> partitions.
> > >  I
> > > > > can
> > > > > > rebalance the leaders, but I'd prefer if this didn't happen in
> the
> > > > first
> > > > > > place.
> > > > > >
> > > > > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > > > > > 2.  Has anyone seen issues like this before?  Where zookeeper
> > > > connections
> > > > > > are flaky enough to cause leader elections?
> > > > > >
> > > > > > Thanks!
> > > > > > -Andrew Otto
> > > > > >
> > > > > >
> > > > > >
> > > > > > [2014-03-20 21:12:26,427] 685698013
> > > > > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > > > > org.apache.zookeeper.ClientCnxn - Client session timed out, have
> > not
> > > > > heard
> > > > > > from server in 10067ms for sessionid 0x1425d903a3f1f89, closing
> > > socket
> > > > > > connection and attempting reconnect
> > > > > > [2014-03-20 21:12:26,528] 685698114 [main-EventThread] INFO
> > > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> > (Disconnected)
> > > > > > [2014-03-20 21:12:27,252] 685698838
> > > > > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > > > > org.apache.zookeeper.ClientCnxn - Opening socket connection to
> > server
> > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > > > > [2014-03-20 21:12:27,271] 685698857
> > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > org.apache.zookeeper.ClientCnxn - Socket connection established
> to
> > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> > > > > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed (Expired)
> > > > > > [2014-03-20 21:12:27,273] 685698859
> > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > org.apache.zookeeper.ClientCnxn - Unable to reconnect to
> ZooKeeper
> > > > > service,
> > > > > > session 0x1425d903a3f1f89 has expired, closing socket connection
> > > > > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > > > > org.apache.zookeeper.ZooKeeper - Initiating client connection,
> > > > > >
> > > > >
> > > >
> > >
> >
> connectString=analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad
> > > > > > sessionTimeout=6000 watcher=org.I0Itec.zkclient.ZkClient@63203b59
> > > > > > [2014-03-20 21:12:27,290] 685698876
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > INFO kafka.server.KafkaZooKeeper - re-registering broker info in
> ZK
> > > for
> > > > > > broker 21
> > > > > > [2014-03-20 21:12:27,290] 685698876 [main-EventThread] INFO
> > > > > > org.apache.zookeeper.ClientCnxn - EventThread shut down
> > > > > > [2014-03-20 21:12:27,290] 685698876 [main-SendThread()] INFO
> > > > > > org.apache.zookeeper.ClientCnxn - Opening socket connection to
> > server
> > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > > > > [2014-03-20 21:12:27,292] 685698878
> > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > org.apache.zookeeper.ClientCnxn - Socket connection established
> to
> > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> > > > > > [2014-03-20 21:12:27,313] 685698899
> > > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > > org.apache.zookeeper.ClientCnxn - Session establishment complete
> on
> > > > > server
> > > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, sessionid =
> > > > > > 0xff429b4201dd2352, negotiated timeout = 6000
> > > > > > [2014-03-20 21:12:27,314] 685698900 [main-EventThread] INFO
> > > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> > > (SyncConnected)
> > > > > > [2014-03-20 21:12:27,324] 685698910
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > INFO kafka.server.KafkaZooKeeper - done re-registering broker
> > > > > > [2014-03-20 21:12:27,324] 685698910
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > INFO kafka.server.KafkaZooKeeper - Subscribing to /brokers/topics
> > > path
> > > > to
> > > > > > watch for new topics
> > > > > > [2014-03-20 21:12:27,370] 685698956
> > > > > >
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > > INFO kafka.server.ZookeeperLeaderElector$LeaderChangeListener -
> New
> > > > > leader
> > > > > > is 22
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Zookeeper reconnect failed due to 'state changed (Expired)'

Posted by Neha Narkhede <ne...@gmail.com>.
I see, that makes sense. Thanks a lot for clarifying!

-Neha


On Fri, Mar 21, 2014 at 11:01 AM, Bae, Jae Hyeon <me...@gmail.com> wrote:

> Let me clarify the situation. I forgot to mention that my case might not be
> general one because Netflix is using Apache Curator as the main zookeeper
> client and ZkClient in Kafka should be bridged to Apache Curator, so the
> behavior I have seen might not be general one.
>
> Kafka's ZKSessionExpireListener.handleNewSession() is reinstating all
> ephemeral nodes and watchers but handleNewSession() was not kicked in my
> case. So, I created Netflix internal version of ZkClient to replace
> ephemeral node creation and watcher reinstating.
>
> I have a plan to remove all external dependency from Kafka soon and I might
> be able to mention more about Kafka's zk resiliency.
>
>
> On Fri, Mar 21, 2014 at 7:51 AM, Jun Rao <ju...@gmail.com> wrote:
>
> > Hi, Jae,
> >
> > My understanding is that ephemeral nodes can only be lost on ZK session
> > expiration. Are you saying that there are other cases when ephemeral
> nodes
> > can be lost?
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Thu, Mar 20, 2014 at 9:52 PM, Bae, Jae Hyeon <me...@gmail.com>
> > wrote:
> >
> > > This issue is zookeeper resiliency.
> > >
> > > What I have done is, ephemeral node creation is replaced by Apache
> > > Curator's PersistentEphemeralNode recipe, to reinstate ephemeral nodes
> > > after zookeeper blip. Also, all watchers also should be reinstated.
> Kafka
> > > internally only handles session expired event but there are many bad
> > > situations of zookeepers where Kafka can go haywire.
> > >
> > >
> > >
> > >
> > > On Thu, Mar 20, 2014 at 9:23 PM, Neha Narkhede <
> neha.narkhede@gmail.com
> > > >wrote:
> > >
> > > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > > >
> > > > 2.  Has anyone seen issues like this before?  Where zookeeper
> > connections
> > > > are flaky enough to cause leader elections?
> > > >
> > > > It means zookeeper expired the session. The most common reason for
> this
> > > is
> > > > client side GC (in your case, client is the Kafka broker) though if
> > your
> > > > zookeeper setup is flaky, this might even point to a server side I/O
> > > > bottleneck or GC. Could you inspect the broker and zookeeper's gc
> logs?
> > > >
> > > >
> > > > On Thu, Mar 20, 2014 at 3:26 PM, Andrew Otto <ot...@wikimedia.org>
> > wrote:
> > > >
> > > > > Hm, just saw something a little fishy.
> > > > >
> > > > > (In the following logs, analytics1021 (id 21) and analytics1022 (id
> > 22)
> > > > > are Brokers and analytics1023,analytics1024,analytics1025 are
> > > > Zookeepers.)
> > > > >
> > > > > At 2014-03-20 21:12:26, analytics1021 lost its connection to
> > zookeeper.
> > > > It
> > > > > reconnected to analytics1023, but as it was doing so, it logged a
> > > > > 'zookeeper state changed (Expired)' message, which apparently
> caused
> > > the
> > > > > newly opened socket to close again.  Am I reading that right?
> > > > >
> > > > > In the time that it took analytics1021 to finally re-establish a
> > > > zookeeper
> > > > > connection, analytics1022 had become the leader for all partitions.
> >  I
> > > > can
> > > > > rebalance the leaders, but I'd prefer if this didn't happen in the
> > > first
> > > > > place.
> > > > >
> > > > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > > > > 2.  Has anyone seen issues like this before?  Where zookeeper
> > > connections
> > > > > are flaky enough to cause leader elections?
> > > > >
> > > > > Thanks!
> > > > > -Andrew Otto
> > > > >
> > > > >
> > > > >
> > > > > [2014-03-20 21:12:26,427] 685698013
> > > > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > > > org.apache.zookeeper.ClientCnxn - Client session timed out, have
> not
> > > > heard
> > > > > from server in 10067ms for sessionid 0x1425d903a3f1f89, closing
> > socket
> > > > > connection and attempting reconnect
> > > > > [2014-03-20 21:12:26,528] 685698114 [main-EventThread] INFO
> > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> (Disconnected)
> > > > > [2014-03-20 21:12:27,252] 685698838
> > > > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > > > org.apache.zookeeper.ClientCnxn - Opening socket connection to
> server
> > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > > > [2014-03-20 21:12:27,271] 685698857
> > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > org.apache.zookeeper.ClientCnxn - Socket connection established to
> > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> > > > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed (Expired)
> > > > > [2014-03-20 21:12:27,273] 685698859
> > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > org.apache.zookeeper.ClientCnxn - Unable to reconnect to ZooKeeper
> > > > service,
> > > > > session 0x1425d903a3f1f89 has expired, closing socket connection
> > > > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > > > org.apache.zookeeper.ZooKeeper - Initiating client connection,
> > > > >
> > > >
> > >
> >
> connectString=analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad
> > > > > sessionTimeout=6000 watcher=org.I0Itec.zkclient.ZkClient@63203b59
> > > > > [2014-03-20 21:12:27,290] 685698876
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > INFO kafka.server.KafkaZooKeeper - re-registering broker info in ZK
> > for
> > > > > broker 21
> > > > > [2014-03-20 21:12:27,290] 685698876 [main-EventThread] INFO
> > > > > org.apache.zookeeper.ClientCnxn - EventThread shut down
> > > > > [2014-03-20 21:12:27,290] 685698876 [main-SendThread()] INFO
> > > > > org.apache.zookeeper.ClientCnxn - Opening socket connection to
> server
> > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > > > [2014-03-20 21:12:27,292] 685698878
> > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > org.apache.zookeeper.ClientCnxn - Socket connection established to
> > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> > > > > [2014-03-20 21:12:27,313] 685698899
> > > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > > org.apache.zookeeper.ClientCnxn - Session establishment complete on
> > > > server
> > > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, sessionid =
> > > > > 0xff429b4201dd2352, negotiated timeout = 6000
> > > > > [2014-03-20 21:12:27,314] 685698900 [main-EventThread] INFO
> > > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> > (SyncConnected)
> > > > > [2014-03-20 21:12:27,324] 685698910
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > INFO kafka.server.KafkaZooKeeper - done re-registering broker
> > > > > [2014-03-20 21:12:27,324] 685698910
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > INFO kafka.server.KafkaZooKeeper - Subscribing to /brokers/topics
> > path
> > > to
> > > > > watch for new topics
> > > > > [2014-03-20 21:12:27,370] 685698956
> > > > >
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > > INFO kafka.server.ZookeeperLeaderElector$LeaderChangeListener - New
> > > > leader
> > > > > is 22
> > > > >
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: Zookeeper reconnect failed due to 'state changed (Expired)'

Posted by "Bae, Jae Hyeon" <me...@gmail.com>.
Let me clarify the situation. I forgot to mention that my case might not be
general one because Netflix is using Apache Curator as the main zookeeper
client and ZkClient in Kafka should be bridged to Apache Curator, so the
behavior I have seen might not be general one.

Kafka's ZKSessionExpireListener.handleNewSession() is reinstating all
ephemeral nodes and watchers but handleNewSession() was not kicked in my
case. So, I created Netflix internal version of ZkClient to replace
ephemeral node creation and watcher reinstating.

I have a plan to remove all external dependency from Kafka soon and I might
be able to mention more about Kafka's zk resiliency.


On Fri, Mar 21, 2014 at 7:51 AM, Jun Rao <ju...@gmail.com> wrote:

> Hi, Jae,
>
> My understanding is that ephemeral nodes can only be lost on ZK session
> expiration. Are you saying that there are other cases when ephemeral nodes
> can be lost?
>
> Thanks,
>
> Jun
>
>
> On Thu, Mar 20, 2014 at 9:52 PM, Bae, Jae Hyeon <me...@gmail.com>
> wrote:
>
> > This issue is zookeeper resiliency.
> >
> > What I have done is, ephemeral node creation is replaced by Apache
> > Curator's PersistentEphemeralNode recipe, to reinstate ephemeral nodes
> > after zookeeper blip. Also, all watchers also should be reinstated. Kafka
> > internally only handles session expired event but there are many bad
> > situations of zookeepers where Kafka can go haywire.
> >
> >
> >
> >
> > On Thu, Mar 20, 2014 at 9:23 PM, Neha Narkhede <neha.narkhede@gmail.com
> > >wrote:
> >
> > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > >
> > > 2.  Has anyone seen issues like this before?  Where zookeeper
> connections
> > > are flaky enough to cause leader elections?
> > >
> > > It means zookeeper expired the session. The most common reason for this
> > is
> > > client side GC (in your case, client is the Kafka broker) though if
> your
> > > zookeeper setup is flaky, this might even point to a server side I/O
> > > bottleneck or GC. Could you inspect the broker and zookeeper's gc logs?
> > >
> > >
> > > On Thu, Mar 20, 2014 at 3:26 PM, Andrew Otto <ot...@wikimedia.org>
> wrote:
> > >
> > > > Hm, just saw something a little fishy.
> > > >
> > > > (In the following logs, analytics1021 (id 21) and analytics1022 (id
> 22)
> > > > are Brokers and analytics1023,analytics1024,analytics1025 are
> > > Zookeepers.)
> > > >
> > > > At 2014-03-20 21:12:26, analytics1021 lost its connection to
> zookeeper.
> > > It
> > > > reconnected to analytics1023, but as it was doing so, it logged a
> > > > 'zookeeper state changed (Expired)' message, which apparently caused
> > the
> > > > newly opened socket to close again.  Am I reading that right?
> > > >
> > > > In the time that it took analytics1021 to finally re-establish a
> > > zookeeper
> > > > connection, analytics1022 had become the leader for all partitions.
>  I
> > > can
> > > > rebalance the leaders, but I'd prefer if this didn't happen in the
> > first
> > > > place.
> > > >
> > > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > > > 2.  Has anyone seen issues like this before?  Where zookeeper
> > connections
> > > > are flaky enough to cause leader elections?
> > > >
> > > > Thanks!
> > > > -Andrew Otto
> > > >
> > > >
> > > >
> > > > [2014-03-20 21:12:26,427] 685698013
> > > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > > org.apache.zookeeper.ClientCnxn - Client session timed out, have not
> > > heard
> > > > from server in 10067ms for sessionid 0x1425d903a3f1f89, closing
> socket
> > > > connection and attempting reconnect
> > > > [2014-03-20 21:12:26,528] 685698114 [main-EventThread] INFO
> > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed (Disconnected)
> > > > [2014-03-20 21:12:27,252] 685698838
> > > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > > org.apache.zookeeper.ClientCnxn - Opening socket connection to server
> > > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > > [2014-03-20 21:12:27,271] 685698857
> > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > org.apache.zookeeper.ClientCnxn - Socket connection established to
> > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> > > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed (Expired)
> > > > [2014-03-20 21:12:27,273] 685698859
> > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > org.apache.zookeeper.ClientCnxn - Unable to reconnect to ZooKeeper
> > > service,
> > > > session 0x1425d903a3f1f89 has expired, closing socket connection
> > > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > > org.apache.zookeeper.ZooKeeper - Initiating client connection,
> > > >
> > >
> >
> connectString=analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad
> > > > sessionTimeout=6000 watcher=org.I0Itec.zkclient.ZkClient@63203b59
> > > > [2014-03-20 21:12:27,290] 685698876
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > INFO kafka.server.KafkaZooKeeper - re-registering broker info in ZK
> for
> > > > broker 21
> > > > [2014-03-20 21:12:27,290] 685698876 [main-EventThread] INFO
> > > > org.apache.zookeeper.ClientCnxn - EventThread shut down
> > > > [2014-03-20 21:12:27,290] 685698876 [main-SendThread()] INFO
> > > > org.apache.zookeeper.ClientCnxn - Opening socket connection to server
> > > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > > [2014-03-20 21:12:27,292] 685698878
> > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > org.apache.zookeeper.ClientCnxn - Socket connection established to
> > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> > > > [2014-03-20 21:12:27,313] 685698899
> > > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > > org.apache.zookeeper.ClientCnxn - Session establishment complete on
> > > server
> > > > analytics1023.eqiad.wmnet/10.64.5.15:2181, sessionid =
> > > > 0xff429b4201dd2352, negotiated timeout = 6000
> > > > [2014-03-20 21:12:27,314] 685698900 [main-EventThread] INFO
> > > > org.I0Itec.zkclient.ZkClient - zookeeper state changed
> (SyncConnected)
> > > > [2014-03-20 21:12:27,324] 685698910
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > INFO kafka.server.KafkaZooKeeper - done re-registering broker
> > > > [2014-03-20 21:12:27,324] 685698910
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > INFO kafka.server.KafkaZooKeeper - Subscribing to /brokers/topics
> path
> > to
> > > > watch for new topics
> > > > [2014-03-20 21:12:27,370] 685698956
> > > >
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > > INFO kafka.server.ZookeeperLeaderElector$LeaderChangeListener - New
> > > leader
> > > > is 22
> > > >
> > > >
> > > >
> > >
> >
>

Re: Zookeeper reconnect failed due to 'state changed (Expired)'

Posted by Jun Rao <ju...@gmail.com>.
Hi, Jae,

My understanding is that ephemeral nodes can only be lost on ZK session
expiration. Are you saying that there are other cases when ephemeral nodes
can be lost?

Thanks,

Jun


On Thu, Mar 20, 2014 at 9:52 PM, Bae, Jae Hyeon <me...@gmail.com> wrote:

> This issue is zookeeper resiliency.
>
> What I have done is, ephemeral node creation is replaced by Apache
> Curator's PersistentEphemeralNode recipe, to reinstate ephemeral nodes
> after zookeeper blip. Also, all watchers also should be reinstated. Kafka
> internally only handles session expired event but there are many bad
> situations of zookeepers where Kafka can go haywire.
>
>
>
>
> On Thu, Mar 20, 2014 at 9:23 PM, Neha Narkhede <neha.narkhede@gmail.com
> >wrote:
>
> > 1.  What does 'zookeeper state changed (Expired)' mean?
> >
> > 2.  Has anyone seen issues like this before?  Where zookeeper connections
> > are flaky enough to cause leader elections?
> >
> > It means zookeeper expired the session. The most common reason for this
> is
> > client side GC (in your case, client is the Kafka broker) though if your
> > zookeeper setup is flaky, this might even point to a server side I/O
> > bottleneck or GC. Could you inspect the broker and zookeeper's gc logs?
> >
> >
> > On Thu, Mar 20, 2014 at 3:26 PM, Andrew Otto <ot...@wikimedia.org> wrote:
> >
> > > Hm, just saw something a little fishy.
> > >
> > > (In the following logs, analytics1021 (id 21) and analytics1022 (id 22)
> > > are Brokers and analytics1023,analytics1024,analytics1025 are
> > Zookeepers.)
> > >
> > > At 2014-03-20 21:12:26, analytics1021 lost its connection to zookeeper.
> > It
> > > reconnected to analytics1023, but as it was doing so, it logged a
> > > 'zookeeper state changed (Expired)' message, which apparently caused
> the
> > > newly opened socket to close again.  Am I reading that right?
> > >
> > > In the time that it took analytics1021 to finally re-establish a
> > zookeeper
> > > connection, analytics1022 had become the leader for all partitions.  I
> > can
> > > rebalance the leaders, but I'd prefer if this didn't happen in the
> first
> > > place.
> > >
> > > 1.  What does 'zookeeper state changed (Expired)' mean?
> > > 2.  Has anyone seen issues like this before?  Where zookeeper
> connections
> > > are flaky enough to cause leader elections?
> > >
> > > Thanks!
> > > -Andrew Otto
> > >
> > >
> > >
> > > [2014-03-20 21:12:26,427] 685698013
> > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > org.apache.zookeeper.ClientCnxn - Client session timed out, have not
> > heard
> > > from server in 10067ms for sessionid 0x1425d903a3f1f89, closing socket
> > > connection and attempting reconnect
> > > [2014-03-20 21:12:26,528] 685698114 [main-EventThread] INFO
> > > org.I0Itec.zkclient.ZkClient - zookeeper state changed (Disconnected)
> > > [2014-03-20 21:12:27,252] 685698838
> > > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > > org.apache.zookeeper.ClientCnxn - Opening socket connection to server
> > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > [2014-03-20 21:12:27,271] 685698857
> > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > org.apache.zookeeper.ClientCnxn - Socket connection established to
> > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > org.I0Itec.zkclient.ZkClient - zookeeper state changed (Expired)
> > > [2014-03-20 21:12:27,273] 685698859
> > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > org.apache.zookeeper.ClientCnxn - Unable to reconnect to ZooKeeper
> > service,
> > > session 0x1425d903a3f1f89 has expired, closing socket connection
> > > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > > org.apache.zookeeper.ZooKeeper - Initiating client connection,
> > >
> >
> connectString=analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad
> > > sessionTimeout=6000 watcher=org.I0Itec.zkclient.ZkClient@63203b59
> > > [2014-03-20 21:12:27,290] 685698876
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > INFO kafka.server.KafkaZooKeeper - re-registering broker info in ZK for
> > > broker 21
> > > [2014-03-20 21:12:27,290] 685698876 [main-EventThread] INFO
> > > org.apache.zookeeper.ClientCnxn - EventThread shut down
> > > [2014-03-20 21:12:27,290] 685698876 [main-SendThread()] INFO
> > > org.apache.zookeeper.ClientCnxn - Opening socket connection to server
> > > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > > [2014-03-20 21:12:27,292] 685698878
> > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > org.apache.zookeeper.ClientCnxn - Socket connection established to
> > > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> > > [2014-03-20 21:12:27,313] 685698899
> > > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > > org.apache.zookeeper.ClientCnxn - Session establishment complete on
> > server
> > > analytics1023.eqiad.wmnet/10.64.5.15:2181, sessionid =
> > > 0xff429b4201dd2352, negotiated timeout = 6000
> > > [2014-03-20 21:12:27,314] 685698900 [main-EventThread] INFO
> > > org.I0Itec.zkclient.ZkClient - zookeeper state changed (SyncConnected)
> > > [2014-03-20 21:12:27,324] 685698910
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > INFO kafka.server.KafkaZooKeeper - done re-registering broker
> > > [2014-03-20 21:12:27,324] 685698910
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > INFO kafka.server.KafkaZooKeeper - Subscribing to /brokers/topics path
> to
> > > watch for new topics
> > > [2014-03-20 21:12:27,370] 685698956
> > >
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > > INFO kafka.server.ZookeeperLeaderElector$LeaderChangeListener - New
> > leader
> > > is 22
> > >
> > >
> > >
> >
>

Re: Zookeeper reconnect failed due to 'state changed (Expired)'

Posted by "Bae, Jae Hyeon" <me...@gmail.com>.
This issue is zookeeper resiliency.

What I have done is, ephemeral node creation is replaced by Apache
Curator's PersistentEphemeralNode recipe, to reinstate ephemeral nodes
after zookeeper blip. Also, all watchers also should be reinstated. Kafka
internally only handles session expired event but there are many bad
situations of zookeepers where Kafka can go haywire.




On Thu, Mar 20, 2014 at 9:23 PM, Neha Narkhede <ne...@gmail.com>wrote:

> 1.  What does 'zookeeper state changed (Expired)' mean?
>
> 2.  Has anyone seen issues like this before?  Where zookeeper connections
> are flaky enough to cause leader elections?
>
> It means zookeeper expired the session. The most common reason for this is
> client side GC (in your case, client is the Kafka broker) though if your
> zookeeper setup is flaky, this might even point to a server side I/O
> bottleneck or GC. Could you inspect the broker and zookeeper's gc logs?
>
>
> On Thu, Mar 20, 2014 at 3:26 PM, Andrew Otto <ot...@wikimedia.org> wrote:
>
> > Hm, just saw something a little fishy.
> >
> > (In the following logs, analytics1021 (id 21) and analytics1022 (id 22)
> > are Brokers and analytics1023,analytics1024,analytics1025 are
> Zookeepers.)
> >
> > At 2014-03-20 21:12:26, analytics1021 lost its connection to zookeeper.
> It
> > reconnected to analytics1023, but as it was doing so, it logged a
> > 'zookeeper state changed (Expired)' message, which apparently caused the
> > newly opened socket to close again.  Am I reading that right?
> >
> > In the time that it took analytics1021 to finally re-establish a
> zookeeper
> > connection, analytics1022 had become the leader for all partitions.  I
> can
> > rebalance the leaders, but I'd prefer if this didn't happen in the first
> > place.
> >
> > 1.  What does 'zookeeper state changed (Expired)' mean?
> > 2.  Has anyone seen issues like this before?  Where zookeeper connections
> > are flaky enough to cause leader elections?
> >
> > Thanks!
> > -Andrew Otto
> >
> >
> >
> > [2014-03-20 21:12:26,427] 685698013
> > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > org.apache.zookeeper.ClientCnxn - Client session timed out, have not
> heard
> > from server in 10067ms for sessionid 0x1425d903a3f1f89, closing socket
> > connection and attempting reconnect
> > [2014-03-20 21:12:26,528] 685698114 [main-EventThread] INFO
> > org.I0Itec.zkclient.ZkClient - zookeeper state changed (Disconnected)
> > [2014-03-20 21:12:27,252] 685698838
> > [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> > org.apache.zookeeper.ClientCnxn - Opening socket connection to server
> > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > [2014-03-20 21:12:27,271] 685698857
> > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > org.apache.zookeeper.ClientCnxn - Socket connection established to
> > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > org.I0Itec.zkclient.ZkClient - zookeeper state changed (Expired)
> > [2014-03-20 21:12:27,273] 685698859
> > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > org.apache.zookeeper.ClientCnxn - Unable to reconnect to ZooKeeper
> service,
> > session 0x1425d903a3f1f89 has expired, closing socket connection
> > [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> > org.apache.zookeeper.ZooKeeper - Initiating client connection,
> >
> connectString=analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad
> > sessionTimeout=6000 watcher=org.I0Itec.zkclient.ZkClient@63203b59
> > [2014-03-20 21:12:27,290] 685698876
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > INFO kafka.server.KafkaZooKeeper - re-registering broker info in ZK for
> > broker 21
> > [2014-03-20 21:12:27,290] 685698876 [main-EventThread] INFO
> > org.apache.zookeeper.ClientCnxn - EventThread shut down
> > [2014-03-20 21:12:27,290] 685698876 [main-SendThread()] INFO
> > org.apache.zookeeper.ClientCnxn - Opening socket connection to server
> > analytics1023.eqiad.wmnet/10.64.5.15:2181
> > [2014-03-20 21:12:27,292] 685698878
> > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > org.apache.zookeeper.ClientCnxn - Socket connection established to
> > analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> > [2014-03-20 21:12:27,313] 685698899
> > [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> > org.apache.zookeeper.ClientCnxn - Session establishment complete on
> server
> > analytics1023.eqiad.wmnet/10.64.5.15:2181, sessionid =
> > 0xff429b4201dd2352, negotiated timeout = 6000
> > [2014-03-20 21:12:27,314] 685698900 [main-EventThread] INFO
> > org.I0Itec.zkclient.ZkClient - zookeeper state changed (SyncConnected)
> > [2014-03-20 21:12:27,324] 685698910
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > INFO kafka.server.KafkaZooKeeper - done re-registering broker
> > [2014-03-20 21:12:27,324] 685698910
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > INFO kafka.server.KafkaZooKeeper - Subscribing to /brokers/topics path to
> > watch for new topics
> > [2014-03-20 21:12:27,370] 685698956
> >
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> > INFO kafka.server.ZookeeperLeaderElector$LeaderChangeListener - New
> leader
> > is 22
> >
> >
> >
>

Re: Zookeeper reconnect failed due to 'state changed (Expired)'

Posted by Neha Narkhede <ne...@gmail.com>.
1.  What does 'zookeeper state changed (Expired)' mean?

2.  Has anyone seen issues like this before?  Where zookeeper connections
are flaky enough to cause leader elections?

It means zookeeper expired the session. The most common reason for this is
client side GC (in your case, client is the Kafka broker) though if your
zookeeper setup is flaky, this might even point to a server side I/O
bottleneck or GC. Could you inspect the broker and zookeeper's gc logs?


On Thu, Mar 20, 2014 at 3:26 PM, Andrew Otto <ot...@wikimedia.org> wrote:

> Hm, just saw something a little fishy.
>
> (In the following logs, analytics1021 (id 21) and analytics1022 (id 22)
> are Brokers and analytics1023,analytics1024,analytics1025 are Zookeepers.)
>
> At 2014-03-20 21:12:26, analytics1021 lost its connection to zookeeper. It
> reconnected to analytics1023, but as it was doing so, it logged a
> 'zookeeper state changed (Expired)' message, which apparently caused the
> newly opened socket to close again.  Am I reading that right?
>
> In the time that it took analytics1021 to finally re-establish a zookeeper
> connection, analytics1022 had become the leader for all partitions.  I can
> rebalance the leaders, but I'd prefer if this didn't happen in the first
> place.
>
> 1.  What does 'zookeeper state changed (Expired)' mean?
> 2.  Has anyone seen issues like this before?  Where zookeeper connections
> are flaky enough to cause leader elections?
>
> Thanks!
> -Andrew Otto
>
>
>
> [2014-03-20 21:12:26,427] 685698013
> [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> org.apache.zookeeper.ClientCnxn - Client session timed out, have not heard
> from server in 10067ms for sessionid 0x1425d903a3f1f89, closing socket
> connection and attempting reconnect
> [2014-03-20 21:12:26,528] 685698114 [main-EventThread] INFO
> org.I0Itec.zkclient.ZkClient - zookeeper state changed (Disconnected)
> [2014-03-20 21:12:27,252] 685698838
> [main-SendThread(analytics1025.eqiad.wmnet:2181)] INFO
> org.apache.zookeeper.ClientCnxn - Opening socket connection to server
> analytics1023.eqiad.wmnet/10.64.5.15:2181
> [2014-03-20 21:12:27,271] 685698857
> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> org.apache.zookeeper.ClientCnxn - Socket connection established to
> analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> org.I0Itec.zkclient.ZkClient - zookeeper state changed (Expired)
> [2014-03-20 21:12:27,273] 685698859
> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> org.apache.zookeeper.ClientCnxn - Unable to reconnect to ZooKeeper service,
> session 0x1425d903a3f1f89 has expired, closing socket connection
> [2014-03-20 21:12:27,273] 685698859 [main-EventThread] INFO
> org.apache.zookeeper.ZooKeeper - Initiating client connection,
> connectString=analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad
> sessionTimeout=6000 watcher=org.I0Itec.zkclient.ZkClient@63203b59
> [2014-03-20 21:12:27,290] 685698876
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> INFO kafka.server.KafkaZooKeeper - re-registering broker info in ZK for
> broker 21
> [2014-03-20 21:12:27,290] 685698876 [main-EventThread] INFO
> org.apache.zookeeper.ClientCnxn - EventThread shut down
> [2014-03-20 21:12:27,290] 685698876 [main-SendThread()] INFO
> org.apache.zookeeper.ClientCnxn - Opening socket connection to server
> analytics1023.eqiad.wmnet/10.64.5.15:2181
> [2014-03-20 21:12:27,292] 685698878
> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> org.apache.zookeeper.ClientCnxn - Socket connection established to
> analytics1023.eqiad.wmnet/10.64.5.15:2181, initiating session
> [2014-03-20 21:12:27,313] 685698899
> [main-SendThread(analytics1023.eqiad.wmnet:2181)] INFO
> org.apache.zookeeper.ClientCnxn - Session establishment complete on server
> analytics1023.eqiad.wmnet/10.64.5.15:2181, sessionid =
> 0xff429b4201dd2352, negotiated timeout = 6000
> [2014-03-20 21:12:27,314] 685698900 [main-EventThread] INFO
> org.I0Itec.zkclient.ZkClient - zookeeper state changed (SyncConnected)
> [2014-03-20 21:12:27,324] 685698910
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> INFO kafka.server.KafkaZooKeeper - done re-registering broker
> [2014-03-20 21:12:27,324] 685698910
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> INFO kafka.server.KafkaZooKeeper - Subscribing to /brokers/topics path to
> watch for new topics
> [2014-03-20 21:12:27,370] 685698956
> [ZkClient-EventThread-24-analytics1023.eqiad.wmnet,analytics1024.eqiad.wmnet,analytics1025.eqiad.wmnet/kafka/eqiad]
> INFO kafka.server.ZookeeperLeaderElector$LeaderChangeListener - New leader
> is 22
>
>
>