Trying to bump once more On Wed, Dec 11, 2024 at 4:02 PM Anton Liauchuk <anton93...@gmail.com> wrote:
> Bumping the thread > > On Sun, Dec 1, 2024 at 8:33 AM Anton Liauchuk <anton93...@gmail.com> > wrote: > >> Hi >> Thank you for your feedback. >> I have numbered the questions to simplify communication. >> >> 1. 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? >> >> I have a use case where source connectors need to send metrics and logs >> to a custom Kafka topic. Although it's possible to use a log reporter to >> extract the required information from logs, there are several limitations >> to consider: >> - It depends on the log format used in `*kafka-runtime*`. >> - A pluggable interface provides greater flexibility for defining custom >> behavior. >> - The API will have better support in future releases of `*kafka-connect* >> `. >> >> 2. 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. >> >> added, thank you! >> >> 3. 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? >> >> In the current implementation, custom reporters are an independent >> addition to the runtime reporters. >> >> 4. 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?) >> >> Error reporters are expected to be source/sink agnostic. >> >> 5. Should reporters be asynchronous/fire-and-forget, or should they have >>> a mechanism for propagating errors that kill the task? >> >> I believe that adding a mechanism for propagating errors to the error >> handler interface is preferable. >> >> 6. 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)? >> >> I believe this approach makes sense. I have added new changes to a >> separate branch and created a PR >> https://github.com/anton-liauchuk/kafka/pull/1/files. I haven’t extended >> the KIP at this stage, as I would like to discuss some items first. In this >> PR, I haven’t prepared all the necessary changes to support a new mode yet; >> it's just POC. >> >> It seems we don’t need to add this functionality to the reporter, as it >> would be better for the reporter interface to focus solely on reporting. I >> have created a new interface called `ErrorHandler`, which provides a way to >> handle error responses. I designed this interface to be similar to >> `org.apache.kafka.streams.errors.ProcessingExceptionHandler` from the >> `kafka-streams` project. >> >> I'm considering extending the tolerance configuration to enable this >> handler with the `*errors.tolerance=custom*` setting. When custom >> tolerance is selected, the client can specify the class name for the error >> handler. Handling the error might result in one of three options: >> - *DROP*: Skips the record. >> - *FAIL*: Fails the task. >> - *ACK*: Skips the message and acknowledges it, applicable for source >> connectors. >> The following stages are where error handling might be used (these stages >> are part of the `*TOLERABLE_EXCEPTIONS*` in ` >> *org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator#TOLERABLE_EXCEPTIONS* >> `): >> - *TRANSFORMATION* >> - *KEY_CONVERTER* >> - *VALUE_CONVERTER* >> - *HEADER_CONVERTER* >> >> I would like some advice on the following items: >> 6.1. Do we still need to define an error reporter interface if we have >> the option to create an error handler? I believe that all necessary >> reporting can be managed within the error handler, making the reporter >> interface seem unnecessary. >> 6.2. Does it make sense to expand the list of stages where the error >> handler can be used? The current list is based on the existing error >> handling logic. For instance, it could be beneficial to handle errors from >> the `*TASK_POLL*` stage. The current implementation does not support >> error handling for errors that are unassigned to any records, but we could >> consider how to extend it if needed. Additionally, we might review the ` >> *KAFKA_PRODUCE*` and `*TASK_PUT*` stages. >> 6.3. If we begin improvements to error handling, should we also explore >> the possibility of supporting error handling for connector or task failures? >> >> >> On Fri, Oct 25, 2024 at 2:30 AM Greg Harris <greg.har...@aiven.io.invalid> >> wrote: >> >>> 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. >>> > >>> >>