Replying to Avi's and Matthias' questions in one go inline:

> On 25 May 2017, at 19:27, Matthias J. Sax <matth...@confluent.io> wrote:
> 
> Thanks for the KIP Eno!
> 
> Couple of comments:
> 
> I think we don't need `RecordContext` in `RecordExceptionHandler#handle`
> because the `ConsumerRecord` provides all this information anyway.

Good point, fixed.

> 
> Why we introduce `ExceptionType` and not just hand in the actual exception?
> 

Ok, changed.

> As return type of `handle()` is void, how would the handler fail? By
> throwing an exception? Maybe it would be better to add a proper return
> type from the beginning on -- this might also make backward
> compatibility easier later on.

Added two options to the KIP, see what you think (Continue/Fail). If we add 
Retry later on, not sure if that'd trigger a KIP.

> 
> Question about `LogAndThresholdExceptionHandler` -- how would we be able
> to track this?

I'm thinking StreamsThread instantiates one such handler that keeps a count of 
failed messages. The handler will need to know the total of successful messages 
too though, and come to think of it some more, we don't have a metric that 
keeps that. We can either add that metric (in JMX) or alternatively change the 
threshold handler to respond to number of failures, instead of %. Preferences?
> 
> 
> With regard to `DeserializationException`, do you thing it might make
> sense to have a "dead letter queue" as a feature to provide out-of-the-box?

We could provide a special topic where bad messages go to, and then we'd have 
to add a config option for the user to provide a topic. Is that what you're 
thinking?

Eno


> 
> 
> -Matthias
> 
> On 5/25/17 2:47 AM, Eno Thereska wrote:
>> Hi there,
>> 
>> I’ve added a KIP on improving exception handling in streams:
>> KIP-161: streams record processing exception handlers. 
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-161%3A+streams+record+processing+exception+handlers
>>  
>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-161:+streams+record+processing+exception+handlers>
>> 
>> Discussion and feedback is welcome, thank you.
>> Eno
>> 
> 

Reply via email to