Skip to content

KAFKA-16448: Add ErrorHandlerContext in deserialization exception handler#16432

Merged
mjsax merged 5 commits intoapache:trunkfrom
sebastienviale:KAFKA-16448-Deserialization-Add-Error-Handler-Context
Jul 30, 2024
Merged

KAFKA-16448: Add ErrorHandlerContext in deserialization exception handler#16432
mjsax merged 5 commits intoapache:trunkfrom
sebastienviale:KAFKA-16448-Deserialization-Add-Error-Handler-Context

Conversation

@sebastienviale
Copy link
Copy Markdown
Contributor

This PR is part of KAFKA-16448 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.

This PR expose the new ErrorHandlerContext as a parameter to the Deserialization exception handlers and deprecate the previous handle signatur

Jira: https://issues.apache.org/jira/browse/KAFKA-16448.

Contributors
@Dabz
@sebastienviale
@loicgreffier

@loicgreffier loicgreffier force-pushed the KAFKA-16448-Deserialization-Add-Error-Handler-Context branch from ade2811 to 0674502 Compare July 22, 2024 07:22
@sebastienviale sebastienviale changed the title Kafka 16448 deserialization add error handler context KAFKA-16448: Add ErrorHandlerContext in deserialization exception handler Jul 22, 2024
@loicgreffier loicgreffier force-pushed the KAFKA-16448-Deserialization-Add-Error-Handler-Context branch 2 times, most recently from 32c8edf to 8f5decd Compare July 22, 2024 22:27
Copy link
Copy Markdown
Member

@cadonna cadonna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the PR @sebastienviale , @loicgreffier, and @Dabz!

Here my feeback!

);

final StreamsException e = assertThrows(StreamsException.class, () -> recordDeserializer.deserialize(context, rawRecord));
assertEquals(e.getMessage(), "Deserialization exception handler is set to fail upon a deserialization error. If you would rather have the streaming pipeline continue after a deserialization error, please set the default.deserialization.exception.handler appropriately.");
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This line is definitely too long. Please add some new lines.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

"true, false",
"false, true"
})
public void shouldThrowStreamsExceptionWhenDeserializationFailsAndDeserializationExceptionHandlerRepliesWithFail(final boolean keyThrowsException,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit:

Suggested change
public void shouldThrowStreamsExceptionWhenDeserializationFailsAndDeserializationExceptionHandlerRepliesWithFail(final boolean keyThrowsException,
public void shouldThrowStreamsExceptionWhenDeserializationFailsAndExceptionHandlerRepliesWithFail(final boolean keyThrowsException,

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

"true, false",
"false, true"
})
public void shouldNotThrowStreamsExceptionWhenDeserializationFailsAndDeserializationExceptionHandlerRepliesWithContinue(final boolean keyThrowsException,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit:

Suggested change
public void shouldNotThrowStreamsExceptionWhenDeserializationFailsAndDeserializationExceptionHandlerRepliesWithContinue(final boolean keyThrowsException,
public void shouldNotThrowStreamsExceptionWhenDeserializationFailsAndExceptionHandlerRepliesWithContinue(final boolean keyThrowsException,

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

@ParameterizedTest
@CsvSource({
"true, false",
"false, true"
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you also need to test "true, true"?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

true, true should fail and exit on the deserialization of the key just like true, false. As the test is parametized, it does not cost so much to add this scenario.

sourceNodeName,
processorContext.taskId());
response = deserializationExceptionHandler.handle(errorHandlerContext, rawRecord, deserializationException);
} catch (final Exception fatalUserException) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The 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?
Sorry for not noticing it earlier!

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cadonna As processing exception handler handles exception at ProcessorNode, indeed exceptions during deserialization are not caught right now. Is it your point?

Maybe we should consider catching and handling exceptions when getting the nextRecord at StreamTask level right before processing it:

record = partitionGroup.nextRecord(recordInfo, wallClockTime);

Going to sleep on it

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The 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.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done in this PR #16675

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done in this PR #16300 for Punctuating part

@loicgreffier loicgreffier force-pushed the KAFKA-16448-Deserialization-Add-Error-Handler-Context branch 2 times, most recently from 589b42b to 7add8c7 Compare July 23, 2024 15:24
Copy link
Copy Markdown
Member

@cadonna cadonna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the updates @sebastienviale, @loicgreffier, and @Dabz!

LGTM!

@loicgreffier loicgreffier force-pushed the KAFKA-16448-Deserialization-Add-Error-Handler-Context branch from a9d57f8 to c4c781c Compare July 24, 2024 08:23
@cadonna cadonna added streams kip Requires or implements a KIP labels Jul 24, 2024
Copy link
Copy Markdown
Member

@mjsax mjsax left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Overall LGTM. A few minor things.

* @param context processor context
* @param record record that failed deserialization
* @param exception the actual exception
* @deprecated Please {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)}
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
* @deprecated Please {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)}
* @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)} instead.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not sure about already merged PRs, but if they also have deprecation notice, this the the format we usually use. Including the version helps user to see how far behind they are and when the can expect the old API to be remove, so we want to include it.

