-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-16448: Add ErrorHandlerContext in deserialization exception handler #16432
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
8d5d266
7afd77f
2ebb123
c5fecd7
afcbadd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||
|---|---|---|---|---|
|
|
@@ -22,6 +22,7 @@ | |||
| import org.apache.kafka.common.utils.LogContext; | ||||
| import org.apache.kafka.streams.errors.DeserializationExceptionHandler; | ||||
| import org.apache.kafka.streams.errors.StreamsException; | ||||
| import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; | ||||
| import org.apache.kafka.streams.processor.api.ProcessorContext; | ||||
|
|
||||
| import org.slf4j.Logger; | ||||
|
|
@@ -69,7 +70,7 @@ ConsumerRecord<Object, Object> deserialize(final ProcessorContext<?, ?> processo | |||
| Optional.empty() | ||||
| ); | ||||
| } catch (final Exception deserializationException) { | ||||
| handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor); | ||||
| handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor, sourceNode().name()); | ||||
| return null; // 'handleDeserializationFailure' would either throw or swallow -- if we swallow we need to skip the record by returning 'null' | ||||
| } | ||||
| } | ||||
|
|
@@ -80,12 +81,27 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa | |||
| final ConsumerRecord<byte[], byte[]> rawRecord, | ||||
| final Logger log, | ||||
| final Sensor droppedRecordsSensor) { | ||||
| handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor, null); | ||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why do we still need this one? Can't we update
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @sebastienviale What about this comments? Merge the PR for now to make progress.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @mjsax I've been looking for a way to update kafka/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java Line 315 in 010ab19
But it doesn't look to be named. E.g., in the following scenario: streamsBuilder
.addGlobalStore(storeBuilder, "PERSON_TOPIC",
Consumed.with(Serdes.String(), SerdesUtils.specificAvroValueSerdes()),
() -> (Processor<String, SpecificRecord, Void, Void>) record -> log.info("Processing record with key = {}, value = {}", record.key(), record.value()));
// Source var is this processor ⬆️
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Well, the "helper" method also only calls the second one, setting
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||||
| } | ||||
|
|
||||
| public static void handleDeserializationFailure(final DeserializationExceptionHandler deserializationExceptionHandler, | ||||
| final ProcessorContext<?, ?> processorContext, | ||||
| final Exception deserializationException, | ||||
| final ConsumerRecord<byte[], byte[]> rawRecord, | ||||
| final Logger log, | ||||
| final Sensor droppedRecordsSensor, | ||||
| final String sourceNodeName) { | ||||
| final DeserializationExceptionHandler.DeserializationHandlerResponse response; | ||||
| try { | ||||
| response = deserializationExceptionHandler.handle( | ||||
| final DefaultErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( | ||||
| (InternalProcessorContext<?, ?>) processorContext, | ||||
| rawRecord, | ||||
| deserializationException); | ||||
| rawRecord.topic(), | ||||
| rawRecord.partition(), | ||||
| rawRecord.offset(), | ||||
| rawRecord.headers(), | ||||
| sourceNodeName, | ||||
| processorContext.taskId()); | ||||
| response = deserializationExceptionHandler.handle(errorHandlerContext, rawRecord, deserializationException); | ||||
| } catch (final Exception fatalUserException) { | ||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I realized now that you did not consider that the exception handler can also throw an exception for the processing exception handler (and I as the reviewer also missed it). Could you open a separate PR that catches exceptions from the processing exception handler and re-throws them as it is done here?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @cadonna As processing exception handler handles exception at Maybe we should consider catching and handling exceptions when getting the kafka/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java Line 770 in a5bfc21
Going to sleep on it
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think, we misunderstood each other. On this line an exception originating from the user-specified handler is caught. The exception can be anything that is thrown by user code. We did not consider exceptions thrown from user code for the processing exception handler.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done in this PR #16675
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done in this PR #16300 for Punctuating part |
||||
| log.error( | ||||
| "Deserialization error callback failed after deserialization error for record {}", | ||||
|
|
||||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit. KIP says
NotImplementedException()-- we should update the KIP accordingly.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
done KIP has been updated