Skip to content

KAFKA-16448: Handle processing exceptions in punctuate#16300

Merged
mjsax merged 2 commits intoapache:trunkfrom
sebastienviale:KAFKA-16448-Punctuate-Processing-Exceptions
Jul 31, 2024
Merged

KAFKA-16448: Handle processing exceptions in punctuate#16300
mjsax merged 2 commits intoapache:trunkfrom
sebastienviale:KAFKA-16448-Punctuate-Processing-Exceptions

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 actually catches processing exceptions from punctuate.

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

Contributors
@Dabz
@sebastienviale
@loicgreffier

@sebastienviale sebastienviale changed the title Kafka 16448 punctuate processing exceptions Kafka-16448 punctuate processing exceptions Jun 12, 2024
@sebastienviale sebastienviale changed the title Kafka-16448 punctuate processing exceptions Kafka-16448: punctuate processing exceptions Jun 12, 2024
@sebastienviale sebastienviale force-pushed the KAFKA-16448-Punctuate-Processing-Exceptions branch 3 times, most recently from 965d110 to 0feaeaa Compare July 19, 2024 09:47
@loicgreffier loicgreffier force-pushed the KAFKA-16448-Punctuate-Processing-Exceptions branch from 9049531 to 445e271 Compare July 21, 2024 22:42
@loicgreffier
Copy link
Copy Markdown
Contributor

loicgreffier commented Jul 21, 2024

@cadonna This PR contains the handling of processing exceptions in StreamTask#punctuate:

  • StreamsException are not handled right now. We could include them in the catch block and rethrow them as is, but might end up with handling exceptions that are not desirable

PR is ready for a first review

@loicgreffier loicgreffier force-pushed the KAFKA-16448-Punctuate-Processing-Exceptions branch from 445e271 to 9ee4179 Compare July 22, 2024 07:17
@sebastienviale sebastienviale changed the title Kafka-16448: punctuate processing exceptions KAFKA-16448: Handle processing exceptions in punctuate Jul 22, 2024
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, @Dabz, @sebastienviale, and @loicgreffier!

Here my feedback

Comment on lines +923 to +924
final ErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext(recordContext.topic(),
recordContext.partition(), recordContext.offset(), recordContext.headers(), null, null, node.name(), id);
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
final ErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext(recordContext.topic(),
recordContext.partition(), recordContext.offset(), recordContext.headers(), null, null, node.name(), id);
final ErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext(
recordContext.topic(),
recordContext.partition(),
recordContext.offset(),
recordContext.headers(),
null,
null,
node.name(),
id
);

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

@@ -916,7 +920,17 @@ public void punctuate(final ProcessorNode<?, ?, ?, ?> node,
} catch (final StreamsException e) {
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.

Now we have a weird situation where you can throw a StreamsException from process() and Streams will handle it in the processing handler but when you throw a StreamsException from punctuate() Streams will not handle 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.

Correct, as mentioned in my previous comment we can handle all RuntimeException and if (exception instanceof StreamsException) already, then we can throw e as is.

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.

Oh, I haven't seen your comment. Sorry!

Your proposal does not change anything, right? We will still have the situtation that if a user throws a StreamsException from punctuate() it will not be handled. But in process() it will be handled.

I think, we have the following options:

  1. We live with this asymmetry between punctuate() and process().
  2. We do not handle StreamsException in process().
  3. We do handle StreamsException in punctuate().

What do you exactly mean with

might end up with handling exceptions that are not desirable

Copy link
Copy Markdown
Contributor

@loicgreffier loicgreffier Jul 22, 2024

Choose a reason for hiding this comment

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

Well, in the ProcessorNode we do not handle TaskCorruptedException or TaskMigratedException which extends StreamsException for example.

Unless I'm wrong these exceptions cannot be caught in the punctuate, right? (I don't want to end up in a situation where we do not handle TaskCorruptedException in the process(), but we do in the punctuate())

👉 Considering TaskCorruptedException/TaskMigratedException cannot reach punctuate(), we updated the PR to handle all StreamsException in the punctuate.

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.

Just catching up. For process() we currently don't handle ClassCastException as well as FailedProcessingException | TaskCorruptedException | TaskMigratedException.

However, only rethrow ClassCastException wrapped as StreamsException() and thus might catch up "upstream" -- wondering if this is intended?

