Re: Kafka Streams Failed to rebalance error

2017-06-09 Thread Matthias J. Sax
Hi, I `CommitFailedException` can still occur if an instance misses a rebalance. I thinks, this is two different problems. Having said this, Streams should recover from `CommitFailedException` automatically by triggering another rebalance afterwards. Nevertheless, we know that there is an issue

Re: Kafka Streams Failed to rebalance error

2017-06-09 Thread João Peixoto
I see your point Eno, but truth is, on my real app I am getting "CommitFailedException", even though I did not change "max.poll.interval.ms" and it remains at Integer.MAX_VALUE. I'm further investigating the origin of that exception. My current working theory is that if a customer processor

Re: Kafka Streams Failed to rebalance error

2017-06-09 Thread Eno Thereska
Even without a state store the tasks themselves will get rebalanced. So definitely you'll trigger the problem with the 1.2.3. steps you describe and that is confirmed. The reason we increased "max.poll.interval.ms" to basically infinite is to just avoid this problem. Eno > On 9 Jun 2017, at

Re: Kafka Streams Failed to rebalance error

2017-06-09 Thread João Peixoto
To help out I made the project that reproduces this issue publicly available at https://github.com/Hartimer/kafka-stream-issue On Thu, Jun 8, 2017 at 11:40 PM João Peixoto wrote: > I am now able to consistently reproduce this issue with a dummy project. > > 1. Set

Re: Kafka Streams Failed to rebalance error

2017-06-09 Thread João Peixoto
I am now able to consistently reproduce this issue with a dummy project. 1. Set "max.poll.interval.ms" to a low value 2. Have the pipeline take longer than the interval above 3. Profit This happens every single time and never recovers. I simulated the delay by adding a breakpoint on my IDE on a

Re: Kafka Streams Failed to rebalance error

2017-06-07 Thread João Peixoto
But my stream definition does not have a state store at all, Rocksdb or in memory... That's the most concerning part... On Wed, Jun 7, 2017 at 9:48 PM Sachin Mittal wrote: > One instance with 10 threads may cause rocksdb issues. > What is the RAM you have? > > Also check CPU

Re: Kafka Streams Failed to rebalance error

2017-06-07 Thread Sachin Mittal
One instance with 10 threads may cause rocksdb issues. What is the RAM you have? Also check CPU wait time. Many rocks db instances on one machine (depends upon number of partitions) may cause lot of disk i/o causing wait times to increase and hence slowing down the message processing causing

Re: Kafka Streams Failed to rebalance error

2017-06-07 Thread João Peixoto
There is one instance with 10 threads. On Wed, Jun 7, 2017 at 9:07 PM Guozhang Wang wrote: > João, > > Do you also have multiple running instances in parallel, and how many > threads are your running within each instance? > > Guozhang > > > On Wed, Jun 7, 2017 at 3:18 PM,

Re: Kafka Streams Failed to rebalance error

2017-06-07 Thread Guozhang Wang
João, Do you also have multiple running instances in parallel, and how many threads are your running within each instance? Guozhang On Wed, Jun 7, 2017 at 3:18 PM, João Peixoto wrote: > Eno before I do so I just want to be sure this would not be a duplicate. I > just

Re: Kafka Streams Failed to rebalance error

2017-06-07 Thread Eno Thereska
Hi there, This might be a bug, would you mind opening a JIRA (copy-pasting below is sufficient). Thanks Eno > On 7 Jun 2017, at 21:38, João Peixoto wrote: > > I'm using Kafka Streams 0.10.2.1 and I still see this error > > 2017-06-07 20:28:37.211 WARN 73 --- [

Re: Kafka Streams Failed to rebalance error

2017-06-07 Thread João Peixoto
I'm using Kafka Streams 0.10.2.1 and I still see this error 2017-06-07 20:28:37.211 WARN 73 --- [ StreamThread-1] o.a.k.s.p.internals.StreamThread : Could not create task 0_31. Will retry. org.apache.kafka.streams.errors.LockException: task [0_31] Failed to lock the state directory for

Re: Kafka Streams Failed to rebalance error

2017-05-16 Thread Matthias J. Sax
Great! :) On 5/16/17 2:31 AM, Sameer Kumar wrote: > I see now that my Kafka cluster is very stable, and these errors dont come > now. > > -Sameer. > > On Fri, May 5, 2017 at 7:53 AM, Sameer Kumar wrote: > >> Yes, I have upgraded my cluster and client both to version

Re: Kafka Streams Failed to rebalance error

2017-05-16 Thread Sameer Kumar
I see now that my Kafka cluster is very stable, and these errors dont come now. -Sameer. On Fri, May 5, 2017 at 7:53 AM, Sameer Kumar wrote: > Yes, I have upgraded my cluster and client both to version 10.2.1 and > currently monitoring the situation. > Will report back

Re: Kafka Streams Failed to rebalance error

2017-05-04 Thread Sameer Kumar
Yes, I have upgraded my cluster and client both to version 10.2.1 and currently monitoring the situation. Will report back in case I find any errors. Thanks for the help though. -Sameer. On Fri, May 5, 2017 at 3:37 AM, Matthias J. Sax wrote: > Did you see Eno's reply? >

Re: Kafka Streams Failed to rebalance error

2017-05-04 Thread Matthias J. Sax
Did you see Eno's reply? Please try out Streams 0.10.2.1 -- this should be fixed there. If not, please report back. I would also recommend to subscribe to the list. It's self-service http://kafka.apache.org/contact -Matthias On 5/3/17 10:49 PM, Sameer Kumar wrote: > My brokers are on version

Re: Kafka Streams Failed to rebalance error

2017-05-03 Thread Sameer Kumar
My brokers are on version 10.1.0 and my clients are on version 10.2.0. Also, do a reply to all, I am currently not subscribed to the list. -Sameer. On Wed, May 3, 2017 at 6:34 PM, Sameer Kumar wrote: > Hi, > > > > I ran two nodes in my streams compute cluster, they were

Re: Kafka Streams Failed to rebalance error

2017-05-03 Thread Eno Thereska
Hi, Which version of Kafka are you using? This should be fixed in 0.10.2.1, any chance you could try that release? Thanks Eno > On 3 May 2017, at 14:04, Sameer Kumar wrote: > > Hi, > > > I ran two nodes in my streams compute cluster, they were running fine for few