Can you double check other PRs so we can update stuff if necessary?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done
I wil check other PRs to be sure

default DeserializationHandlerResponse handle(final ProcessorContext context,
final ConsumerRecord<byte[], byte[]> record,
final Exception exception) {
throw new UnsupportedOperationException();
Copy link
Copy Markdown
Member

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.

Copy link
Copy Markdown
Contributor Author

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

final ConsumerRecord<byte[], byte[]> rawRecord,
final Logger log,
final Sensor droppedRecordsSensor) {
handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor, null);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we still need this one? Can't we update GlobalStateManagerImpl to also pass in a sourceNode.name ?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The 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.

Copy link
Copy Markdown
Contributor

@loicgreffier loicgreffier Jul 30, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@mjsax I've been looking for a way to update GlobalStateManagerImpl so we can remove this method. But it looks like we have no source node name in GlobalStateManagerImpl. There's a "source" processor:

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 ⬆️

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Well, the "helper" method also only calls the second one, setting sourceNodeName to null. Why can't we just call the second one from GlobalStateManagerImpl directly also setting it to null, and thus avoiding the need for the overload? The overload does not really add much value, or does it?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@mjsax Addressing it in #16746

this.taskId = taskId;
}

public DefaultErrorHandlerContext(final ProcessorContext processorContext,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this constructor replace the other one? And we just pass in null for processorContext?

Or we use setRecordContext() -- but having constructor overloads is always something I think is better to avoid if we can.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

return this.taskId;
}

public ProcessorContext processorContext() {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this return Optional<ProcessorContext> given that it could be null ?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

sebastienviale and others added 4 commits July 29, 2024 16:17
…ndler

Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
…ndler

Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
…ndler remove rawKey and rawValue

Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
…ndler remove rawKey and rawValue

Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
@sebastienviale sebastienviale force-pushed the KAFKA-16448-Deserialization-Add-Error-Handler-Context branch from 7ffcc19 to c5fecd7 Compare July 29, 2024 14:17
…ndler

Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
@mjsax mjsax merged commit faaef52 into apache:trunk Jul 30, 2024
mjsax pushed a commit that referenced this pull request Jul 30, 2024
…dler (#16432)

This PR is part of KIP1033 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.

This PR expose the new ErrorHandlerContext as a parameter to the Deserialization exception handlers and deprecate the previous handle signature.

Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>

Reviewers: Bruno Cadonna <bruno@confluent.io>, Matthias J. Sax <matthias@confluent.io>
@mjsax
Copy link
Copy Markdown
Member

mjsax commented Jul 30, 2024

Thanks for the PR.

Merged to trunk and cherry-picked to 3.9 branch.

abhi-ksolves pushed a commit to ksolves/kafka that referenced this pull request Jul 31, 2024
…dler (apache#16432)

This PR is part of KIP1033 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.

This PR expose the new ErrorHandlerContext as a parameter to the Deserialization exception handlers and deprecate the previous handle signature.

Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>

Reviewers: Bruno Cadonna <bruno@confluent.io>, Matthias J. Sax <matthias@confluent.io>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

kip Requires or implements a KIP streams

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants