Re: kafka broker loosing offsets?

2017-10-23 Thread Eric Ho
unsubscribe *Eric Ho | Qventus* Awarded Top Innovation in Cost Savings On Fri, Oct 6, 2017 at 7:52 AM, Dmitriy Vsekhvalnov wrote: > Hi

Re: kafka broker loosing offsets?

2017-10-20 Thread Dmitriy Vsekhvalnov
Hey guys, just want to post that upgrade to 0.11.0.1 solved the issue. After excessive disaster testing no re-consumption of old offsets were experienced. On Thu, Oct 12, 2017 at 1:35 AM, Vincent Dautremont < vincent.dautrem...@olamobile.com.invalid> wrote: > Hi, > We have 4 differents Kafka

Re: kafka broker loosing offsets?

2017-10-11 Thread Vincent Dautremont
Hi, We have 4 differents Kafka cluster running, 2 on 0.10.1.0 1 on 0.10.0.1 1 that was on 0.11.0.0 and last week updated to 0.11.0.1 I’ve only seen the issue happen 2 times in production usage on the cluster on 0.11.0.0 since it’s running (about 3months). But I’ll monitor and report it here if

Re: kafka broker loosing offsets?

2017-10-11 Thread Michal Michalski
Hi Dmitriy, I didn't follow the whole thread, but if it's not an issue with Kafka 0.11.0.0 (there was another thread about it recently), make sure your Replication Factor for the offsets topic is 3 (you mentioned "RF=3 for all topics", but I wasn't sure it includes the offsets one). There was a

Re: kafka broker loosing offsets?

2017-10-11 Thread Dmitriy Vsekhvalnov
Yeah just pops up in my list. Thanks, i'll take a look. Vincent Dautremont, if you still reading it, did you try upgrade to 0.11.0.1? Fixed issue? On Wed, Oct 11, 2017 at 6:46 PM, Ben Davison wrote: > Hi Dmitriy, > > Did you check out this thread "Incorrect consumer

Re: kafka broker loosing offsets?

2017-10-11 Thread Ben Davison
Hi Dmitriy, Did you check out this thread "Incorrect consumer offsets after broker restart 0.11.0.0" from Phil Luckhurst, it sounds similar. Thanks, Ben On Wed, Oct 11, 2017 at 4:44 PM Dmitriy Vsekhvalnov wrote: > Hey, want to resurrect this thread. > > Decided to do

Re: kafka broker loosing offsets?

2017-10-11 Thread Dmitriy Vsekhvalnov
Hey, want to resurrect this thread. Decided to do idle test, where no load data is produced to topic at all. And when we kill #101 or #102 - nothing happening. But when we kill #200 - consumers starts to re-consume old events from random position. Anybody have ideas what to check? I really

Re: kafka broker loosing offsets?

2017-10-09 Thread Dmitriy Vsekhvalnov
Hi tao, we had unclean leader election enabled at the beginning. But then disabled it and also reduced 'max.poll.records' value. It helped little bit. But after today's testing there is strong correlation between lag spike and what broker we crash. For lowest ID (100) broker : 1. it always at

Re: kafka broker loosing offsets?

2017-10-06 Thread tao xiao
Do you have unclean leader election turned on? If killing 100 is the only way to reproduce the problem, it is possible with unclean leader election turned on that leadership was transferred to out of ISR follower which may not have the latest high watermark On Sat, Oct 7, 2017 at 3:51 AM Dmitriy

Re: kafka broker loosing offsets?

2017-10-06 Thread Dmitriy Vsekhvalnov
About to verify hypothesis on monday, but looks like that in latest tests. Need to double check. On Fri, Oct 6, 2017 at 11:25 PM, Stas Chizhov wrote: > So no matter in what sequence you shutdown brokers it is only 1 that causes > the major problem? That would indeed be a bit

Re: kafka broker loosing offsets?

2017-10-06 Thread Stas Chizhov
So no matter in what sequence you shutdown brokers it is only 1 that causes the major problem? That would indeed be a bit weird. have you checked offsets of your consumer - right after offsets jump back - does it start from the topic start or does it go back to some random position? Have you

Re: kafka broker loosing offsets?

2017-10-06 Thread Dmitriy Vsekhvalnov
Yeah, probably we can dig around. One more observation, the most lag/re-consumption trouble happening when we kill broker with lowest id (e.g. 100 from [100,101,102]). When crashing other brokers - there is nothing special happening, lag growing little bit but nothing crazy (e.g. thousands, not

Re: kafka broker loosing offsets?

2017-10-06 Thread Stas Chizhov
Ted: when choosing earliest/latest you are saying: if it happens that there is no "valid" offset committed for a consumer (for whatever reason: bug/misconfiguration/no luck) it will be ok to start from the beginning or end of the topic. So if you are not ok with that you should choose none.

Re: kafka broker loosing offsets?

2017-10-06 Thread Ted Yu
A brief search brought me to related discussion on this JIRA: https://issues.apache.org/jira/browse/KAFKA-3806?focusedCommentId=15906349=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15906349 FYI On Fri, Oct 6, 2017 at 10:37 AM, Manikumar

Re: kafka broker loosing offsets?

2017-10-06 Thread Manikumar
@Ted Yes, I think we should add log warning message. On Fri, Oct 6, 2017 at 9:50 PM, Vincent Dautremont < vincent.dautrem...@olamobile.com.invalid> wrote: > is there a way to read messages on a topic partition from a specific node > we that we choose (and not by the topic partition leader) ? >

Re: kafka broker loosing offsets?

2017-10-06 Thread Vincent Dautremont
is there a way to read messages on a topic partition from a specific node we that we choose (and not by the topic partition leader) ? I would like to read myself that each of the __consumer_offsets partition replicas have the same consumer group offset written in it in it. On Fri, Oct 6, 2017 at

Re: kafka broker loosing offsets?

2017-10-06 Thread Dmitriy Vsekhvalnov
Stas: we rely on spring-kafka, it commits offsets "manually" for us after event handler completed. So it's kind of automatic once there is constant stream of events (no idle time, which is true for us). Though it's not what pure kafka-client calls "automatic" (flush commits at fixed intervals).

Re: kafka broker loosing offsets?

2017-10-06 Thread Stas Chizhov
You don't have autocmmit enables that means you commit offsets yourself - correct? If you store them per partition somewhere and fail to clean it up upon rebalance next time the consumer gets this partition assigned during next rebalance it can commit old stale offset- can this be the case? fre

Re: kafka broker loosing offsets?

2017-10-06 Thread Dmitriy Vsekhvalnov
Reprocessing same events again - is fine for us (idempotent). While loosing data is more critical. What are reasons of such behaviour? Consumers are never idle, always commiting, probably something wrong with broker setup then? On Fri, Oct 6, 2017 at 6:58 PM, Ted Yu wrote:

Re: kafka broker loosing offsets?

2017-10-06 Thread Ted Yu
Stas: bq. using anything but none is not really an option If you have time, can you explain a bit more ? Thanks On Fri, Oct 6, 2017 at 8:55 AM, Stas Chizhov wrote: > If you set auto.offset.reset to none next time it happens you will be in > much better position to find

Re: kafka broker loosing offsets?

2017-10-06 Thread Stas Chizhov
If you set auto.offset.reset to none next time it happens you will be in much better position to find out what happens. Also in general with current semantics of offset reset policy IMO using anything but none is not really an option unless it is ok for consumer to loose some data (latest) or

Re: kafka broker loosing offsets?

2017-10-06 Thread Ted Yu
Should Kafka log warning if log.retention.hours is lower than number of hours specified by offsets.retention.minutes ? On Fri, Oct 6, 2017 at 8:35 AM, Manikumar wrote: > normally, log.retention.hours (168hrs) should be higher than > offsets.retention.minutes (336

Re: kafka broker loosing offsets?

2017-10-06 Thread Vincent Dautremont
Hi, I'm having the same setup as Dimitry, I've experienced exactly the same issue already 2 times this last month. (the only difference with Dimitry's setup is that I have librdkafka 0.9.5 clients. It's like if the __consumer_offsets partitions were not synced but still reported as synced (and so

Re: kafka broker loosing offsets?

2017-10-06 Thread Manikumar
normally, log.retention.hours (168hrs) should be higher than offsets.retention.minutes (336 hrs)? On Fri, Oct 6, 2017 at 8:58 PM, Dmitriy Vsekhvalnov wrote: > Hi Ted, > > Broker: v0.11.0.0 > > Consumer: > kafka-clients v0.11.0.0 > auto.offset.reset = earliest > > > >

Re: kafka broker loosing offsets?

2017-10-06 Thread Dmitriy Vsekhvalnov
Hi Ted, Broker: v0.11.0.0 Consumer: kafka-clients v0.11.0.0 auto.offset.reset = earliest On Fri, Oct 6, 2017 at 6:24 PM, Ted Yu wrote: > What's the value for auto.offset.reset ? > > Which release are you using ? > > Cheers > > On Fri, Oct 6, 2017 at 7:52 AM, Dmitriy

Re: kafka broker loosing offsets?

2017-10-06 Thread Ted Yu
What's the value for auto.offset.reset ? Which release are you using ? Cheers On Fri, Oct 6, 2017 at 7:52 AM, Dmitriy Vsekhvalnov wrote: > Hi all, > > we several time faced situation where consumer-group started to re-consume > old events from beginning. Here is

kafka broker loosing offsets?

2017-10-06 Thread Dmitriy Vsekhvalnov
Hi all, we several time faced situation where consumer-group started to re-consume old events from beginning. Here is scenario: 1. x3 broker kafka cluster on top of x3 node zookeeper 2. RF=3 for all topics 3. log.retention.hours=168 and offsets.retention.minutes=20160 4. running sustainable load