Skip to content

KAFKA-16448: Add ErrorHandlerContext in production exception handler#16433

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

KAFKA-16448: Add ErrorHandlerContext in production exception handler#16433
mjsax merged 7 commits intoapache:trunkfrom
sebastienviale:KAFKA-16448-Production-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 Production exception handler and deprecate the previous handle signature

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

Contributors
@Dabz
@sebastienviale
@loicgreffier

@loicgreffier loicgreffier force-pushed the KAFKA-16448-Production-Add-Error-Handler-Context branch from f512db8 to c60b79e Compare July 22, 2024 07:23
@sebastienviale sebastienviale changed the title Kafka 16448 production add error handler context KAFKA-16448: Add ErrorHandlerContext in production exception handler Jul 22, 2024
@loicgreffier loicgreffier force-pushed the KAFKA-16448-Production-Add-Error-Handler-Context branch from 36098ac to 3fe0f59 Compare July 22, 2024 22:28
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 feedback!

public class DefaultProductionExceptionHandler implements ProductionExceptionHandler {
@Override
public ProductionExceptionHandlerResponse handle(final ProducerRecord<byte[], byte[]> record,
public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context,
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 do not think we can modify this without deprecating the old method and mentioning it in the KIP since it is part of the public API.
I am aware that this method is only called internally.

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, we did it for the deserialization exception handler implementations. I'm wondering why we did not do the same for the production.. 😄

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 I updated KIP-1033 with a word regarding deserialization and production handler implementations

Comment on lines +60 to +61
* @param record the record that failed to serialize
* @param exception the exception that occurred during serialization
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.

Could you please revert the formatting changes in this two 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

Comment on lines +221 to +226
final ProductionExceptionHandler.SerializationExceptionOrigin origin;
if (keyBytes == null) {
origin = ProductionExceptionHandler.SerializationExceptionOrigin.KEY;
} else {
origin = ProductionExceptionHandler.SerializationExceptionOrigin.VALUE;
}
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, it would be better to have something like:

try {
    keyBytes = keySerializer.serialize(topic, headers, key);
} catch (final ClassCastException exception) {
    handleClassCastException(...)
} catch (final Exception exception) {
   handleException(ProductionExceptionHandler.SerializationExceptionOrigin.KEY, ...)
}

try {
    valBytes = valueSerializer.serialize(topic, headers, value);
} catch (final ClassCastException exception) {
    handleClassCastException(...)
} catch (final Exception exception) {
   handleException(ProductionExceptionHandler.SerializationExceptionOrigin.VALUE, ...)
}

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.

I had to remove the final from keyBytes and valueBytes and initialize them when moving the ClassCastException code to a dedicated method

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.

If you want to keep the final and make the code a bit better readable, you could rename handleClassCastException() to createStreamsExceptionForClassCastException() and return the StreamsException from there. Then you can do

} catch (final ClassCastException exception) {
            throw createStreamsExceptionForClassCastException()(
                topic,
                key,
                value,
                keySerializer,
                valueSerializer,
                exception);

With this it is clear that the catch-branch terminates the method.

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.

The update has been pushed!

Comment on lines +1803 to +1804
assertEquals(expectedContext.recordContext().rawRecord().key(), context.sourceRawKey());
assertEquals(expectedContext.recordContext().rawRecord().value(), context.sourceRawValue());
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 you need to use assertArrayEquals() here to verify the byte arrays.

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

@loicgreffier loicgreffier force-pushed the KAFKA-16448-Production-Add-Error-Handler-Context branch 2 times, most recently from d6dc8f1 to 162d75b Compare July 23, 2024 15:03
@cadonna cadonna added streams kip Requires or implements a KIP labels Jul 24, 2024
@loicgreffier loicgreffier force-pushed the KAFKA-16448-Production-Add-Error-Handler-Context branch from 94edf82 to 697e0b7 Compare July 24, 2024 08:43
}

@Test
public void shouldThrowStreamsExceptionWhenValueSerializationFailedAndProductionExceptionHandlerRepliesWithFail() {
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 am missing the version of this test where the key serialization fails.

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, unit test added

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!

LGTM!

* Production exception handler that always instructs streams to continue when an exception
* happens while attempting to produce result records.
*/
public class AlwaysContinueProductionExceptionHandler implements ProductionExceptionHandler {
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 does this PR remove this class?

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

We have created a parameterized ProductionExceptionHandler (ProductionExceptionHandlerMock) in the RecordCollectorTest class that additionally does some assertions, including the origin of the exception.

Thus AlwaysContinueProductionExceptionHandler was redundant with ProductionExceptionHandlerMock, it has been replaced in this PR while updating and adding some additional unit tests

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.

But AlwaysContinueProductionExceptionHandler is a public built-in handler users can configure... -- it's public API, and thus has nothing to do with ProductionExceptionHandlerMock and our internal testing.

It's a build-in alternative to the default handler.

If we really want to no have it any longer, we would need to deprecate -- but we cannot just remove public API. But I don't see why we would want to deprecate and remove 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 Unless I'm wrong, AlwaysContinueProductionExceptionHandler belongs to src/test/java and is only used for testing purpose in the RecordCollectorTest.

Given that it is only use for testing, should it be consider public? If yes, we can obviously restore the class

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.

Ah. You are right. My bad. I guess I mixed it up with the two built-in handler for DeserializationExceptionHandler, but we only have the "alway fail" build-in handler for the production case...

*
* @param record The record that failed to produce
* @param exception The exception that occurred during production
* @deprecated Use {@link #handle(ErrorHandlerContext, ProducerRecord, 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.

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

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

*
* @param record the record that failed to serialize
* @param exception the exception that occurred during serialization
* @deprecated Use {@link #handle(ErrorHandlerContext, ProducerRecord, 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.

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

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

@Deprecated
default ProductionExceptionHandlerResponse handle(final ProducerRecord<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.

KIP says NotImplementedException -- we should update the KIP -- same for the other method below.

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.

KIP has been updated

droppedRecordsSensor.record();
}

private <K, V> StreamsException createStreamsExceptionForClassCastException(final String topic,
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.

Given that we introduce a helper and have two dedicated try-catch blocks for key and value now, it seems we can tailor this method to key or value similar to what we do for handleException instead of this generic one covering both at once?

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 hope this what you are waiting !

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.

Yes, and I merged this PR to make progress. However, I think, we can unify both methods into a single one, passing in "key" or "value" as parameter to avoid code duplication (similar to the "origin" parameter use in handleException(...))

The generic type would could be updated to <KV> for "key or value type" or just to <T> if you prefer.

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 #16736

sebastienviale and others added 7 commits July 29, 2024 16:12
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
…r remove rawKey and rawValue

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

Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
…r 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-Production-Add-Error-Handler-Context branch from 42839a4 to 902ae98 Compare July 29, 2024 14:14
@mjsax mjsax merged commit b6d5f05 into apache:trunk Jul 30, 2024

droppedRecordsSensor.record();
}
private <K> StreamsException createStreamsExceptionForKeyClassCastException(final String topic,
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: missing newline

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 #16736

mjsax pushed a commit that referenced this pull request Jul 30, 2024
…16433)

This PR is part of KIP-1033 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 Production exception handler 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
…pache#16433)

This PR is part of KIP-1033 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 Production exception handler 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