Makes sense. I created a JIRA for it:
https://issues.apache.org/jira/browse/KAFKA-6376
@Matt: feel free to pick it up if you are interested
(or anybody else :))
-Matthias
On 12/13/17 4:07 PM, Guozhang Wang wrote:
> Metrics: this is a good point.
>
> Note that currently we have two metrics
Metrics: this is a good point.
Note that currently we have two metrics for `skipped-records` on different
levels:
1) on the highest level, the thread-level, we have a `skipped-records`,
that records all the skipped records due to deserialization errors.
2) on the lower processor-node level, we
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,
Thanks for the clearly written KIP, no further comments from my end.
-Bill
On Wed, Dec 6, 2017 at 9:52 AM, Matt Farmer 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
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
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
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
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
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
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
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 wrote:
> Hi there, sorry for the delay in responding. Last week had a holiday and
> several busy work days in
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
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
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
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
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
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
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
>
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
That makes sense.
Guozhang
On Sun, Nov 5, 2017 at 12:33 PM, Matt Farmer 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.
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
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
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 wrote:
> Ah. I actually created both of those in the PR and forgot to mention them
> by name in the KIP!
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
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
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
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
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
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
> 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
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
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 wrote:
> Thanks for the KIP
Thanks for the KIP Matt.
Regarding the handle interface of ProductionExceptionHandlerResponse, 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
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
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
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
wrote:
> Thanks for
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
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 wrote:
> Can you create JIRA that corresponds to the KIP ?
>
> For the new config, how about naming it
>
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 wrote:
> Hello everyone,
>
> This is the
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
40 matches
Mail list logo