I also just looked into the call-trace and it seems we actually also have dedicate

} catch (final StreamsException exception) {
            record = null;
            throw exception;
} catch  (final RuntimeException e) {

inside StreamTask#process but we only handle FailedProcessingException in the second catch for the RuntimeException. Is this by design? Atm FailedProcessingException extends KafkaException so it still works, but as pointed out on an already merged PR, I think it should actually extend StreamsException (https://github.com/apache/kafka/pull/16093/files#r1687232849) what implies we would need some changes.

In general, StreamsException is actually not design to be thrown by users (not to be catch by users), but it can still bubble up from process() -- So I think it does make sense to catch and pass into the handler for both cases process() and punctuate().

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

If punctuate() calls context.forward() and an exception occurs in a processor I'm having doubts if the exception will either:

  • Go up to the try-catch block we added around punctuate()
  • Go up to the try-catch block we added around ProcessorNode#process

Try-catch block on punctuate() was initially added to handle exceptions that can occurs in punctuator:

context.schedule(Duration.ofMinutes(1), PunctuationType.WALL_CLOCK_TIME,
    timestamp -> {
        throw new RuntimeException("Punctuate error");
    });

Doing further tests on this one..

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 NVM, in the following case:

builder
  .stream(...)
  .process(...) // punctuate() calls context.forward() here
  .map((key, value) -> throw new RuntimeException());

RuntimeException is being caught and handled in the ProcessorNode#process, then thrown as a FailedProcessingException and caught in the StreamTask#punctuate.

In punctuate, So I think we should:

  • catch FailedProcessingException coming from nodes, unwrap it and throw a new StreamsException ➡️ It avoids passing the exception through the processing exception handler twice.
  • catch TaskCorruptedException | TaskMigratedException and throw it as is ➡️ We do not want to handle them / wrap them to anything else.
  • catch Exception, apply the processing exception handler and throw StreamsException ➡️ We handle anything else is not coming from a processor node.

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, SGTM.

It must be a mix of what we do in ProcessingNode#process() and StreamTask#process().

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 I've updated the PR

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.

Thanks. LGTM.

@@ -916,7 +920,17 @@ public void punctuate(final ProcessorNode<?, ?, ?, ?> node,
} catch (final StreamsException e) {
throw e;
} catch (final RuntimeException e) {
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.

For process(), Streams catches a Exception and here it is a RuntimeException. As far as I understand the difference is that RuntimeException only catches unchecked exceptions whereas Exception catches unchecked and checked exception. Since Streams does not throw any checked exceptions from either process() or punctuate it is fine to use in both places RuntimeException.
Could you change this also for process() in this PR?

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

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.

Sorry, I have to re-iterate on this. I now realized that for all other handlers we catch Exception and not RuntimeException. To be consistent let's catch Exception here and in the ProcessorNode.

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

<!-- Streams -->
<suppress checks="ClassFanOutComplexity"
files="(KafkaStreams|KStreamImpl|KTableImpl|InternalTopologyBuilder|StreamsPartitionAssignor|StreamThread|IQv2StoreIntegrationTest|KStreamImplTest|RocksDBStore).java"/>
files="(KafkaStreams|KStreamImpl|KTableImpl|InternalTopologyBuilder|StreamsPartitionAssignor|StreamThread|IQv2StoreIntegrationTest|KStreamImplTest|RocksDBStore|StreamTask).java"/>
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.

That is a pity 😞 ! But I do not see a quick fix to avoid the change.

.handle(errorHandlerContext, null, e);

if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) {
throw new StreamsException(String.format("%sException caught while punctuating processor '%s'", logPrefix, node.name()), e);
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 add a error log here?

Copy link
Copy Markdown
Contributor

@loicgreffier loicgreffier Jul 22, 2024

Choose a reason for hiding this comment

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

Done. process() and punctuate will log the same error

.map(KeyValue::new)
.mapValues(value -> value)
.process(runtimeErrorProcessorSupplierMock())
.process(runtimeErrorPunctuateProcessorSupplierMock())
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.

You should not test punctuate with the TopologyTestDriver. Method punctuate is called outside of the topology in the poll loop. Using the TopologyTestDriver, you basically call task.maybePunctuateSystemTime(); in driver.advanceWallClockTime() which is a helper test infra method. This is not really testing the integration of ProcessingExceptionHandler with punctuate() in the Streams runtime.
Testing the integration of ProcessingExceptionHandler in process() with the TopologyTestDriver makes a little bit more sense because you can test the case when a downstream processor throws and it should not affect an upstream processor, but that case you do not have with punctuate(). However, also for process() a real intergration test would be better.
So given the tight schedule, my proposal is to revert the changes regarding punctuate() in this file and only have unit tests. Then, you can add a real integration test in a separate PR when we merged all the rest first.

@cadonna cadonna requested a review from mjsax July 22, 2024 12:21
@loicgreffier loicgreffier force-pushed the KAFKA-16448-Punctuate-Processing-Exceptions branch from e7a767e to 56a6654 Compare July 22, 2024 13:26
// Rethrow exceptions that should not be handled here
throw e;
} catch (final Exception e) {
} catch (final RuntimeException e) {
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.

Great! Thanks!

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!

A last nit.

" continue after a processing error, please set the " +
PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.");

throw e instanceof StreamsException ? (StreamsException) e : new StreamsException(String.format("%sException caught while punctuating processor '%s'", logPrefix, node.name()), e);
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
throw e instanceof StreamsException ? (StreamsException) e : new StreamsException(String.format("%sException caught while punctuating processor '%s'", logPrefix, node.name()), e);
throw e instanceof StreamsException ? e : new StreamsException(String.format("%sException caught while punctuating processor '%s'", logPrefix, node.name()), e);

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.

Fixed

@loicgreffier loicgreffier force-pushed the KAFKA-16448-Punctuate-Processing-Exceptions branch from 6ccc605 to e368db8 Compare July 22, 2024 13:53
@cadonna cadonna added streams kip Requires or implements a KIP labels Jul 22, 2024
@loicgreffier loicgreffier force-pushed the KAFKA-16448-Punctuate-Processing-Exceptions branch from 4aede2a to fb50fe4 Compare July 24, 2024 08:22
Comment on lines +2707 to +2775
@Override
public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record<?, ?> record, final Exception exception) {
throw new RuntimeException("KABOOM!");
}
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
@Override
public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record<?, ?> record, final Exception exception) {
throw new RuntimeException("KABOOM!");
}
@Override
public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record<?, ?> record, final Exception exception) {
throw new RuntimeException("KABOOM!");
}

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 sebastienviale force-pushed the KAFKA-16448-Punctuate-Processing-Exceptions branch 2 times, most recently from 3fdeedf to 9fd5248 Compare July 30, 2024 06:55
throw e;
} catch (final RuntimeException e) {
throw new StreamsException(String.format("%sException caught while punctuating processor '%s'", logPrefix, node.name()), e);
} catch (final Exception e) {
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 catch Exception here, but not RuntimeException? If no checked exception is declared, it seem not to be "correct"?

Copy link
Copy Markdown
Contributor

@loicgreffier loicgreffier Jul 31, 2024

Choose a reason for hiding this comment

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

@mjsax For this reason: #16300 (comment)

Exception is caught in ProcessorNode#process, RecordCollectorImpl#send and RecordDeserializer#deserialize so the last discussion mentionned to stick to Exception here to be consistent.

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.

Thanks. I disagree to @cadonna conclusion -- we should rather catch RuntimeException everywhere (consistency yes, but the other way around).

I'll cover this on the other PR I did open.


try {
response = processingExceptionHandler.handle(errorHandlerContext, null, e);
} catch (final Exception fatalUserException) {
Copy link
Copy Markdown
Member

@mjsax mjsax Jul 31, 2024

Choose a reason for hiding this comment

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

Same here. Why not RuntimeException -- given that no checked exception is declared for the handler(...) method, it's not possible to get a Exception here.

Same seems to apply to ProcessorNode code which has the same try-catch block.

Let me do a small PR myself to address 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 Same answer: #16300 (comment). ProcessorNode#process and StreamTask#punctuate stick with RecordCollectorImpl and RecordDeserializer.

@loicgreffier loicgreffier force-pushed the KAFKA-16448-Punctuate-Processing-Exceptions branch 2 times, most recently from 3faaede to ef50124 Compare July 31, 2024 09:56
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
@loicgreffier loicgreffier force-pushed the KAFKA-16448-Punctuate-Processing-Exceptions branch from ef50124 to 0dbe434 Compare July 31, 2024 15:16
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
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.

LGTM. Can merge after Jenkins passed.

Does this complete the KIP?

@loicgreffier
Copy link
Copy Markdown
Contributor

@mjsax This PR was the last one! If any additional updates are needed, let us know

I'm updating the KIP to remove references to sourceRawKey/sourceRawValue. To be moved to KIP-1034.

@mjsax
Copy link
Copy Markdown
Member

mjsax commented Jul 31, 2024

I'm updating the KIP to remove references to sourceRawKey/sourceRawValue. To be moved to KIP-1034.

Great. This would have been my next question. Also, after you updated the wiki page, please send a follow-up email to the VOTE thread of the KIP, and call out the changes made. It's not just sourceRawKey/sourceRawValue, but also this PR, right? The KIP does not say anything about punctuations? And the KIP should also talk about the case, when the handler crashed and throws an exception. -- Was there anything else that was changed and should be called out?

@mjsax mjsax merged commit f1ef7a5 into apache:trunk Jul 31, 2024
mjsax pushed a commit that referenced this pull request Jul 31, 2024
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 actually catches processing exceptions from punctuate.

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 31, 2024

Merged to trunk and cherry-picked to 3.9 branch.

@loicgreffier
Copy link
Copy Markdown
Contributor

Was there anything else that was changed and should be called out?

The number of processing exception handlers that are instantiated: 1 / Stream Task. This was not initially mentioned in KIP-1033. It is has been updated.

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