Home | About | Sematext search-lucene.com search-hadoop.com
 Search Hadoop and all its subprojects:

Switch to Threaded View
Kafka >> mail # user >> Data loss in case of request.required.acks set to -1


Copy link to this message
-
Re: Data loss in case of request.required.acks set to -1
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 <[EMAIL PROTECTED]> 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 <[EMAIL PROTECTED]> 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 <
> > [EMAIL PROTECTED]> 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 <[EMAIL PROTECTED]>
> > wrote:
> > >
> > > > Yes, please go ahead.
> > > >
> > > >
> > > > On Thu, Dec 19, 2013 at 2:30 AM, Hanish Bansal <
> > > > [EMAIL PROTECTED]> 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 <[EMAIL PROTECTED]
> >
> > > > 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 <
> [EMAIL PROTECTED]>
> > > > > 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

*Thanks & Regards*
*Hanish Bansal*