One more after thought: should we add a metric for this? We also have a
metric for `skippedDueToDeserializationError-rate` ?


-Matthias



On 12/6/17 7:54 AM, Bill Bejeck wrote:
> Thanks for the clearly written KIP, no further comments from my end.
> 
> -Bill
> 
> On Wed, Dec 6, 2017 at 9:52 AM, Matt Farmer <m...@frmr.me> wrote:
> 
>> There is already a vote thread for this KIP. I can bump it so that it’s
>> towards the top of your inbox.
>>
>> With regard to your concerns:
>>
>> 1) We do not have the "ProductionExceptionHandler" interface defined in the
>> wiki page, thought it is sort of clear that it is a one-function interface
>> with record and exception. Could you add it?
>>
>>
>> It is defined, it’s just not defined using a code snippet. The KIP reads as
>> follows:
>>
>> ===
>>
>> A public interface named ProductionExceptionHandler with a single method,
>> handle, that has the following signature:
>>
>>    - ProductionExceptionHandlerResponse handle(ProducerRecord<byte[],
>>    byte[]> record, Exception exception)
>>
>>
>> ===
>>
>> If you’d like me to add a code snippet illustrating this that’s simple for
>> me to do, but it seemed superfluous.
>>
>> 2) A quick question about your example code: where would be the "logger"
>> object be created?
>>
>>
>> SLF4J loggers are typically created as a class member in the class. Such
>> as:
>>
>> private Logger logger = LoggerFactory.getLogger(HelloWorld.class);
>>
>> I omit that in my implementation examples for brevity.
>>
>> On December 6, 2017 at 2:14:58 AM, Guozhang Wang (wangg...@gmail.com)
>> wrote:
>>
>> Hello Matt,
>>
>> Thanks for writing up the KIP. I made a pass over it and here is a few
>> minor comments. I think you can consider starting a voting thread for this
>> KIP while addressing them.
>>
>> 1) We do not have the "ProductionExceptionHandler" interface defined in the
>> wiki page, thought it is sort of clear that it is a one-function interface
>> with record and exception. Could you add it?
>>
>> 2) A quick question about your example code: where would be the "logger"
>> object be created? Note that the implementation of this interface have to
>> give a non-param constructor, or as a static field of the class but in that
>> case you would not be able to log which instance is throwing this error (we
>> may have multiple producers within a single instance, even within a
>> thread). Just a reminder to consider in your implementation.
>>
>>
>> Guozhang
>>
>> On Tue, Dec 5, 2017 at 3:15 PM, Matthias J. Sax <matth...@confluent.io>
>> wrote:
>>
>>> Thanks a lot for the update! Great write-up! Very clearly explained what
>>> the change will look like!
>>>
>>> Looks good to me. No further comments from my side.
>>>
>>>
>>> -Matthias
>>>
>>>
>>> On 12/5/17 9:14 AM, Matt Farmer wrote:
>>>> I have updated this KIP accordingly.
>>>>
>>>> Can you please take a look and let me know if what I wrote looks
>> correct
>>> to
>>>> you?
>>>>
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>> 210+-+Provide+for+custom+error+handling++when+Kafka+
>>> Streams+fails+to+produce
>>>>
>>>> Thanks!
>>>>
>>>> Matt
>>>>
>>>>
>>>> On December 4, 2017 at 9:39:13 PM, Matt Farmer (m...@frmr.me) wrote:
>>>>
>>>> Hey Matthias, thanks for getting back to me.
>>>>
>>>> That's fine. But if we add it to `test` package, we don't need to talk
>>>> about it in the KIP. `test` is not public API.
>>>>
>>>> Yes, that makes sense. It was in the KIP originally because I was, at
>> one
>>>> point, planning on including it. We can remove it now that we’ve
>> decided
>>> we
>>>> won’t include it in the public API.
>>>>
>>>> Understood. That makes sense. We should explain this clearly in the KIP
>>>> and maybe log all other following exceptions at DEBUG level?
>>>>
>>>>
>>>> I thought it was clear in the KIP, but I can go back and double check
>> my
>>>> wording and revise it to try and make it clearer.
>>>>
>>>> I’ll take a look at doing more work on the KIP and the Pull Request
>>>> tomorrow.
>>>>
>>>> Thanks again!
>>>>
>>>> On December 4, 2017 at 5:50:33 PM, Matthias J. Sax (
>>> matth...@confluent.io)
>>>> wrote:
>>>>
>>>> Hey,
>>>>
>>>> About your questions:
>>>>
>>>>>>> Acknowledged, so is ProducerFencedException the only kind of
>>> exception I
>>>>>>> need to change my behavior on? Or are there other types I need to
>>>> check? Is
>>>>>>> there a comprehensive list somewhere?
>>>>
>>>> I cannot think if any other atm. We should list all fatal exceptions
>> for
>>>> which we don't call the handler and explain why (exception is "global"
>>>> and will affect all other records, too | ProducerFenced is
>> self-healing).
>>>>
>>>> We started to collect and categorize exception here (not completed
>> yet):
>>>> https://cwiki.apache.org/confluence/display/KAFKA/
>>> Kafka+Streams+Architecture#KafkaStreamsArchitecture-TypesofExceptions
>>>> :
>>>>
>>>> This list should be a good starting point though.
>>>>
>>>>> I include it in the test package because I have tests that assert that
>>> if
>>>>> the record collector impl encounters an Exception and receives a
>>> CONTINUE
>>>>> that it actually does CONTINUE.
>>>>
>>>> That's fine. But if we add it to `test` package, we don't need to talk
>>>> about it in the KIP. `test` is not public API.
>>>>
>>>>> I didn't want to invoke the handler in places where the CONTINUE or
>> FAIL
>>>>> result would just be ignored. Presumably, after a FAIL has been
>>> returned,
>>>>> subsequent exceptions are likely to be repeats or noise from my
>>>>> understanding of the code paths. If this is incorrect we can revisit.
>>>>
>>>> Understood. That makes sense. We should explain this clearly in the KIP
>>>> and maybe log all other following exceptions at DEBUG level?
>>>>
>>>>
>>>> -Matthias
>>>>
>>>>
>>>> On 12/1/17 11:43 AM, Matt Farmer wrote:
>>>>> Bump! It's been three days here and I haven't seen any further
>> feedback.
>>>>> Eager to get this resolved, approved, and merged. =)
>>>>>
>>>>> On Tue, Nov 28, 2017 at 9:53 AM Matt Farmer <m...@frmr.me> wrote:
>>>>>
>>>>>> Hi there, sorry for the delay in responding. Last week had a holiday
>>> and
>>>>>> several busy work days in it so I'm just now getting around to
>>>> responding.
>>>>>>
>>>>>>> We would only exclude
>>>>>>> exception Streams can handle itself (like ProducerFencedException)
>> --
>>>>>>> thus, if the handler has code to react to this, it would not be bad,
>>> as
>>>>>>> this code is just never called.
>>>>>> [...]
>>>>>>> Thus, I am still in favor of not calling the
>>> ProductionExceptionHandler
>>>>>>> for fatal exception.
>>>>>>
>>>>>> Acknowledged, so is ProducerFencedException the only kind of
>> exception
>>> I
>>>>>> need to change my behavior on? Or are there other types I need to
>>> check?
>>>> Is
>>>>>> there a comprehensive list somewhere?
>>>>>>
>>>>>>> About the "always continue" case. Sounds good to me to remove it
>> (not
>>>>>>> sure why we need it in test package?)
>>>>>>
>>>>>> I include it in the test package because I have tests that assert
>> that
>>> if
>>>>>> the record collector impl encounters an Exception and receives a
>>> CONTINUE
>>>>>> that it actually does CONTINUE.
>>>>>>
>>>>>>> What is there reasoning for invoking the handler only for the first
>>>>>>> exception?
>>>>>>
>>>>>> I didn't want to invoke the handler in places where the CONTINUE or
>>> FAIL
>>>>>> result would just be ignored. Presumably, after a FAIL has been
>>> returned,
>>>>>> subsequent exceptions are likely to be repeats or noise from my
>>>>>> understanding of the code paths. If this is incorrect we can revisit.
>>>>>>
>>>>>> Once I get the answers to these questions I can make another pass on
>>> the
>>>>>> pull request!
>>>>>>
>>>>>> Matt
>>>>>>
>>>>>> On Mon, Nov 20, 2017 at 4:07 PM Matthias J. Sax <
>> matth...@confluent.io
>>>>
>>>>>> wrote:
>>>>>>
>>>>>>> Thanks for following up!
>>>>>>>
>>>>>>> One thought about an older reply from you:
>>>>>>>
>>>>>>>>>>> I strongly disagree here. The purpose of this handler isn't
>> *just*
>>>> to
>>>>>>>>>>> make a decision for streams. There may also be desirable side
>>>>>>> effects that
>>>>>>>>>>> users wish to cause when production exceptions occur. There may
>> be
>>>>>>>>>>> side-effects that they wish to cause when
>> AuthenticationExceptions
>>>>>>> occur,
>>>>>>>>>>> as well. We should still give them the hooks to preform those
>> side
>>>>>>> effects.
>>>>>>>
>>>>>>> And your follow up:
>>>>>>>
>>>>>>>>> - I think I would rather invoke it for all exceptions that could
>>>>>>> occur
>>>>>>>>> from attempting to produce - even those exceptions were returning
>>>>>>> CONTINUE
>>>>>>>>> may not be a good idea (e.g. Authorization exception). Until there
>>>>>>> is a
>>>>>>>>> different type for exceptions that are totally fatal (for example
>> a
>>>>>>>>> FatalStreamsException or some sort), maintaining a list of
>>>>>>> exceptions that
>>>>>>>>> you can intercept with this handler and exceptions you cannot
>> would
>>>>>>> be
>>>>>>>>> really error-prone and hard to keep correct.
>>>>>>>
>>>>>>> I understand what you are saying, however, consider that Streams
>> needs
>>>>>>> to die for a fatal exception. Thus, if you call the handler for a
>>> fatal
>>>>>>> exception, we would need to ignore the return value and fail -- this
>>>>>>> seems to be rather intuitive. Furthermore, users can register an
>>>>>>> uncaught-exception-handler and side effects for fatal errors can be
>>>>>>> triggered there.
>>>>>>>
>>>>>>> Btw: an AuthorizationException is fatal -- not sure what you mean by
>>> an
>>>>>>> "totally fatal" exception -- there is no superlative to fatal from
>> my
>>>>>>> understanding.
>>>>>>>
>>>>>>> About maintaining a list of exceptions: I don't think this is too
>>> hard,
>>>>>>> and users also don't need to worry about it IMHO. We would only
>>> exclude
>>>>>>> exception Streams can handle itself (like ProducerFencedException)
>> --
>>>>>>> thus, if the handler has code to react to this, it would not be bad,
>>> as
>>>>>>> this code is just never called. In case that there is an exception
>>>>>>> Streams could actually handle but we still call the handler (ie,
>> bug),
>>>>>>> there should be no harm either -- the handler needs to return either
>>>>>>> CONTINUE or FAIL and we would obey. It could only happen, that
>> Streams
>>>>>>> dies---as request by the user(!)---even if Streams could actually
>>> handle
>>>>>>> the error and move on. But this seems to be not a issue from my
>> point
>>> of
>>>>>>> view.
>>>>>>>
>>>>>>> Thus, I am still in favor of not calling the
>>> ProductionExceptionHandler
>>>>>>> for fatal exception.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> About the "always continue" case. Sounds good to me to remove it
>> (not
>>>>>>> sure why we need it in test package?) and to rename the "failing"
>>>>>>> handler to "Default" (even if "default" is less descriptive and I
>>> would
>>>>>>> still prefer "Fail" in the name).
>>>>>>>
>>>>>>>
>>>>>>> Last question:
>>>>>>>
>>>>>>>>> - Continue to *only* invoke it on the first exception that we
>>>>>>>>> encounter (before sendException is set)
>>>>>>>
>>>>>>>
>>>>>>> What is there reasoning for invoking the handler only for the first
>>>>>>> exception?
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>> On 11/20/17 10:43 AM, Matt Farmer wrote:
>>>>>>>> Alright, here are some updates I'm planning to make after thinking
>> on
>>>>>>> this
>>>>>>>> for awhile:
>>>>>>>>
>>>>>>>> - Given that the "always continue" handler isn't something I'd
>>>>>>> recommend
>>>>>>>> for production use as is, I'm going to move it into the test
>>>>>>> namespace and
>>>>>>>> remove it from mention in the public API.
>>>>>>>> - I'm going to rename the "AlwaysFailProductionExceptionHandler"
>> to
>>>>>>>> "DefaultProductionExceptionHandler"
>>>>>>>> - Given that the API for the exception handler involves being
>>>>>>> invoked by
>>>>>>>> streams to make a decision about whether or not to continue — I
>>>>>>> think that
>>>>>>>> we should:
>>>>>>>> - Continue to *only* invoke it on the first exception that we
>>>>>>>> encounter (before sendException is set)
>>>>>>>> - Stop invoking it for the self-healing fenced exceptions.
>>>>>>>> - I think I would rather invoke it for all exceptions that could
>>>>>>> occur
>>>>>>>> from attempting to produce - even those exceptions were returning
>>>>>>> CONTINUE
>>>>>>>> may not be a good idea (e.g. Authorization exception). Until there
>>>>>>> is a
>>>>>>>> different type for exceptions that are totally fatal (for example a
>>>>>>>> FatalStreamsException or some sort), maintaining a list of
>>>>>>> exceptions that
>>>>>>>> you can intercept with this handler and exceptions you cannot would
>>>>>>> be
>>>>>>>> really error-prone and hard to keep correct.
>>>>>>>> - I'm happy to file a KIP for the creation of this new Exception
>>>>>>> type
>>>>>>>> if there is interest.
>>>>>>>>
>>>>>>>> @ Matthias — What do you think about the above?
>>>>>>>>
>>>>>>>> On Tue, Nov 14, 2017 at 9:44 AM Matt Farmer <m...@frmr.me> wrote:
>>>>>>>>
>>>>>>>>> I responded before reading your code review and didn't see the bit
>>>>>>> about
>>>>>>>>> how ProducerFencedException is self-healing. This error handling
>>> logic
>>>>>>> is
>>>>>>>>> *quite* confusing to reason about... I think I'm going to sit down
>>>> with
>>>>>>>>> the code a bit more today, but I'm inclined to think that if the
>>>> fenced
>>>>>>>>> exceptions are, indeed, self healing that we still invoke the
>>> handler
>>>>>>> but
>>>>>>>>> ignore its result for those exceptions.
>>>>>>>>>
>>>>>>>>> On Tue, Nov 14, 2017 at 9:37 AM Matt Farmer <m...@frmr.me> wrote:
>>>>>>>>>
>>>>>>>>>> Hi there,
>>>>>>>>>>
>>>>>>>>>> Following up here...
>>>>>>>>>>
>>>>>>>>>>> One tiny comment: I would prefer to remove the "Always" from the
>>>>>>>>>> handler implementation names -- it sounds "cleaner" to me without
>>> it.
>>>>>>>>>>
>>>>>>>>>> Let me think on this. I generally prefer expressiveness to
>>>> clean-ness,
>>>>>>>>>> and I think that comes out in the names I chose for things. The
>>> straw
>>>>>>> man
>>>>>>>>>> in my head is the person that tried to substitute in the
>>>>>>> "AlwaysContinue"
>>>>>>>>>> variant and the "Always" is a trigger to really consider whether
>> or
>>>>>>> not
>>>>>>>>>> they *always* want to try to continue.
>>>>>>>>>>
>>>>>>>>>> To be truthful, after some thought, using the "AlwaysContinue"
>>>> variant
>>>>>>>>>> isn't something I'd recommend generally in a production system.
>>>>>>> Ideally
>>>>>>>>>> these handlers are targeted at handling a specific series of
>>>>>>> exceptions
>>>>>>>>>> that a user wants to ignore, and not ignoring all exceptions.
>> More
>>> on
>>>>>>> this
>>>>>>>>>> in a minute.
>>>>>>>>>>
>>>>>>>>>>> For the first category, it seems to not make sense to call the
>>>>>>> handle but
>>>>>>>>>> Streams should always fail. If we follow this design, the KIP
>>> should
>>>>>>> list
>>>>>>>>>> all exceptions for which the handler is not called.
>>>>>>>>>>
>>>>>>>>>> I strongly disagree here. The purpose of this handler isn't
>> *just*
>>> to
>>>>>>>>>> make a decision for streams. There may also be desirable side
>>> effects
>>>>>>> that
>>>>>>>>>> users wish to cause when production exceptions occur. There may
>> be
>>>>>>>>>> side-effects that they wish to cause when
>> AuthenticationExceptions
>>>>>>> occur,
>>>>>>>>>> as well. We should still give them the hooks to preform those
>> side
>>>>>>> effects.
>>>>>>>>>>
>>>>>>>>>> In light of the above, I'm thinking that the
>>>>>>>>>> "AlwaysContinueProductionExceptionHandler" variant should
>>> probably be
>>>>>>>>>> removed from the public API and moved into tests since that's
>> where
>>>>>>> it's
>>>>>>>>>> most useful. The more I think on it, the more I feel that having
>>> that
>>>>>>> in
>>>>>>>>>> the public API is a landmine. If you agree, then, we can rename
>> the
>>>>>>>>>> "AlwaysFailProductionExceptionHandler" to
>>>>>>>>>> "DefaultProductionExceptionHandler".
>>>>>>>>>>
>>>>>>>>>> Thoughts?
>>>>>>>>>>
>>>>>>>>>> On Fri, Nov 10, 2017 at 6:13 PM Matthias J. Sax <
>>>>>>> matth...@confluent.io>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> I just review the PR, and there is one thing we should discuss.
>>>>>>>>>>>
>>>>>>>>>>> There are different types of exceptions that could occur. Some
>>> apply
>>>>>>> to
>>>>>>>>>>> all records (like Authorization exception) while others are for
>>>>>>> single
>>>>>>>>>>> records only (like record too large).
>>>>>>>>>>>
>>>>>>>>>>> For the first category, it seems to not make sense to call the
>>>> handle
>>>>>>>>>>> but Streams should always fail. If we follow this design, the
>> KIP
>>>>>>> should
>>>>>>>>>>> list all exceptions for which the handler is not called.
>>>>>>>>>>>
>>>>>>>>>>> WDYT?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> -Matthias
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On 11/10/17 2:56 PM, Matthias J. Sax wrote:
>>>>>>>>>>>> Just catching up on this KIP.
>>>>>>>>>>>>
>>>>>>>>>>>> One tiny comment: I would prefer to remove the "Always" from
>> the
>>>>>>>>>>> handler
>>>>>>>>>>>> implementation names -- it sounds "cleaner" to me without it.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>
>>>>>>>>>>>> On 11/5/17 12:57 PM, Matt Farmer wrote:
>>>>>>>>>>>>> It is agreed, then. I've updated the pull request. I'm trying
>> to
>>>>>>> also
>>>>>>>>>>>>> update the KIP accordingly, but cwiki is being slow and
>> dropping
>>>>>>>>>>>>> connections..... I'll try again a bit later but please
>> consider
>>>> the
>>>>>>>>>>> KIP
>>>>>>>>>>>>> updated for all intents and purposes. heh.
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Sun, Nov 5, 2017 at 3:45 PM Guozhang Wang <
>>> wangg...@gmail.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> That makes sense.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Sun, Nov 5, 2017 at 12:33 PM, Matt Farmer <m...@frmr.me>
>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Interesting. I'm not sure I agree. I've been bitten many
>> times
>>>> by
>>>>>>>>>>>>>>> unintentionally shipping code that fails to properly
>> implement
>>>>>>>>>>> logging. I
>>>>>>>>>>>>>>> always discover this at the exact *worst* moment, too.
>>> (Normally
>>>>>>> at
>>>>>>>>>>> 3 AM
>>>>>>>>>>>>>>> during an on-call shift. Hah.) However, if others feel the
>>> same
>>>>>>> way
>>>>>>>>>>> I
>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>> probably be convinced to remove it.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> We could also meet halfway and say that when a customized
>>>>>>>>>>>>>>> ProductionExceptionHandler instructs Streams to CONTINUE, we
>>> log
>>>>>>> at
>>>>>>>>>>> DEBUG
>>>>>>>>>>>>>>> level instead of WARN level. Would that alternative be
>>> appealing
>>>>>>> to
>>>>>>>>>>> you?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Sun, Nov 5, 2017 at 12:32 PM Guozhang Wang <
>>>>>>> wangg...@gmail.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks for the updates. I made a pass over the wiki again
>> and
>>>> it
>>>>>>>>>>> looks
>>>>>>>>>>>>>>>> good.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> About whether record collector should still internally log
>>> the
>>>>>>>>>>> error in
>>>>>>>>>>>>>>>> addition to what the customized ProductionExceptionHandler
>>>>>>> does. I
>>>>>>>>>>>>>>>> personally would prefer only to log if the returned value
>> is
>>>>>>> FAIL
>>>>>>>>>>> to
>>>>>>>>>>>>>>>> indicate that this thread is going to shutdown and trigger
>>> the
>>>>>>>>>>>>>> exception
>>>>>>>>>>>>>>>> handler.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Sun, Nov 5, 2017 at 6:09 AM, Matt Farmer <m...@frmr.me>
>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hello, a bit later than I'd anticipated, but I've updated
>>> this
>>>>>>>>>>> KIP as
>>>>>>>>>>>>>>>>> outlined above. The updated KIP is now ready for review
>>> again!
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Sat, Nov 4, 2017 at 1:03 PM Matt Farmer <m...@frmr.me>
>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Ah. I actually created both of those in the PR and forgot
>>> to
>>>>>>>>>>>>>> mention
>>>>>>>>>>>>>>>> them
>>>>>>>>>>>>>>>>>> by name in the KIP! Thanks for pointing out the
>> oversight.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I’ll revise the KIP this afternoon accordingly.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The logging is actually provided for in the record
>>> collector.
>>>>>>>>>>>>>>> Whenever
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> handler continues it’ll log a warning to ensure that it’s
>>>>>>>>>>>>>>> *impossible*
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> write a handler that totally suppresses production
>>> exceptions
>>>>>>>>>>> from
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> log.
>>>>>>>>>>>>>>>>>> As such, the default continue handler just returns the
>>>>>>> continue
>>>>>>>>>>>>>>> value.
>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>> can add details about those semantics to the KIP as well.
>>>>>>>>>>>>>>>>>> On Sat, Nov 4, 2017 at 12:46 PM Matthias J. Sax <
>>>>>>>>>>>>>>> matth...@confluent.io
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> One more comment.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> You mention a default implementation for the handler
>> that
>>>>>>>>>>> fails. I
>>>>>>>>>>>>>>>>>>> think, this should be part of the public API and thus
>>> should
>>>>>>>>>>> have
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> proper defined named that is mentioned in the KIP.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> We could also add a second implementation for the
>>>>>>>>>>> log-and-move-on
>>>>>>>>>>>>>>>>>>> strategy, as both are the two most common cases. This
>>> class
>>>>>>>>>>> should
>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>> be part of public API (so users can just set in the
>>> config)
>>>>>>>>>>> with a
>>>>>>>>>>>>>>>>>>> proper name.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Otherwise, I like the KIP a lot! Thanks.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On 11/1/17 12:23 AM, Matt Farmer wrote:
>>>>>>>>>>>>>>>>>>>> Thanks for the heads up. Yes, I think my changes are
>>>>>>> compatible
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> PR, but there will be a merge conflict that happens
>>>> whenever
>>>>>>>>>>> one
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> PRs
>>>>>>>>>>>>>>>>>>>> is merged. Happy to reconcile the changes in my PR if
>>> 4148
>>>>>>> goes
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> first. :)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Tue, Oct 31, 2017 at 6:44 PM Guozhang Wang <
>>>>>>>>>>>>>> wangg...@gmail.com
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> That sounds reasonable, thanks Matt.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> As for the implementation, please note that there is
>>>>>>> another
>>>>>>>>>>>>>>>> ongoing
>>>>>>>>>>>>>>>>> PR
>>>>>>>>>>>>>>>>>>>>> that may touch the same classes that you are working
>> on:
>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/4148
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> So it may help if you can also take a look at that PR
>>> and
>>>>>>> see
>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> compatible with your changes.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Tue, Oct 31, 2017 at 10:59 AM, Matt Farmer <
>>>>>>> m...@frmr.me>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I've opened this pull request to implement the KIP as
>>>>>>>>>>>>>> currently
>>>>>>>>>>>>>>>>>>> written:
>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/4165. It still
>>> needs
>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>> tests
>>>>>>>>>>>>>>>>>>>>>> added,
>>>>>>>>>>>>>>>>>>>>>> but largely represents the shape I was going for.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> If there are more points that folks would like to
>>>> discuss,
>>>>>>>>>>>>>>> please
>>>>>>>>>>>>>>>>> let
>>>>>>>>>>>>>>>>>>> me
>>>>>>>>>>>>>>>>>>>>>> know. If I don't hear anything by tomorrow afternoon
>>> I'll
>>>>>>>>>>>>>>> probably
>>>>>>>>>>>>>>>>>>> start
>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>> [VOTE] thread.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>> Matt
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Fri, Oct 27, 2017 at 7:33 PM Matt Farmer <
>>> m...@frmr.me
>>>>>>>>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I can’t think of a reason that would be problematic.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Most of the time I would write a handler like this,
>> I
>>>>>>> either
>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> ignore the error or fail and bring everything down
>> so
>>>>>>> that I
>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>> spin
>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>> back up later and resume from earlier offsets. When
>> we
>>>>>>> start
>>>>>>>>>>>>>> up
>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>>>>> crashing we’ll eventually try to process the message
>>> we
>>>>>>>>>>>>>> failed
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> produce
>>>>>>>>>>>>>>>>>>>>>>> again.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I’m concerned that “putting in a queue for later”
>>> opens
>>>>>>> you
>>>>>>>>>>>>>> up
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> putting
>>>>>>>>>>>>>>>>>>>>>>> messages into the destination topic in an unexpected
>>>>>>> order.
>>>>>>>>>>>>>>>> However
>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>> others feel differently, I’m happy to talk about it.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Fri, Oct 27, 2017 at 7:10 PM Guozhang Wang <
>>>>>>>>>>>>>>>> wangg...@gmail.com>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Please correct me if I'm wrong, but my
>> understanding
>>>> is
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>>>>>> metadata is always null if an exception occurred
>>> while
>>>>>>>>>>>>>> trying
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> produce.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> That is right. Thanks.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I looked at the example code, and one thing I
>>> realized
>>>>>>> that
>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>> not passing the context in the handle function, we
>>> may
>>>>>>> not
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>> implement
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> logic to send the fail records into another queue
>> for
>>>>>>>>>>> future
>>>>>>>>>>>>>>>>>>>>> processing.
>>>>>>>>>>>>>>>>>>>>>>>> Would people think that would be a big issue?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Oct 26, 2017 at 12:14 PM, Matt Farmer <
>>>>>>>>>>> m...@frmr.me
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Hello all,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP based on this conversation,
>> and
>>>>>>> made
>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>>>>>>>>> interface, config setting, and parameters line up
>>> more
>>>>>>>>>>>>>>> closely
>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> interface in KIP-161 (deserialization handler).
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I believe there are a few specific questions I
>> need
>>> to
>>>>>>>>>>>>>> reply
>>>>>>>>>>>>>>>>>>> to.....
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> The question I had about then handle parameters
>> are
>>>>>>>>>>> around
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> record,
>>>>>>>>>>>>>>>>>>>>>>>>>> should it be `ProducerRecord<byte[], byte[]>`, or
>>> be
>>>>>>>>>>>>>>> generics
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>> `ProducerRecord<? extends K, ? extends V>` or
>>>>>>>>>>>>>>>> `ProducerRecord<?
>>>>>>>>>>>>>>>>>>>>>>>> extends
>>>>>>>>>>>>>>>>>>>>>>>>>> Object, ? extends Object>`?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> At this point in the code we're guaranteed that
>> this
>>>>>>> is a
>>>>>>>>>>>>>>>>>>>>>>>>> ProducerRecord<byte[], byte[]>, so the generics
>>> would
>>>>>>> just
>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>> harder
>>>>>>>>>>>>>>>>>>>>>>>>> to work with the key and value.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Also, should the handle function include the
>>>>>>>>>>>>>>> `RecordMetadata`
>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>> well
>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>> case it is not null?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Please correct me if I'm wrong, but my
>> understanding
>>>> is
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>>>>>> metadata is always null if an exception occurred
>>> while
>>>>>>>>>>>>>> trying
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> produce.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> We may probably try to write down at least the
>>>>>>> following
>>>>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>> see if the given API is sufficient for it
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I've added some examples to the KIP. Let me know
>>> what
>>>>>>> you
>>>>>>>>>>>>>>>> think.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>> Matt
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Oct 23, 2017 at 9:00 PM Matt Farmer <
>>>>>>> m...@frmr.me
>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for this feedback. I’m at a conference
>> right
>>>>>>> now
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>>>>>>> planning
>>>>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>> updating the KIP again with details from this
>>>>>>>>>>> conversation
>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>> week.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I’ll shoot you a more detailed response then! :)
>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, Oct 23, 2017 at 8:16 PM Guozhang Wang <
>>>>>>>>>>>>>>>>> wangg...@gmail.com
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP Matt.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Regarding the handle interface of
>>>>>>>>>>>>>>>>> ProductionExceptionHandlerResp
>>>>>>>>>>>>>>>>>>>>>> onse,
>>>>>>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>> you write it on the wiki also, along with the
>>> actual
>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>>>>>>>> names
>>>>>>>>>>>>>>>>>>>>>>>>>>> (e.g. what
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>> 161%3A+streams+
>>>>>>>>>>>>>>>>>>>>>>>>> deserialization+exception+handlers
>>>>>>>>>>>>>>>>>>>>>>>>>>> described).
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> The question I had about then handle parameters
>>> are
>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> record,
>>>>>>>>>>>>>>>>>>>>>>>>>>> should it be `ProducerRecord<byte[], byte[]>`,
>> or
>>> be
>>>>>>>>>>>>>>> generics
>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>> `ProducerRecord<? extends K, ? extends V>` or
>>>>>>>>>>>>>>>> `ProducerRecord<?
>>>>>>>>>>>>>>>>>>>>>>>> extends
>>>>>>>>>>>>>>>>>>>>>>>>>>> Object, ? extends Object>`?
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Also, should the handle function include the
>>>>>>>>>>>>>>> `RecordMetadata`
>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>> well in
>>>>>>>>>>>>>>>>>>>>>>>>>>> case it is not null?
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> We may probably try to write down at least the
>>>>>>> following
>>>>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>> see if the given API is sufficient for it: 1)
>>> throw
>>>>>>>>>>>>>>> exception
>>>>>>>>>>>>>>>>>>>>>>>>> immediately
>>>>>>>>>>>>>>>>>>>>>>>>>>> to fail fast and stop the world, 2) log the
>> error
>>>> and
>>>>>>>>>>>>>> drop
>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>> proceed silently, 3) send such errors to a
>>> specific
>>>>>>>>>>>>>> "error"
>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>>>>>> topic,
>>>>>>>>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>>> record it as an app-level metrics (
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>> https://kafka.apache.org/documentation/#kafka_streams_
>>>>>>>>>>>>>>>>> monitoring
>>>>>>>>>>>>>>>>>>>>> )
>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>>> monitoring.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Oct 20, 2017 at 5:47 PM, Matt Farmer <
>>>>>>>>>>>>>> m...@frmr.me
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I did some more digging tonight.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> @Ted: It looks like the deserialization handler
>>>> uses
>>>>>>>>>>>>>>>>>>>>>>>>>>>> "default.deserialization.exception.handler"
>> for
>>> the
>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>>>>>> name. No
>>>>>>>>>>>>>>>>>>>>>>>>>>>> ".class" on the end. I'm inclined to think this
>>>>>>> should
>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>>>>>>>>> "default.production.exception.handler".
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, Oct 20, 2017 at 8:22 PM Matt Farmer <
>>>>>>>>>>>>>> m...@frmr.me
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Okay, I've dug into this a little bit.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think getting access to the serialized
>> record
>>> is
>>>>>>>>>>>>>>>> possible,
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>> changing
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the naming and return type is certainly
>> doable.
>>>>>>>>>>>>>> However,
>>>>>>>>>>>>>>>>>>>>>> because
>>>>>>>>>>>>>>>>>>>>>>>>> we're
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> hooking into the onCompletion callback we have
>>> no
>>>>>>>>>>>>>>> guarantee
>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProcessorContext state hasn't changed by the
>>> time
>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>> particular
>>>>>>>>>>>>>>>>>>>>>>>>>>> handler
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> runs. So I think the signature would change to
>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>> like:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ProductionExceptionHandlerResponse
>> handle(final
>>>>>>>>>>>>>>>>>>>>>>>> ProducerRecord<..>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> record,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> final Exception exception)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Would this be acceptable?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Oct 19, 2017 at 7:33 PM Matt Farmer <
>>>>>>>>>>>>>>> m...@frmr.me>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ah good idea. Hmmm. I can line up the naming
>>> and
>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>> type
>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>> I’m
>>>>>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> sure if I can get my hands on the context and
>>> the
>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>>> itself
>>>>>>>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> other changes.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Let me dig in and follow up here tomorrow.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, Oct 19, 2017 at 7:14 PM Matthias J.
>>> Sax <
>>>>>>>>>>>>>>>>>>>>>>>>>>> matth...@confluent.io>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Are you familiar with KIP-161?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>> 161%3A+streams+
>>>>>>>>>>>>>>>>>>>>>>>>>>>> deserialization+exception+handlers
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I thinks, we should align the design
>>> (parameter
>>>>>>>>>>>>>> naming,
>>>>>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>>>>>>>>> types,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> class names etc) of KIP-210 to KIP-161 to
>> get
>>> a
>>>>>>>>>>>>>> unified
>>>>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>>>>>>>>>> experience.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 10/18/17 4:20 PM, Matt Farmer wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I’ll create the JIRA ticket.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I think that config name will work. I’ll
>>> update
>>>>>>> the
>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>>>> accordingly.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Oct 18, 2017 at 6:09 PM Ted Yu <
>>>>>>>>>>>>>>>>>>>>>> yuzhih...@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Can you create JIRA that corresponds to
>> the
>>>>>>> KIP ?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For the new config, how about naming it
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> production.exception.processor.class
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ? This way it is clear that class name
>>> should
>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>> specified.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Oct 18, 2017 at 2:40 PM, Matt
>>> Farmer <
>>>>>>>>>>>>>>>>>>>>>> m...@frmr.me>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is the discussion thread for the
>> KIP
>>>>>>> that I
>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>> filed
>>>>>>>>>>>>>>>>>>>>>>>>>>> here:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 210+-+Provide+for+custom+
>>>>>>>>>>>>>>> error+handling++when+Kafka+
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams+fails+to+produce
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Looking forward to getting some feedback
>>> from
>>>>>>>>>>>>>> folks
>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>> idea
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> working toward a solution we can
>>> contribute
>>>>>>> back.
>>>>>>>>>>>>>> :)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Matt Farmer
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> --
>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>
>>>>
>>>
>>>
>>
>>
>> --
>> -- Guozhang
>>
> 

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to