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: Partitions per Machine for a topic

2014-07-25 Thread Jun Rao
We do support adding partitions in an existing topic. Thanks, Jun On Fri, Jul 25, 2014 at 1:59 PM, Kashyap Mhaisekar wrote: > Great. Thanks all. But there is no easy way to change partitions once the > topic is created right? I run kafka 0.8.2 and its a difficult job to > reassign partitions.

Re: Serious Bug? Segment getting deleted as soon as it is rolled over

2014-07-25 Thread Jun Rao
Thanks for finding this out and sharing this info. Jun On Fri, Jul 25, 2014 at 12:09 PM, Kashyap Paidimarri wrote: > Solved (We're using Kafka 0.8.1 and this was caused by the bug in dynamic > topic config changes) > > Found the problem. > We had changed retention.ms for this topic to 100,000

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: Partitions per Machine for a topic

2014-07-25 Thread Kashyap Mhaisekar
Great. Thanks all. But there is no easy way to change partitions once the topic is created right? I run kafka 0.8.2 and its a difficult job to reassign partitions. On Fri, Jul 25, 2014 at 3:10 PM, Clark Haskins < chask...@linkedin.com.invalid> wrote: > You can have more partitions than machines

Re: KAFKA-1477 (authentication layer) and 0.8.2

2014-07-25 Thread Robert Heise
Could not agree more Joe.  It would be terrific to see the complimentary Authorization requirements reach the development and release stages, do you have any insight into the trajectory for that effort? Thanks Rob On Friday, July 25, 2014 12:12 PM, Joe Stein wrote: Hi Chris, glad to hear

Re: Partitions per Machine for a topic

2014-07-25 Thread Clark Haskins
You can have more partitions than machines in the cluster, you cannot however have a replication factor that is great than the number of machines in the cluster. You could easily have a topic with 100 partitions on a 3 node cluster. -Clark Clark Elliott Haskins III LinkedIn DDS Site Reliability

Re: ConsumerConnector not processing partitions on a particular kafka broker.

2014-07-25 Thread Pablo Picko
Hey Joel I can see a ConsumerFetcherThread in the dump, I include the full dump this time as an attachment in case it proves useful to you. Thanks for all the help Pablo On Fri, Jul 25, 2014 at 7:30 PM, Joel Koshy wrote: > Did you see any fetcher threads in the thread dump? If not it seem

Re: Serious Bug? Segment getting deleted as soon as it is rolled over

2014-07-25 Thread Kashyap Paidimarri
Solved (We're using Kafka 0.8.1 and this was caused by the bug in dynamic topic config changes) Found the problem. We had changed retention.ms for this topic to 100,000 (100 seconds) earlier this month (using kafka-topics.sh admin tool). Then after Kafka had purged data we proceeded to set the ret

Re: ConsumerConnector not processing partitions on a particular kafka broker.

2014-07-25 Thread Joel Koshy
Did you see any fetcher threads in the thread dump? If not it seems they may have exited for some reason and the iterators are blocked on receiving data. On Fri, Jul 25, 2014 at 12:50:13PM +0100, Pablo Picko wrote: > Hey Joel > > I actually did issue a kill -3 to get a view on the consumer at the

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: KAFKA-1477 (authentication layer) and 0.8.2

2014-07-25 Thread Chris Neal
Thanks everyone for the responses. I can do testing as well once it is available on trunk...potentially before with the custom fork as well. I did notice one thing when testing with the console-producer on the first put to a topic after connecting: The Producer reported this warning to me: [201

Re: KAFKA-1477 (authentication layer) and 0.8.2

2014-07-25 Thread Rajasekar Elango
Yes we are very much interested in getting this code merged to trunk. I can also do testing once it's available on trunk. Thanks, Raja. On Fri, Jul 25, 2014 at 12:11 PM, Joe Stein wrote: > Hi Chris, glad to hear that even more folks are going to (want to) use the > feature. I didn't author th

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: kafka support in collectd and syslog-ng

2014-07-25 Thread Steve Morin
Cool On Fri, Jul 25, 2014 at 9:25 AM, Joe Stein wrote: > Awesome! > > /*** > Joe Stein > Founder, Principal Consultant > Big Data Open Source Security LLC > http://www.stealth.ly > Twitter: @allthingshadoop

Re: kafka support in collectd and syslog-ng

2014-07-25 Thread Joe Stein
Awesome! /*** Joe Stein Founder, Principal Consultant Big Data Open Source Security LLC http://www.stealth.ly Twitter: @allthingshadoop / On Fri, Jul 25, 2014 at

Re: KAFKA-1477 (authentication layer) and 0.8.2

2014-07-25 Thread Joe Stein
Hi Chris, glad to hear that even more folks are going to (want to) use the feature. I didn't author the patch (Raja and Ivan did) and created the fork so folks could test it without much fuss. I just commented on the ticket to address Jun's last comment and think it also answers your question too

Re: kafka support in collectd and syslog-ng

2014-07-25 Thread Jay Kreps
This is great! -Jay On Fri, Jul 25, 2014 at 8:55 AM, Pierre-Yves Ritschard wrote: > Hi list, > > Just a quick note to let you know that kafka support has now been merged in > collectd, which means that system and application metrics can directly be > produced on a topic from the collectd daemon

kafka support in collectd and syslog-ng

2014-07-25 Thread Pierre-Yves Ritschard
Hi list, Just a quick note to let you know that kafka support has now been merged in collectd, which means that system and application metrics can directly be produced on a topic from the collectd daemon. Additionally, syslog-ng will soon ship with a kafka producing module as well, it will be par

KAFKA-1477 (authentication layer) and 0.8.2

2014-07-25 Thread Chris Neal
Hi guys, This JIRA (https://issues.apache.org/jira/browse/KAFKA-1477) leads me to believe that an authentication layer implementation is planned as part of the 0.8.2 release. I was wondering if this is still the case? There was an earlier thread talking about security, but there hasn't been acti

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: ConsumerConnector not processing partitions on a particular kafka broker.

2014-07-25 Thread Pablo Picko
Hey Joel I actually did issue a kill -3 to get a view on the consumer at the time of the issue. I have just found the output I had 20 threads and all of them look like the following. I think it looks Ok. 2014/07/24 00:24:03 | "pool-2-thread-20" prio=3D10 = tid=3D0x7f55f4764800 nid=3D0x76b1 wa

Re: ConsumerConnector not processing partitions on a particular kafka broker.

2014-07-25 Thread Pablo Picko
Hey Joe Thanks for the info I have found out that my logger was misconfigured I've redeployed now with the proper settings and hopefully I can catch the proper error messages. I can confirm I'm seeing the appropriate log detail now. Thanks Pablo On Thu, Jul 24, 2014 at 8:47 PM, Joe Stein wrote