Hi Anton,

Thanks for the KIP! I think that looking at internal APIs as inspiration
for new external APIs is a good idea, and I'm glad that you found an
interface close to the problem you're trying to solve.

What sort of implementation do you have in mind for this interface? What
use-case does this interface enable that is not possible with log scraping,
or implementing a source-connector DLQ to Kafka?
Before we make something pluggable, we should consider if the existing
framework implementations could be improved directly.

Could you add the ErrorContext class to your public API description? I
don't think that is an existing interface. Also please specify the
package/fully qualified names for these classes.

How do you expect this will interact with the existing log and DLQ
reporters?
Will users specifying a custom error reporter be able to turn off the other
reporters?

Are error reporters expected to be source/sink agnostic (like the Log
reporter) or are they permitted to function for just one type (like the DLQ
reporter?)

The runtime interface returns a Future<RecordMetadata>, which is an
abstraction specific for the DLQ reporter and ignored for the Log reporter,
and I see that you've omitted it from the new API.
Should reporters be asynchronous/fire-and-forget, or should they have a
mechanism for propagating errors that kill the task?

Would it make sense for error reporting to also involve error handling:
i.e. let the plugin decide how to handle errors (drop record, trigger
retries, fail the task, etc)?
In Connect there's been a longstanding pattern where every connector
reimplements error handling individually, often hardcoding response
behaviors to various errors, because the existing errors.tolerance
configuration is too limiting.
Maybe making this pluggable leads us towards a solution where there could
be a pluggable "error handler" that can implement reporting for many
different errors, but also allow for simple reconfiguration of error
handling behavior.

Thanks,
Greg

On Thu, Oct 24, 2024 at 3:57 PM Anton Liauchuk <anton93...@gmail.com> wrote:

> Bumping the thread. Please review this KIP. Thanks!
>
> On Sun, Oct 13, 2024 at 11:44 PM Anton Liauchuk <anton93...@gmail.com>
> wrote:
> >
> > Hi all,
> >
> > I have opened
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1097+error+record+reporter
> >
> > POC: https://github.com/apache/kafka/pull/17493
> >
> > Please review KIP and PR, feedbacks and suggestions are welcome.
>

Reply via email to