kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Guozhang Wang <wangg...@gmail.com>
Subject Re: Data loss in case of request.required.acks set to -1
Date Mon, 23 Dec 2013 19:22:51 GMT
Hanish,

Originally when you create the two partitions their leadership should be
evenly distributed to two brokers, i.e. one broker get one partition.
But from your case broker 1 is the leader for both partition 1 and 0, and
from the replica list broker 0 should be originally the leader for
partition1 since the leader of a partition should be the first one in the
replica list.

This means broker 0 was bounced or halted (e.g. by a GC, etc) before, and
hence the leadership of partition 1 migrates to broker 1, and also it is
still catching up after the bounce since it is not in isr for any
partitions yet. In this case, when you bounce broker 1, broker 0 which is
not in ISR will be selected as the new leader for both and hence cause data
loss.

If you are doing experiments on rolling bounce of say N replication factor,
one thing to do is wait for the isr to have at least 2 brokers before
bouncing the next one, otherwise data loss will not be guaranteed even if
number of replicas is larger than 2.

If you want to read more I would recommend this blog about Kafka's
guarantee:

http://blog.empathybox.com/post/62279088548/a-few-notes-on-kafka-and-jepsen

Guozhang




On Sun, Dec 22, 2013 at 10:38 PM, Hanish Bansal <
hanish.bansal.agarwal@gmail.com> wrote:

