Re: Lost messages during leader election

2014-07-29 Thread Jad Naous
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 > wrote: > > > OK, it l

Re: Lost messages during leader election

2014-07-29 Thread Guozhang Wang
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 wrote: > OK, it looks like if the initial replica count was 2 for the topic, then > this doesn't happen. Go

Re: Lost messages during leader election

2014-07-29 Thread 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 | AppDynamics

Re: Lost messages during leader election

2014-07-28 Thread Guozhang Wang
Yeah, please try that and let us know.

Re: Lost messages during leader election

2014-07-28 Thread Jad Naous
Ah good point. The replication factor at the beginning is 1! I'll rerun the test with replication factor = 2. On Mon, Jul 28, 2014 at 5:17 PM, Guozhang Wang wrote: > Also with 2 brokers initially, is the replication factor also 2 at the > beginning or just 1? > > On Mon, Jul 28, 2014 at 1:55 PM

Re: Lost messages during leader election

2014-07-28 Thread Jad Naous
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 PM, Guozhang Wang > wrote: > > > > > Thanks Jad, >

Re: Lost messages during leader election

2014-07-28 Thread Guozhang Wang
Also with 2 brokers initially, is the replication factor also 2 at the beginning or just 1? 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.

Re: Lost messages during leader election

2014-07-28 Thread Guozhang Wang
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 PM, Guozhang Wang wrote: > > > Thanks Jad, > > > > The mailing list may blocked the attachments. I have file a JIRA for your > > issu

Re: Lost messages during leader election

2014-07-28 Thread Jad Naous
Done! Thanks! On Mon, Jul 28, 2014 at 4:16 PM, Guozhang Wang wrote: > Thanks Jad, > > The mailing list may blocked the attachments. I have file a JIRA for your > issue, could you upload the logs there? > > https://issues.apache.org/jira/browse/KAFKA-1561 > > Guozhang > > > On Mon, Jul 28, 2014

Re: Lost messages during leader election

2014-07-28 Thread Guozhang Wang
Thanks Jad, The mailing list may blocked the attachments. I have file a JIRA for your issue, could you upload the logs there? https://issues.apache.org/jira/browse/KAFKA-1561 Guozhang On Mon, Jul 28, 2014 at 1:55 PM, Jad Naous wrote: > The logs are quite large. I've sifted through them, and

Re: Lost messages during leader election

