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/02/21 19:22:18 UTC

Unexpected broker election

Hi all,

This has happened a couple of times to me now in the past month, and I’m not entirely sure of the cause, although I have a suspicion.

Early this morning (UTC), it looks like one of my two brokers (id 21) lost its connection to Zookeeper for a very short period of time.  This caused the second broker (id 22) to quickly become the leader for all partitions.  Once broker 21 was able to re-establish its Zookeeper connection, it noticed that it has a stale list for the ISR, got its updated list, and started replicating from broker 22 for all partitions.  Broker 21 then quickly rejoined the ISR, but annoyingly (but expectedly), broker 22 remained the leader.  All of this happened in under a minute.

I’m wondering if https://issues.apache.org/jira/browse/KAFKA-766 is related.  The current batch size on our producers is 6000 msgs or 1000 ms (I’ve been meaning to reduce this).  We do about 6000 msgs per second / per producer, and have 10 partitions in this relevant topic.  A couple of days ago, we noticed flapping ISR Shrink/Expand logs, so I upped replica.lag.max.messages to 10000, so that it would surely be above our batch size.  I still occasionally see flapping ISR Shrinks/Expands, but hope that when I reduce the producer batch size, I will stop seeing these.

Anyway, I’m not entirely sure what happened here.  Could flapping ISRs potentially cause this?

For reference, the relevant logs from my brokers and a zookeeper are here: https://gist.github.com/ottomata/9139443

Thanks!
-Andrew Otto



Re: Unexpected broker election

Posted by Jun Rao <ju...@gmail.com>.
Have you looked at
https://cwiki.apache.org/confluence/display/KAFKA/FAQ#FAQ-Whypartitionleadersmigratethemselvessometimes
?

Thanks,

Jun


On Sat, Feb 22, 2014 at 2:06 PM, Andrew Otto <ot...@wikimedia.org> wrote:

> Yeah, I can do that, but I'd prefer if the first broker didn't drop out of
> the ISR in the first place.  Just trying to figure out why it did...
>
>
> On Feb 21, 2014, at 11:30 PM, Jun Rao <ju...@gmail.com> wrote:
>
> > So, it sounds like you want the leader to be moved back to the failed
> > broker that has caught up. For now, you can use this tool (
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Replication+tools#Replicationtools-2.PreferredReplicaLeaderElectionTool
> ).
> > In 0.8.1 release, we have an option to balance the leaders automatically
> > every configurable period of time.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Fri, Feb 21, 2014 at 10:22 AM, Andrew Otto <ot...@wikimedia.org>
> wrote:
> >
> >> Hi all,
> >>
> >> This has happened a couple of times to me now in the past month, and I'm
> >> not entirely sure of the cause, although I have a suspicion.
> >>
> >> Early this morning (UTC), it looks like one of my two brokers (id 21)
> lost
> >> its connection to Zookeeper for a very short period of time.  This
> caused
> >> the second broker (id 22) to quickly become the leader for all
> partitions.
> >> Once broker 21 was able to re-establish its Zookeeper connection, it
> >> noticed that it has a stale list for the ISR, got its updated list, and
> >> started replicating from broker 22 for all partitions.  Broker 21 then
> >> quickly rejoined the ISR, but annoyingly (but expectedly), broker 22
> >> remained the leader.  All of this happened in under a minute.
> >>
> >> I'm wondering if https://issues.apache.org/jira/browse/KAFKA-766 is
> >> related.  The current batch size on our producers is 6000 msgs or 1000
> ms
> >> (I've been meaning to reduce this).  We do about 6000 msgs per second /
> per
> >> producer, and have 10 partitions in this relevant topic.  A couple of
> days
> >> ago, we noticed flapping ISR Shrink/Expand logs, so I upped
> >> replica.lag.max.messages to 10000, so that it would surely be above our
> >> batch size.  I still occasionally see flapping ISR Shrinks/Expands, but
> >> hope that when I reduce the producer batch size, I will stop seeing
> these.
> >>
> >> Anyway, I'm not entirely sure what happened here.  Could flapping ISRs
> >> potentially cause this?
> >>
> >> For reference, the relevant logs from my brokers and a zookeeper are
> here:
> >> https://gist.github.com/ottomata/9139443
> >>
> >> Thanks!
> >> -Andrew Otto
> >>
> >>
> >>
>
>

Re: Unexpected broker election

Posted by Andrew Otto <ot...@wikimedia.org>.
Yeah, I can do that, but I’d prefer if the first broker didn’t drop out of the ISR in the first place.  Just trying to figure out why it did…


On Feb 21, 2014, at 11:30 PM, Jun Rao <ju...@gmail.com> wrote:

> So, it sounds like you want the leader to be moved back to the failed
> broker that has caught up. For now, you can use this tool (
> https://cwiki.apache.org/confluence/display/KAFKA/Replication+tools#Replicationtools-2.PreferredReplicaLeaderElectionTool).
> In 0.8.1 release, we have an option to balance the leaders automatically
> every configurable period of time.
> 
> Thanks,
> 
> Jun
> 
> 
> On Fri, Feb 21, 2014 at 10:22 AM, Andrew Otto <ot...@wikimedia.org> wrote:
> 
>> Hi all,
>> 
>> This has happened a couple of times to me now in the past month, and I'm
>> not entirely sure of the cause, although I have a suspicion.
>> 
>> Early this morning (UTC), it looks like one of my two brokers (id 21) lost
>> its connection to Zookeeper for a very short period of time.  This caused
>> the second broker (id 22) to quickly become the leader for all partitions.
>> Once broker 21 was able to re-establish its Zookeeper connection, it
>> noticed that it has a stale list for the ISR, got its updated list, and
>> started replicating from broker 22 for all partitions.  Broker 21 then
>> quickly rejoined the ISR, but annoyingly (but expectedly), broker 22
>> remained the leader.  All of this happened in under a minute.
>> 
>> I'm wondering if https://issues.apache.org/jira/browse/KAFKA-766 is
>> related.  The current batch size on our producers is 6000 msgs or 1000 ms
>> (I've been meaning to reduce this).  We do about 6000 msgs per second / per
>> producer, and have 10 partitions in this relevant topic.  A couple of days
>> ago, we noticed flapping ISR Shrink/Expand logs, so I upped
>> replica.lag.max.messages to 10000, so that it would surely be above our
>> batch size.  I still occasionally see flapping ISR Shrinks/Expands, but
>> hope that when I reduce the producer batch size, I will stop seeing these.
>> 
>> Anyway, I'm not entirely sure what happened here.  Could flapping ISRs
>> potentially cause this?
>> 
>> For reference, the relevant logs from my brokers and a zookeeper are here:
>> https://gist.github.com/ottomata/9139443
>> 
>> Thanks!
>> -Andrew Otto
>> 
>> 
>> 


Re: Unexpected broker election

Posted by Jun Rao <ju...@gmail.com>.
So, it sounds like you want the leader to be moved back to the failed
broker that has caught up. For now, you can use this tool (
https://cwiki.apache.org/confluence/display/KAFKA/Replication+tools#Replicationtools-2.PreferredReplicaLeaderElectionTool).
In 0.8.1 release, we have an option to balance the leaders automatically
every configurable period of time.

Thanks,

Jun


On Fri, Feb 21, 2014 at 10:22 AM, Andrew Otto <ot...@wikimedia.org> wrote:

> Hi all,
>
> This has happened a couple of times to me now in the past month, and I'm
> not entirely sure of the cause, although I have a suspicion.
>
> Early this morning (UTC), it looks like one of my two brokers (id 21) lost
> its connection to Zookeeper for a very short period of time.  This caused
> the second broker (id 22) to quickly become the leader for all partitions.
>  Once broker 21 was able to re-establish its Zookeeper connection, it
> noticed that it has a stale list for the ISR, got its updated list, and
> started replicating from broker 22 for all partitions.  Broker 21 then
> quickly rejoined the ISR, but annoyingly (but expectedly), broker 22
> remained the leader.  All of this happened in under a minute.
>
> I'm wondering if https://issues.apache.org/jira/browse/KAFKA-766 is
> related.  The current batch size on our producers is 6000 msgs or 1000 ms
> (I've been meaning to reduce this).  We do about 6000 msgs per second / per
> producer, and have 10 partitions in this relevant topic.  A couple of days
> ago, we noticed flapping ISR Shrink/Expand logs, so I upped
> replica.lag.max.messages to 10000, so that it would surely be above our
> batch size.  I still occasionally see flapping ISR Shrinks/Expands, but
> hope that when I reduce the producer batch size, I will stop seeing these.
>
> Anyway, I'm not entirely sure what happened here.  Could flapping ISRs
> potentially cause this?
>
> For reference, the relevant logs from my brokers and a zookeeper are here:
> https://gist.github.com/ottomata/9139443
>
> Thanks!
> -Andrew Otto
>
>
>