Re: [VOTE] 0.10.2.0 RC1

2017-02-14 Thread Ian Duffy
Hi Matthias and Guozhang, +1 (non-binding) We eventually solved this on our end. It boiled down to dodgy data and unexpected hidden errors in our processing steps. This eventually resulted in a session time out which triggered the rebalances. Thanks again for all your help. Ian. On 14 February

Re: [VOTE] 0.10.2.0 RC1

2017-02-14 Thread Moczarski, Swen
Thanks a lot for the fast fix! I tested my code with the fix and it works fine without exception. Regards, Swen Am 2/13/17, 8:38 PM schrieb "Guozhang Wang" : Thanks for reporting the JIRA Swen. Jason has a patch ready under KAFKA-4761 and I have reviewed it. You could try it ou

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Guozhang Wang
Hi Ian, Thanks for the responses. Another suggestion I had is about your session.timeout.ms config value (currently set to 10 seconds): since Nina meaned that a processing of a single record could be as long as minutes, that session timeout would be insufficient as no heartbeats will be sent by th

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Matthias J. Sax
Ian, an important hint: it is highly recommended to change "state.dir" configuration parameter from "/tmp/kafka-streams" to a different directory. It might be, that /tmp gets deleted and thus you loose all your cached data. While this is no an issues with regard to data loss (as all data is relia

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Ian Duffy
Hi Guozhang, Thank you for your assistance on this. > About the stack trace pasted before: is it tailing some warning logs like "Could not create task ... Will retry" Yes, we see the following: 17/02/13 21:49:55 WARN internals.StreamThread: Could not create task 0_93. Will retry. org.apache.kaf

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Guozhang Wang
@Ian, Nina Thanks for the detailed description of your apps. A couple of follow-up questions I have to help us further investigate your issue: 1. About the stack trace pasted before: is it tailing some warning logs like "Could not create task ... Will retry" (i.e. it is part of that warning log)

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Guozhang Wang
Thanks for reporting the JIRA Swen. Jason has a patch ready under KAFKA-4761 and I have reviewed it. You could try it out and see if it has fixed your issue. After this is merged in, we will need another RC. Guozhang On Mon, Feb 13, 2017 at 9:52 AM, Moczarski, Swen < smoczar...@ebay-kleinanzei

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Moczarski, Swen
+0 (non-binding) Thanks for compiling a new release candidate. I get an NullPointerException when setting batch.size=0 on producer config. This worked before with 0.10.1.1. See https://issues.apache.org/jira/browse/KAFKA-4761 Regards, Swen Am 2/10/17, 5:51 PM schrieb "Ewen Cheslack-Postava" :

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Nina Hanzlikova
Hi, I am a colleague of Ian's. We use the following processing pipeline in stream app he mentions: https://github.com/zalando-incubator/pipeline-backbone The streams are built using: object Run extends App { // ... private val latch = new CountDownLatch(1) private val builder = { val

Re: [VOTE] 0.10.2.0 RC1

2017-02-13 Thread Eno Thereska
+1 (non binding) Checked streams. Verified that stream tests work and examples off confluentinc/examples/kafka-streams work. Thanks Eno > On 10 Feb 2017, at 16:51, Ewen Cheslack-Postava wrote: > > Hello Kafka users, developers and client-developers, > > This is RC1 for release of Apache Kafk

Re: [VOTE] 0.10.2.0 RC1

2017-02-12 Thread Guozhang Wang
Hello Ian, I looked at the releasing artifact and tried running a simple streams app with multiple threads with the jar but cannot reproduce your issue. If you are still blocked on this I'm wondering if you could provide some code sketch without any app business logic for me to try to reproduce it

Re: [VOTE] 0.10.2.0 RC1

2017-02-12 Thread Ian Duffy
Hi Matthias, Thank you for your fast response. I am not using any custom class loaders and it is the 10.2 jar that is being used. I'll try clearing out the state on next failure. The config parameters we are setting are: consumer: heartbeat.interval.ms = "100" auto.offset.reset = "earli

Re: [VOTE] 0.10.2.0 RC1

2017-02-10 Thread Matthias J. Sax
Hi Ian, thanks for reporting this. I had a look at the stack trace and code and the whole situation is quite confusing. The exception itself is expected but we have a try-catch-block that should swallow the exception and it should never bubble up: In AbstractTaskCreator.retryWithBackoff a call

Re: [VOTE] 0.10.2.0 RC1

2017-02-10 Thread Ian Duffy
Seeing the following failure when using multi-threaded streams Feb 10 17:21:15 ip-172-31-137-57 docker/43e65fe123cd[826]: org.apache.kafka.streams.errors.LockException: task [0_21] Failed to lock the state directory: /tmp/kafka-streams/text_pipeline_id/0_21 Feb 10 17:21:15 ip-172-31-137-57 docker/

[VOTE] 0.10.2.0 RC1

2017-02-10 Thread Ewen Cheslack-Postava
Hello Kafka users, developers and client-developers, This is RC1 for release of Apache Kafka 0.10.2.0. This is a minor version release of Apache Kafka. It includes 19 new KIPs. See the release notes and release plan (https://cwiki.apache.org/ confluence/display/KAFKA/Release+Plan+0.10.2.0) for mo