seem to solve the problem.
> Regards,
> Jiang
>
> From: users@kafka.apache.org At: Jul 19 2014 00:06:52
> To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -), users@kafka.apache.org
> Subject: Re: message loss for sync producer, acks=2, topic replicas=3
>
> Hi Jiang,
>
&g
From: users@kafka.apache.org At: Jul 19 2014 00:06:52
> To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -), users@kafka.apache.org
> Subject: Re: message loss for sync producer, acks=2, topic replicas=3
>
> Hi Jiang,
>
> One thing you can try is to set acks=-1, and set the
> r
14 00:06:52
To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -), users@kafka.apache.org
Subject: Re: message loss for sync producer, acks=2, topic replicas=3
Hi Jiang,
One thing you can try is to set acks=-1, and set the
replica.lag.max.messages properly such that it will not kicks all follower
rep
figure this out for data-loss sensitive applications.
>
> Thanks,
> Jiang
>
> From: users@kafka.apache.org At: Jul 18 2014 18:41:00
> To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -), users@kafka.apache.org
> Subject: Re: message loss for sync producer, acks=2, topic replicas=3
figure
this out for data-loss sensitive applications.
Thanks,
Jiang
From: users@kafka.apache.org At: Jul 18 2014 18:41:00
To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -), users@kafka.apache.org
Subject: Re: message loss for sync producer, acks=2, topic replicas=3
You probably don't need t
; >props.put("message.send.max.retries", "60");
> > > >
> > > >props.put("retry.backoff.ms", "300");
> > > >
> > > > Consumer:
> > > >
> > > >props.put("zookeeper.session
oducer sends a
> > > message m1 and receives acks from L and F1. Before the messge is
> > replicated
> > > to F2, L is down. In the following leader election, F2, instead of F1,
> > > becomes the leader, and loses m1 somehow.
> > > Could that be the root c
n: at one moment the
> > > leader L, and two followers F1, F2 are all in ISR. The producer sends a
> > > message m1 and receives acks from L and F1. Before the messge is
> > replicated
> > > to F2, L is down. In the following leader election, F2, instead
ment the
> > > leader L, and two followers F1, F2 are all in ISR. The producer sends a
> > > message m1 and receives acks from L and F1. Before the messge is
> > replicated
> > > to F2, L is down. In the following leader election, F2, instead of F1,
> > > bec
In the following leader election, F2, instead of F1,
> > becomes the leader, and loses m1 somehow.
> > Could that be the root cause?
> > Thanks,
> > Jiang
> >
> > From: users@kafka.apache.org At: Jul 15 2014 15:05:25
> > To: users@kafka.apache.org
> >
loses m1 somehow.
> Could that be the root cause?
> Thanks,
> Jiang
>
> From: users@kafka.apache.org At: Jul 15 2014 15:05:25
> To: users@kafka.apache.org
> Subject: Re: message loss for sync producer, acks=2, topic replicas=3
>
> Guozhang,
>
> Please find the c
ot cause?
> Thanks,
> Jiang
>
> From: users@kafka.apache.org At: Jul 15 2014 15:05:25
> To: users@kafka.apache.org
> Subject: Re: message loss for sync producer, acks=2, topic replicas=3
>
> Guozhang,
>
> Please find the config below:
>
> Producer:
>
the leader, and
loses m1 somehow.
Could that be the root cause?
Thanks,
Jiang
From: users@kafka.apache.org At: Jul 15 2014 15:05:25
To: users@kafka.apache.org
Subject: Re: message loss for sync producer, acks=2, topic replicas=3
Guozhang,
Please find the config below:
Producer:
props.put
PartitionCount:1ReplicationFactor:3
Configs:retention.bytes=1000000
Thanks,
Jiang
From: users@kafka.apache.org At: Jul 15 2014 13:59:03
To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -), users@kafka.apache.org
Subject: Re: message loss for sync producer, acks=2, topic replicas=3
che.org At: Jul 15 2014 13:27:50
> To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -), users@kafka.apache.org
> Subject: Re: message loss for sync producer, acks=2, topic replicas=3
>
> Hello Jiang,
>
> Which version of Kafka are you using, and did you kill the broker with -9?
>
Guozhang,
I'm testing on 0.8.1.1; just kill pid, no -9.
Regards,
Jiang
From: users@kafka.apache.org At: Jul 15 2014 13:27:50
To: JIANG WU (PRICEHISTORY) (BLOOMBERG/ 731 LEX -), users@kafka.apache.org
Subject: Re: message loss for sync producer, acks=2, topic replicas=3
Hello Jiang,
Hello Jiang,
Which version of Kafka are you using, and did you kill the broker with -9?
Guozhang
On Tue, Jul 15, 2014 at 9:23 AM, Jiang Wu (Pricehistory) (BLOOMBERG/ 731
LEX -) wrote:
> Hi,
> I observed some unexpected message loss in kafka fault tolerant test. In
> the test, a topic with 3 r
17 matches
Mail list logo