Great! Thanks!
On Tue, Jul 29, 2014 at 10:57 AM, Guozhang Wang wrote:
> Good to know. The issue with initial replica factor = 1 is still a real bug
> though, and I will probably fix it along with KAFKA-1430.
>
> Guozhang
>
>
> On Tue, Jul 29, 2014 at 10:27 AM, Jad Naous
OK, it looks like if the initial replica count was 2 for the topic, then
this doesn't happen. Good catch Guozhang, and sorry for the oversight!
On Mon, Jul 28, 2014 at 5:29 PM, Guozhang Wang wrote:
> Yeah, please try that and let us know.
>
--
*Jad Naous* | Engineering |
28, 2014 at 1:55 PM, Jad Naous
> wrote:
>
> > The logs are quite large. I've sifted through them, and I'm attaching the
> > logs for the relevant parts where the lost message goes through the
> system.
> > Here's what the test does:
> >
> > 0) S
Yes.
On Mon, Jul 28, 2014 at 5:11 PM, Guozhang Wang wrote:
> Just to confirm, are you running version 0.8.1.1?
>
> Guozhang
>
>
> On Mon, Jul 28, 2014 at 4:23 PM, Jad Naous
> wrote:
>
> > Done! Thanks!
> >
> >
> > On Mon, Jul 28, 2014 at 4:16
t;
>
> On Mon, Jul 28, 2014 at 1:55 PM, Jad Naous
> wrote:
>
> > The logs are quite large. I've sifted through them, and I'm attaching the
> > logs for the relevant parts where the lost message goes through the
> system.
> > Here's what the test d
all the data was consumed
You can seem from the producer.log that the data: {"field1": ["10"],
"idField": "id-5-59"} was sent to broker0 successfully, but the consumer
never sees it.
Thanks,
Jad.
On Mon, Jul 28, 2014 at 10:33 AM, Guozhang Wan
cationFactor).
>
> I now agree that this is not related to KAFKA-1211 but a different small
> bug. We need to probably file another JIRA for this. But I think after this
> one is fixed (which should be much easier than KAFKA-1211), Jad's scenario
> should not cause data l
> >
> > And let's say there is a leader election between step 4) and 5), for your
> > case it is due to preferred leader election, but it could also be that
> > current leader fails, etc. Then on becoming the new leader the follower
> > will truncate its data to
n
> leader migration. The issue here though, is that follower's HW update is
> slightly late the leader's HW update, and hence the truncation may cause
> data loss. We will try to fix this issue in KAFKA-1211.
>
> Guozhang
>
>
> On Fri, Jul 25, 2014 at 4:12 PM, Jad N
referred leader election, but it could also be that
> current leader fails, etc. Then on becoming the new leader the follower
> will truncate its data to 8, which is the HW it knows. Hence the data loss.
>
> The proposed solution in KAFKA-1211 will tackle this issue.
>
> Guozhang
> preferred leader elector does consider whether the selected replica is in
> ISR or not. This means that by the time the election is triggered, broker 1
> is added to ISR by broker 0. Could you check before step 3, is there any
> log entries on broker 0 adding broker 1 to ISR or u
rom other follower replicas, to the position of the minimum
> starting offsets of the fetch requests. For producer.ack=-1, the leader
> will only return the response once it knows the HW has been updated to be
> larger than the produce end offset.
>
> Guozhang
>
>
> On Fri, J
. So in that scenario it
> should still be no data loss.
>
>
> On Thu, Jul 24, 2014 at 7:48 PM, Jad Naous
> wrote:
>
> > Actually, is the following scenario possible?
> > - We start off with only 1 replica (the leader)
> > - the producer continuously sends messag
, but it's not fully caught
up to the old leader and then we lose the last message...
On Thu, Jul 24, 2014 at 6:29 PM, Jad Naous
wrote:
> Ah yes. OK, thanks! So it seems like we should only manually trigger
> re-election after seeing that all replicas are in the ISR. Is there a bug
een acked by all
> replicas to respond. So that will not cause data loss.
>
>
> On Thu, Jul 24, 2014 at 6:07 PM, Jad Naous
> wrote:
>
> > Hi Guozhang,
> >
> > Isn't it also possible to lose messages even if the preferred leader is
> in
> > the ISR, wh
ng
>
>
> On Thu, Jul 24, 2014 at 4:21 PM, Jad Naous
> wrote:
>
> > Hi,
> >
> > I have a test that continuously sends messages to one broker, brings up
> > another broker, and adds it as a replica for all partitions, with it
> being
> > the preferred r
this problem also happens there.
The log on the old leader gets truncated to the offset on the new leader.
What's the solution? Can I make a new broker leader for partitions that are
currently active without losing data?
Thanks,
Jad.
--
*Jad Naous* | Engineering | AppDynamics
<http://www.appdynamics.com>
17 matches
Mail list logo