2014-07-28 Thread Jad Naous
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) Start two brokers, one producer, one consumer. Topic has 20 partitions, using default partitioning scheme (which seems

Re: Lost messages during leader election

2014-07-28 Thread Guozhang Wang
Also, for the initial two replica case did you see any error/warning logs on your producers? Guozhang On Mon, Jul 28, 2014 at 10:32 AM, Guozhang Wang wrote: > Hi Jad, > > Just to clarify, you also see data loss when you created the topic with > replica factor 2, and two replicas running, and a

Re: Lost messages during leader election

2014-07-28 Thread Guozhang Wang
Hi Jad, Just to clarify, you also see data loss when you created the topic with replica factor 2, and two replicas running, and after an auto leader election triggered? If that is the case could you attach the logs of all involved brokers here? For your second question, KAFKA-1211 is designed to

Re: Lost messages during leader election

2014-07-28 Thread Jad Naous
Guozhang, I have actually also seen this happen when there are two replicas initially. So this problem is not limited to 1 replica. The issue is the truncation after leader election, which will also happen on the second replica. Coming back to your objections: First case: inconsistency between

Re: Lost messages during leader election

2014-07-28 Thread Guozhang Wang
Jun, Jad, I think in this case data loss can still happen, since the replica factor was previously one, and in handling the produce requests, if the server decides that all the produced partitions have a replica factor of 1 it will also directly send back the response instead of putting the reques

Re: Lost messages during leader election

2014-07-27 Thread Jad Naous
So in summary, is it true to say that currently triggering leader reelection is not a safe operation? I have been able to reproduce that message loss pretty reliably in tests. If that is the case, isn't that an important operation in a large cluster where nodes go up and down? On Jul 25, 2014 10:00

Re: Lost messages during leader election

2014-07-25 Thread Jun Rao
Actually, I don't think KAFKA-1211 will happen with just 2 replicas. When a replica becomes a leader, it never truncates its log. Only when a replica becomes follower, it truncates its log to HW. So in this particular case, the new leader will not truncate data to offset 8. Thanks, Jun On Fri,

Re: Lost messages during leader election

2014-07-25 Thread Jad Naous
Thanks for the great explanation! So I guess for now, it's not safe to do leader election for rebalancing leadership? Or under what conditions can I trigger re-election without losing messages? It seems like this issue can always happen. On Fri, Jul 25, 2014 at 5:09 PM, Guozhang Wang wrote: > H

Re: Lost messages during leader election

2014-07-25 Thread Guozhang Wang
Hello Jad, The reason why the new leader needs to truncate to HW is to that only messages that are appended to all the ISR replicas and hence whose offset is within the HW are treated as "committed". Blindly follow the largest LEO may cause inconsistency between replicas and between servers and cl

Re: Lost messages during leader election

2014-07-25 Thread Jad Naous
Hi Guozhang, Yes, I think they are related. It seems odd to me that there should be any truncation at all since that is always an opportunity for data loss. It seems like we would want to avoid that at all costs, assuming we uphold the invariant that messages committed to an offset on any replica

Re: Lost messages during leader election

2014-07-25 Thread Guozhang Wang
Hi Jad, Yes. In this case I think you are actually hitting KAFKA-1211. The summary of the issue is that, it takes one more fetch request round trip for the follower replica to advance the HW after the leader has advanced HW. So for your case, the whole process is like this: 1. leader LEO at 10, f

Re: Lost messages during leader election

2014-07-25 Thread Jad Naous
Hi Guozhang, Yes, broker 1 is in the ISR (in fact, I wait for broker 1 to be in the ISR before triggering election). However, I think there is something still amiss. I still see data loss. Here are some relevant log lines from broker 0 (different test run). The log on broker 0 is getting truncated

Re: Lost messages during leader election

2014-07-25 Thread Guozhang Wang
Hello Jad, I double-checked the source code again, and found that actually the 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

Re: Lost messages during leader election

2014-07-25 Thread Jad Naous
Thank you so much for your explanation and your patience! On Fri, Jul 25, 2014 at 10:08 AM, Guozhang Wang wrote: > HW is updated as to the offset that the messages have been committed to all > replicas. This is only updated by the leader, when it receives the fetch > requests from other followe

Re: Lost messages during leader election

2014-07-25 Thread Guozhang Wang
HW is updated as to the offset that the messages have been committed to all replicas. This is only updated by the leader, when it receives the fetch requests from other follower replicas, to the position of the minimum starting offsets of the fetch requests. For producer.ack=-1, the leader will onl

Re: Lost messages during leader election

2014-07-25 Thread Jad Naous
Hi Guozhang, I apologize for my misunderstanding, I would really like to understand this thoroughly. When/how is the HW set, and how does that interact with acks being sent to the producer? Is it that the hw sets the offset for messages for which acks have been sent, and so a replica only becomes

Re: Lost messages during leader election

2014-07-25 Thread Guozhang Wang
Hi Jad, A follower replica can join ISR only when it has caught up to HW, which in this case would be the end of the leader replica. 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 sta

Re: Lost messages during leader election

2014-07-24 Thread Ashwin Jayaprakash
I'm still not sure I understand after his reply - http://qnalist.com/questions/5034216/lost-messages-during-leader-election - I really need a tutorial on Kafka. I don't understand why they made it so complicated when Cassandra and Hbase are similar but simpler. * Ashwin J

Re: Lost messages during leader election

2014-07-24 Thread Jad Naous
Actually, is the following scenario possible? - We start off with only 1 replica (the leader) - the producer continuously sends messages - a new replica (the preferred one) comes online - it becomes an ISR just after an ack is sent to the producer - the new replica gets elected as the new leader, b

Re: Lost messages during leader election

2014-07-24 Thread Jad Naous
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 to follow this up? Thanks, Jad. On Thu, Jul 24, 2014 at 6:27 PM, Guozhang Wang wrote: > With ack=-1 all messages produced to leader must have been acke

Re: Lost messages during leader election

2014-07-24 Thread Guozhang Wang
With ack=-1 all messages produced to leader must have been 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, when the curr

Re: Lost messages during leader election

2014-07-24 Thread Jad Naous
Hi Guozhang, Isn't it also possible to lose messages even if the preferred leader is in the ISR, when the current leader is ahead by a few messages, but the preferred leader still has not caught up? Thanks, Jad. On Thu, Jul 24, 2014 at 4:59 PM, Guozhang Wang wrote: > Hi Jad, > > Thanks for b

Re: Lost messages during leader election

2014-07-24 Thread Guozhang Wang
Hi Jad, Thanks for bring this up. It seems to be a valid issue: in the current auto leader rebalancer thread's logic, if the imbalance ratio threshold is violated, then it will trigger the preferred leader election whether or not the preferred leader is in ISR or not. Guozhang On Thu, Jul 24, 2

Lost messages during leader election

2014-07-24 Thread Jad Naous
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 replica for some. I have auto.leader.rebalance.enable=true, so replica election gets triggered. Data is being pumped to the old broke