Re: Occasional NPE in NamedCache

2016-10-18 Thread Damian Guy
Hi Frank, Which version of kafka are you running? The line numbers in the stack trace don't match up with what i am seeing on 0.10.1 or on trunk. FYI - I created a JIRA for this here: https://issues.apache.org/jira/browse/KAFKA-4311 Thanks, Damian On Tue, 18 Oct 2016 at 15:52 Damian Guy wrote:

Re: Occasional NPE in NamedCache

2016-10-18 Thread Damian Guy
Also, it'd be great if you could share your streams topology. Thanks, Damian On Tue, 18 Oct 2016 at 15:48 Damian Guy wrote: > Hi Frank, > > Are you able to reproduce this? I'll have a look into it, but it is not > immediately clear how it could get into this state. > > Thanks, > Damian > > > On

Re: Occasional NPE in NamedCache

2016-10-18 Thread Damian Guy
Hi Frank, Are you able to reproduce this? I'll have a look into it, but it is not immediately clear how it could get into this state. Thanks, Damian On Tue, 18 Oct 2016 at 11:08 Frank Lyaruu wrote: > I might have run into a related problem: > > [StreamThread-1] ERROR > org.apache.kafka.stream

Re: Occasional NPE in NamedCache

2016-10-18 Thread Frank Lyaruu
I might have run into a related problem: [StreamThread-1] ERROR org.apache.kafka.streams.processor.internals.StreamThread - stream-thread [StreamThread-1] Failed to close state manager for StreamTask 0_0: org.apache.kafka.streams.errors.ProcessorStateException: task [0_0] Failed to close state sto

Re: Occasional NPE in NamedCache

2016-10-13 Thread Guozhang Wang
BTW this is tracked and resolved as https://issues.apache.org/jira/browse/KAFKA-4300. On Thu, Oct 13, 2016 at 1:17 PM, Guozhang Wang wrote: > Thanks Frank for reporting the bug, and many thanks to Damian for the > quick catch! > > On Thu, Oct 13, 2016 at 12:30 PM, Frank Lyaruu wrote: > >> The i

Re: Occasional NPE in NamedCache

2016-10-13 Thread Guozhang Wang
Thanks Frank for reporting the bug, and many thanks to Damian for the quick catch! On Thu, Oct 13, 2016 at 12:30 PM, Frank Lyaruu wrote: > The issue seems to be gone. Amazing work, thanks...! > > On Thu, Oct 13, 2016 at 6:56 PM, Damian Guy wrote: > > > Hi, i believe i found the problem. If poss

Re: Occasional NPE in NamedCache

2016-10-13 Thread Frank Lyaruu
The issue seems to be gone. Amazing work, thanks...! On Thu, Oct 13, 2016 at 6:56 PM, Damian Guy wrote: > Hi, i believe i found the problem. If possible could you please try with > this: https://github.com/dguy/kafka/tree/cache-bug > > Thanks, > Damian > > On Thu, 13 Oct 2016 at 17:46 Damian Guy

Re: Occasional NPE in NamedCache

2016-10-13 Thread Damian Guy
Hi, i believe i found the problem. If possible could you please try with this: https://github.com/dguy/kafka/tree/cache-bug Thanks, Damian On Thu, 13 Oct 2016 at 17:46 Damian Guy wrote: > Hi Frank, > > Thanks for reporting. Can you provide a sample of the join you are > running? > > Thanks, > D

Re: Occasional NPE in NamedCache

2016-10-13 Thread Damian Guy
Hi Frank, Thanks for reporting. Can you provide a sample of the join you are running? Thanks, Damian On Thu, 13 Oct 2016 at 16:10 Frank Lyaruu wrote: > Hi Kafka people, > > I'm joining a bunch of Kafka Topics using Kafka Streams, with the Kafka > 0.10.1 release candidate. > > It runs ok for a

Occasional NPE in NamedCache

2016-10-13 Thread Frank Lyaruu
Hi Kafka people, I'm joining a bunch of Kafka Topics using Kafka Streams, with the Kafka 0.10.1 release candidate. It runs ok for a few thousand of messages, and then it dies with the following exception: Exception in thread "StreamThread-1" java.lang.NullPointerException at org.apache.kafka.str