Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-05-17 Thread Guozhang Wang
Sachin, We are discussing how to work around KAFKA-4740 for poison pill records: https://issues.apache.org/jira/browse/KAFKA-5157 And Please share your scenario and your opinions on the solution there. Guozhang On Tue, May 16, 2017 at 9:50 PM, Sachin Mittal wrote: > Folks is there any updat

Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-05-16 Thread Sachin Mittal
Folks is there any updated on https://issues.apache.org/jira/browse/KAFKA-4740. This is now so far only pressing issue for our streams application. It happens from time to time and so far our only solution is to increase the offset of the group for that partition beyond the offset that caused this

Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-03-30 Thread Damian Guy
Not that i'm aware of On Thu, 30 Mar 2017 at 16:00 Sachin Mittal wrote: > Damian, > Is there any way where I can just dump out the contents at a given offset > from a given log segment file. > > I am not sure how DumpLogSegment > < > https://cwiki.apache.org/confluence/display/KAFKA/System+Tools

Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-03-30 Thread Sachin Mittal
Damian, Is there any way where I can just dump out the contents at a given offset from a given log segment file. I am not sure how DumpLogSegment helps. I already know the log segment file where that messag

Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-03-30 Thread Sachin Mittal
I am not sure if https://issues.apache.org/jira/browse/KAFKA-4740 is same issue as mine. What I suspect may be happening is that: at java.nio.Buffer.limit(Buffer.java:275) ~[na:1.8.0_122-ea] at org.apache.kafka.common.utils.Utils.sizeDelimited(Utils.java:791) ~[kafka-clients-0.10.2.0.jar:na]

Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-03-30 Thread Damian Guy
Sachin, Not sure if this will help, but you might want to try running https://cwiki.apache.org/confluence/display/KAFKA/System+Tools#SystemTools-DumpLogSegment on the partition that is causing you problems. Thanks Damian On Thu, 30 Mar 2017 at 14:29 Michael Noll wrote: > Sachin, > > there's a

Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-03-30 Thread Michael Noll
Sachin, there's a JIRA that seems related to what you're seeing: https://issues.apache.org/jira/browse/KAFKA-4740 Perhaps you could check the above and report back? -Michael On Thu, Mar 30, 2017 at 3:23 PM, Michael Noll wrote: > Hmm, I re-read the stacktrace again. It does look like the va

Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-03-30 Thread Michael Noll
Hmm, I re-read the stacktrace again. It does look like the value-side being the culprit (as Sachin suggested earlier). -Michael On Thu, Mar 30, 2017 at 3:18 PM, Michael Noll wrote: > Sachin, > > you have this line: > > > builder.stream(Serdes.String(), serde, "advice-stream") > > Could the pro

Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-03-30 Thread Michael Noll
Sachin, you have this line: > builder.stream(Serdes.String(), serde, "advice-stream") Could the problem be that not the record values are causing the problem -- because your value deserializer does try-catch any such errors -- but that the record *keys* are malformed? The built-in `Serdes.Strin

Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-03-30 Thread Sachin Mittal
My streams application does run in debug mode only. Also I have checked the code around these lines at org.apache.kafka.common.utils.Utils.sizeDelimited(Utils.java:791) ~[kafka-clients-0.10.2.0.jar:na] at org.apache.kafka.common.record.Record.value(Record.java:268) ~[kafka-clients-0.10.2.0.jar

Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-03-30 Thread Damian Guy
The suggestions in that FAQ won't help as it is too late, i.e., the message has already been received into Streams. You could create a simple app that uses the Consumer, seeks to the offset, and tries to read the message. If you did this in debug mode you might find out some more information. On

Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-03-30 Thread Sachin Mittal
Well I try to read that offset via kafka-console-consumer.sh too and it fails with same error. So was wondering if I can apply any of the suggestion as per http://docs.confluent.io/3.2.0/streams/faq.html#handling-corrupted-records-and-deserialization-errors-poison-pill-messages If there is any ot

Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-03-30 Thread Damian Guy
Hi Sachin, Have you tried firing up a consumer (non-streams), seeking to that offset on the topic and seeing what the message is? Might be easier to debug? Like you say, it is failing in the consumer. Thanks, Damian On Thu, 30 Mar 2017 at 10:35 Sachin Mittal wrote: > I think I am following the

Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-03-30 Thread Sachin Mittal
I think I am following the third option. My pipeline is: serde= Serdes.serdeFrom(new VSerializer(), new VDeserializer ()); builder.stream(Serdes.String(), serde, "advice-stream") .filter(new Predicate() { ...}) .groupByKey() .aggregate(new Initializer() {...}, new Aggregator() {...}, windo

Re: weird SerializationException when consumer is fetching and parsing record in streams application

2017-03-30 Thread Michael Noll
Could this be a corrupted message ("poison pill") in your topic? If so, take a look at http://docs.confluent.io/current/streams/faq.html#handling-corrupted-records-and-deserialization-errors-poison-pill-messages FYI: We're currently investigating a more elegant way to address such poison pill pro