[ 
https://issues.apache.org/jira/browse/KAFKA-7066?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16514953#comment-16514953
 ] 

Stephane Maarek commented on KAFKA-7066:
----------------------------------------

Thanks [~mjsax] . I think that helps, but my PR looks like it goes at the most 
common - lowest level for all these issues, which addresses all kinds of 
stores. 
With logging though, I'd rather have too much than less, so I don't think any 
issues supersedes others

> Make Streams Runtime Error User Friendly in Case of Serialisation exception
> ---------------------------------------------------------------------------
>
>                 Key: KAFKA-7066
>                 URL: https://issues.apache.org/jira/browse/KAFKA-7066
>             Project: Kafka
>          Issue Type: Improvement
>          Components: streams
>    Affects Versions: 1.1.0
>            Reporter: Stephane Maarek
>            Assignee: Stephane Maarek
>            Priority: Major
>             Fix For: 2.0.0
>
>
> This kind of exception can be cryptic for the beginner:
> {code:java}
> ERROR stream-thread 
> [favourite-colors-application-a336770d-6ba6-4bbb-8681-3c8ea91bd12e-StreamThread-1]
>  Failed to process stream task 2_0 due to the following error: 
> (org.apache.kafka.streams.processor.internals.AssignedStreamsTasks:105)
> java.lang.ClassCastException: java.lang.Long cannot be cast to 
> java.lang.String
> at 
> org.apache.kafka.common.serialization.StringSerializer.serialize(StringSerializer.java:28)
> at org.apache.kafka.streams.state.StateSerdes.rawValue(StateSerdes.java:178)
> at 
> org.apache.kafka.streams.state.internals.MeteredKeyValueBytesStore$1.innerValue(MeteredKeyValueBytesStore.java:66)
> at 
> org.apache.kafka.streams.state.internals.MeteredKeyValueBytesStore$1.innerValue(MeteredKeyValueBytesStore.java:57)
> at 
> org.apache.kafka.streams.state.internals.InnerMeteredKeyValueStore.put(InnerMeteredKeyValueStore.java:198)
> at 
> org.apache.kafka.streams.state.internals.MeteredKeyValueBytesStore.put(MeteredKeyValueBytesStore.java:117)
> at 
> org.apache.kafka.streams.kstream.internals.KTableAggregate$KTableAggregateProcessor.process(KTableAggregate.java:95)
> at 
> org.apache.kafka.streams.kstream.internals.KTableAggregate$KTableAggregateProcessor.process(KTableAggregate.java:56)
> at 
> org.apache.kafka.streams.processor.internals.ProcessorNode$1.run(ProcessorNode.java:46)
> at 
> org.apache.kafka.streams.processor.internals.StreamsMetricsImpl.measureLatencyNs(StreamsMetricsImpl.java:208)
> at 
> org.apache.kafka.streams.processor.internals.ProcessorNode.process(ProcessorNode.java:124)
> at 
> org.apache.kafka.streams.processor.internals.AbstractProcessorContext.forward(AbstractProcessorContext.java:174)
> at 
> org.apache.kafka.streams.processor.internals.SourceNode.process(SourceNode.java:80)
> at 
> org.apache.kafka.streams.processor.internals.StreamTask.process(StreamTask.java:224)
> at 
> org.apache.kafka.streams.processor.internals.AssignedStreamsTasks.process(AssignedStreamsTasks.java:94)
> at 
> org.apache.kafka.streams.processor.internals.TaskManager.process(TaskManager.java:411)
> at 
> org.apache.kafka.streams.processor.internals.StreamThread.processAndMaybeCommit(StreamThread.java:918)
> at 
> org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:798)
> at 
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:750)
> at 
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:720){code}
> We should add more detailed logging already present in SinkNode to assist the 
> user into solving this issue



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to