> Hi Guazhang,
>
> When both nodes are alive then topic isr status is:
>
> topic: test-trunk111    partition: 0    leader: 0    replicas: 1,0    isr:
> 0
> topic: test-trunk111    partition: 1    leader: 0    replicas: 0,1    isr:
> 0
>
> Now as the leader node is broker-0 so when i am producing the data then
> meanwhile kill the leader node.
> After leader goes down, topic isr status is:
>
> topic: test-trunk111    partition: 0    leader: 1    replicas: 1,0    isr:
> 1
> topic: test-trunk111    partition: 1    leader: 1    replicas: 0,1    isr:
> 1
>
> Now after all data produced when i consumed the data, there is some data
> loss.
>
> *Also in controller logs there is entry like:*
>
> [2013-12-23 10:25:07,648] DEBUG [OfflinePartitionLeaderSelector]: No broker
> in ISR is alive for [test-trunk111,1]. Pick the leader from the alive
> assigned replicas: 1 (kafka.controller.OfflinePartitionLeaderSelector)
> [2013-12-23 10:25:07,648] WARN [OfflinePartitionLeaderSelector]: No broker
> in ISR is alive for [test-trunk111,1]. Elect leader 1 from live brokers 1.
> There's potential data loss.
> (kafka.controller.OfflinePartitionLeaderSelector)
> [2013-12-23 10:25:07,649] INFO [OfflinePartitionLeaderSelector]: Selected
> new leader and ISR {"leader":1,"leader_epoch":1,"isr":[1]} for offline
> partition [test-trunk111,1]
> (kafka.controller.OfflinePartitionLeaderSelector)
>
> Is there any solution for this behaviour ?
>
>
> On Fri, Dec 20, 2013 at 7:27 PM, Guozhang Wang <wangguoz@gmail.com> wrote:
>
> > Hanish,
> >
> > One thing you can check is when you kill one of the brokers, is the other
> > broker on the ISR last of the partition that killed broker is hosting.
> This
> > can be done using the kafka-topics tool.
> >
> > Also you can check if the controller log if there is any entry like "No
> > broker in ISR is alive for %s. Elect leader %d from live brokers %s.
> > There's potential data loss."
> >
> > Guozhang
> >
> >
> > On Fri, Dec 20, 2013 at 9:11 AM, Jun Rao <junrao@gmail.com> wrote:
> >
> > > Could you reproduce this easily? If so, could you file a jira and
> > describe
> > > the steps?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Thu, Dec 19, 2013 at 9:41 PM, Hanish Bansal <
> > > hanish.bansal.agarwal@gmail.com> wrote:
> > >
> > > > Hi Guozhang,
> > > >
> > > > I have tried with Kafka-0.8.1 after applying patch 1188 but thats not
> > > > helping in this case.
> > > >
> > > > Also controlled.shutdown.enable is also not helpful in case of
> > abnormally
> > > > shutdown (i.e. SIGKILL (-9)).
> > > >
> > > > Any other suggestion?
> > > >
> > > >
> > > > On Thu, Dec 19, 2013 at 3:59 PM, Guozhang Wang <wangguoz@gmail.com>
> > > wrote:
> > > >
> > > > > Yes, please go ahead.
> > > > >
> > > > >
> > > > > On Thu, Dec 19, 2013 at 2:30 AM, Hanish Bansal <
> > > > > hanish.bansal.agarwal@gmail.com> wrote:
> > > > >
> > > > > > Hi Guazhang,
> > > > > >
> > > > > > Can I try it now with trunk HEAD kafka code after applying the
> > patch
> > > > > > KAFKA-1188.patch<
> > > > > >
> > > > >
> > > >
> > >
> >
> https://issues.apache.org/jira/secure/attachment/12619475/KAFKA-1188.patch
> > > > > > >?
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Wed, Dec 18, 2013 at 9:49 PM, Guozhang Wang <
> wangguoz@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > Kafka server's shutdown hook should capture all SIG but
NOT
> > SIGKILL
> > > > > (-9),
> > > > > > > and the controlled shut down process will not be triggered
in
> > this
> > > > > case.
> > > > > > >
> > > > > > > That said, if the other replica is in ISR then even kill
-9
> >  should
> > > > not
> > > > > > > lose data. I am currently working on this JIRA that might
be
> > > related
> > > > if
> > > > > > > brokers are bounced iteratively:
> > > > > > >
> > > > > > > https://issues.apache.org/jira/browse/KAFKA-1188
> > > > > > >
> > > > > > > Hanish, could you retry trunk HEAD once this one is resolved?
> > > > > > >
> > > > > > >
> > > > > > > On Wed, Dec 18, 2013 at 12:00 PM, Joe Stein <
> > joe.stein@stealth.ly>
> > > > > > wrote:
> > > > > > >
> > > > > > > > leader election should start for the brokers that
are in the
> > isr
> > > > for
> > > > > > the
> > > > > > > > partitions that are on that replica that are leaders
by the
> > other
> > > > > > > replicas
> > > > > > > > still in the isr, and the leader failed removed from
the isr.
> > >  The
> > > > > isr
> > > > > > > will
> > > > > > > > shrink for all other partitions this broker is in
the isr on
> > but
> > > > not
> > > > > > the
> > > > > > > > leader.
> > > > > > > >
> > > > > > > > so lots of re-giggling and the time there is going
to be
> > related
> > > to
> > > > > how
> > > > > > > > many partitions and brokers you have.
> > > > > > > >
> > > > > > > > On Wed, Dec 18, 2013 at 2:49 PM, Robert Rodgers <
> > > > rsrodgers@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > what happens if the physical machine dies or
the kernel
> > panics?
> > > > > > > > >
> > > > > > > > > On Dec 18, 2013, at 9:44 AM, Hanish Bansal <
> > > > > > > > > hanish.bansal.agarwal@gmail.com> wrote:
> > > > > > > > >
> > > > > > > > > > Yup definitely i would like to try that
If
> > > > > > controlled.shutdown.enable
> > > > > > > > > > property works in case of kill -9.
> > > > > > > > > >
> > > > > > > > > > I hope that this option will be perfect.
> > > > > > > > > >
> > > > > > > > > > Thanks for quick response, really appreciate
it.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Wed, Dec 18, 2013 at 10:52 PM, Joe Stein
<
> > > > > joe.stein@stealth.ly>
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > >> Wouldn't you want to set the controlled.shutdown.enable
> to
> > > > true
> > > > > so
> > > > > > > the
> > > > > > > > > >> broker would do this for you before
ending itself?
> > > > > > > > > >>
> > > > > > > > > >> /*******************************************
> > > > > > > > > >> Joe Stein
> > > > > > > > > >> Founder, Principal Consultant
> > > > > > > > > >> Big Data Open Source Security LLC
> > > > > > > > > >> http://www.stealth.ly
> > > > > > > > > >> Twitter: @allthingshadoop <
> > > > > http://www.twitter.com/allthingshadoop
> > > > > > >
> > > > > > > > > >> ********************************************/
> > > > > > > > > >>
> > > > > > > > > >>
> > > > > > > > > >> On Wed, Dec 18, 2013 at 11:36 AM, pushkar
priyadarshi <
> > > > > > > > > >> priyadarshi.pushkar@gmail.com> wrote:
> > > > > > > > > >>
> > > > > > > > > >>> my doubt was they are dropping off
at producer level
> > > only.so
> > > > > > > > suggested
> > > > > > > > > >>> playing with paramaters like retries
and backoff.msand
> > > also
> > > > > > with
> > > > > > > > > >>> refreshinterval on producer side.
> > > > > > > > > >>>
> > > > > > > > > >>> Regards,
> > > > > > > > > >>> Pushkar
> > > > > > > > > >>>
> > > > > > > > > >>>
> > > > > > > > > >>> On Wed, Dec 18, 2013 at 10:01 PM,
Guozhang Wang <
> > > > > > > wangguoz@gmail.com>
> > > > > > > > > >>> wrote:
> > > > > > > > > >>>
> > > > > > > > > >>>> Hanish,
> > > > > > > > > >>>>
> > > > > > > > > >>>> Did you "kill -9" one of the
brokers only or bouncing
> > them
> > > > > > > > > iteratively?
> > > > > > > > > >>>>
> > > > > > > > > >>>> Guozhang
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>> On Wed, Dec 18, 2013 at 8:02
AM, Joe Stein <
> > > > > > joe.stein@stealth.ly>
> > > > > > > > > >> wrote:
> > > > > > > > > >>>>
> > > > > > > > > >>>>> How many replicas do you
have?
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>> On Wed, Dec 18, 2013 at
8:57 AM, Hanish Bansal <
> > > > > > > > > >>>>> hanish.bansal.agarwal@gmail.com>
wrote:
> > > > > > > > > >>>>>
> > > > > > > > > >>>>>> Hi pushkar,
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> I tried with configuring
 "message.send.max.retries"
> > to
> > > > 10.
> > > > > > > > Default
> > > > > > > > > >>>> value
> > > > > > > > > >>>>>> for this is 3.
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> But still facing data
loss.
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> On Wed, Dec 18, 2013
at 12:44 PM, pushkar
> priyadarshi
> > <
> > > > > > > > > >>>>>> priyadarshi.pushkar@gmail.com>
wrote:
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>> You can try setting
a higher value for
> > > > > > > "message.send.max.retries"
> > > > > > > > > >>> in
> > > > > > > > > >>>>>>> producer config.
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> Regards,
> > > > > > > > > >>>>>>> Pushkar
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>> On Wed, Dec 18,
2013 at 5:34 PM, Hanish Bansal <
> > > > > > > > > >>>>>>> hanish.bansal.agarwal@gmail.com>
wrote:
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>>> Hi All,
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> We are having
kafka cluster of 2 nodes. (using
> 0.8.0
> > > > final
> > > > > > > > > >>> release)
> > > > > > > > > >>>>>>>> Replication
Factor: 2
> > > > > > > > > >>>>>>>> Number of partitions:
2
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> I have configured
request.required.acks in
> producer
> > > > > > > > > >> configuration
> > > > > > > > > >>>> to
> > > > > > > > > >>>>>> -1.
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> As mentioned
in documentation
> > > > > > > > > >>>>>>>>
> > > > > http://kafka.apache.org/documentation.html#producerconfigs,
> > > > > > > > > >>>> setting
> > > > > > > > > >>>>>> this
> > > > > > > > > >>>>>>>> value to -1
provides guarantee that no messages
> will
> > > be
> > > > > > lost.
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> I am getting
below behaviour:
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> If kafka is
running as foreground process and i am
> > > > > shutting
> > > > > > > > > >> down
> > > > > > > > > >>>> the
> > > > > > > > > >>>>>>> kafka
> > > > > > > > > >>>>>>>> leader node
using "ctrl+C" then no data is lost.
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> But if i abnormally
terminate the kafka using
> "kill
> > -9
> > > > > > <pid>"
> > > > > > > > > >>> then
> > > > > > > > > >>>>>> still
> > > > > > > > > >>>>>>>> facing data
loss even after configuring
> > > > > > request.required.acks
> > > > > > > > > >> to
> > > > > > > > > >>>> -1.
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>> Any suggestions?
> > > > > > > > > >>>>>>>> --
> > > > > > > > > >>>>>>>> *Thanks &
Regards*
> > > > > > > > > >>>>>>>> *Hanish Bansal*
> > > > > > > > > >>>>>>>>
> > > > > > > > > >>>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>> --
> > > > > > > > > >>>>>> *Thanks & Regards*
> > > > > > > > > >>>>>> *Hanish Bansal*
> > > > > > > > > >>>>>>
> > > > > > > > > >>>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>>
> > > > > > > > > >>>> --
> > > > > > > > > >>>> -- Guozhang
> > > > > > > > > >>>>
> > > > > > > > > >>>
> > > > > > > > > >>
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > --
> > > > > > > > > > *Thanks & Regards*
> > > > > > > > > > *Hanish Bansal*
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > -- Guozhang
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > *Thanks & Regards*
> > > > > > *Hanish Bansal*
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > *Thanks & Regards*
> > > > *Hanish Bansal*
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>
>
>
> --
> *Thanks & Regards*
> *Hanish Bansal*
>



-- 
-- Guozhang

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