-
Notifications
You must be signed in to change notification settings - Fork 14.1k
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
KAFKA-16448: Add ProcessingExceptionHandler in Streams configuration #16092
Merged
cadonna
merged 17 commits into
apache:trunk
from
loicgreffier:KAFKA-16448-Add-Processing-Exception-Handler-StreamsConfig
May 30, 2024
Merged
Changes from 12 commits
Commits
Show all changes
17 commits
Select commit
Hold shift + click to select a range
9cfac39
KAFKA-16448: Add ProcessingExceptionHandler interface and implementat…
loicgreffier b957861
KAFKA-16448: Add ProcessingExceptionHandler in Streams configuration
loicgreffier 9eef86d
KAFKA-16448: Add ProcessingExceptionHandler interface and implementat…
loicgreffier 40fdac5
KAFKA-16448: Update from parent branch
loicgreffier 3cb797b
KAFKA-16448: Fix indentation
loicgreffier c1c0d5c
KAFKA-16448: Remove unecessary classes
loicgreffier bea1fb8
KAFKA-16448: add Streams configuration test for ProcessingExceptionHa…
sebastienviale b7058d2
KAFKA-16448: add Streams configuration test for ProcessingExceptionHa…
sebastienviale 0a54bae
Merge branch 'KAFKA-16448-Add-Processing-Exception-Handler-StreamsCon…
sebastienviale 014b799
Merge branch 'KAFKA-16448-Add-Processing-Exception-Handler-StreamsCon…
sebastienviale 10e69a9
Merge branch 'KAFKA-16448-Add-Processing-Exception-Handler-StreamsCon…
sebastienviale 3c7de0c
KAFKA-16448: add Streams configuration test for invalid ProcessingExc…
sebastienviale 5e7a656
KAFKA-16448: add error message test in case of invalid ProcessingExce…
sebastienviale 60bba3b
KAFKA-16448: Add ProcessingExceptionHandler in Streams configuration
loicgreffier 25115ce
KAFKA-16448: Add ProcessingExceptionHandler in Streams configuration
loicgreffier 4b56def
Merge branch 'KAFKA-16448-Add-Processing-Exception-Handler-StreamsCon…
sebastienviale 7d67257
KAFKA-16448: remove unused imports
sebastienviale File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
133 changes: 133 additions & 0 deletions
133
streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContext.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,133 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.kafka.streams.errors; | ||
|
||
import org.apache.kafka.common.header.Headers; | ||
import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; | ||
import org.apache.kafka.streams.processor.Punctuator; | ||
import org.apache.kafka.streams.processor.TaskId; | ||
|
||
/** | ||
* This interface allows user code to inspect the context of a record that has failed processing. | ||
*/ | ||
public interface ErrorHandlerContext { | ||
/** | ||
* Return the topic name of the current input record; could be {@code null} if it is not | ||
* available. | ||
* | ||
* <p> For example, if this method is invoked within a {@link Punctuator#punctuate(long) | ||
* punctuation callback}, or while processing a record that was forwarded by a punctuation | ||
* callback, the record won't have an associated topic. | ||
* Another example is | ||
* {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...)} | ||
* (and siblings), that do not always guarantee to provide a valid topic name, as they might be | ||
* executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL. | ||
* | ||
* @return the topic name | ||
*/ | ||
String topic(); | ||
|
||
/** | ||
* Return the partition ID of the current input record; could be {@code -1} if it is not | ||
* available. | ||
* | ||
* <p> For example, if this method is invoked within a {@link Punctuator#punctuate(long) | ||
* punctuation callback}, or while processing a record that was forwarded by a punctuation | ||
* callback, the record won't have an associated partition ID. | ||
* Another example is | ||
* {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...)} | ||
* (and siblings), that do not always guarantee to provide a valid partition ID, as they might be | ||
* executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL. | ||
* | ||
* @return the partition ID | ||
*/ | ||
int partition(); | ||
|
||
/** | ||
* Return the offset of the current input record; could be {@code -1} if it is not | ||
* available. | ||
* | ||
* <p> For example, if this method is invoked within a {@link Punctuator#punctuate(long) | ||
* punctuation callback}, or while processing a record that was forwarded by a punctuation | ||
* callback, the record won't have an associated offset. | ||
* Another example is | ||
* {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...)} | ||
* (and siblings), that do not always guarantee to provide a valid offset, as they might be | ||
* executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL. | ||
* | ||
* @return the offset | ||
*/ | ||
long offset(); | ||
|
||
/** | ||
* Return the headers of the current source record; could be an empty header if it is not | ||
* available. | ||
* | ||
* <p> For example, if this method is invoked within a {@link Punctuator#punctuate(long) | ||
* punctuation callback}, or while processing a record that was forwarded by a punctuation | ||
* callback, the record might not have any associated headers. | ||
* Another example is | ||
* {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...)} | ||
* (and siblings), that do not always guarantee to provide valid headers, as they might be | ||
* executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL. | ||
* | ||
* @return the headers | ||
*/ | ||
Headers headers(); | ||
|
||
/** | ||
* Return the non-deserialized byte[] of the input message key if the context has been triggered by a message. | ||
* | ||
* <p> If this method is invoked within a {@link Punctuator#punctuate(long) | ||
* punctuation callback}, or while processing a record that was forwarded by a punctuation | ||
* callback, it will return {@code null}. | ||
* | ||
* <p> If this method is invoked in a sub-topology due to a repartition, the returned key would be one sent | ||
* to the repartition topic. | ||
* | ||
* @return the raw byte of the key of the source message | ||
*/ | ||
byte[] sourceRawKey(); | ||
|
||
/** | ||
* Return the non-deserialized byte[] of the input message value if the context has been triggered by a message. | ||
* | ||
* <p> If this method is invoked within a {@link Punctuator#punctuate(long) | ||
* punctuation callback}, or while processing a record that was forwarded by a punctuation | ||
* callback, it will return {@code null}. | ||
* | ||
* <p> If this method is invoked in a sub-topology due to a repartition, the returned value would be one sent | ||
* to the repartition topic. | ||
* | ||
* @return the raw byte of the value of the source message | ||
*/ | ||
byte[] sourceRawValue(); | ||
|
||
/** | ||
* Return the current processor node ID. | ||
* | ||
* @return the processor node ID | ||
*/ | ||
String processorNodeId(); | ||
|
||
/** | ||
* Return the task ID. | ||
* | ||
* @return the task ID | ||
*/ | ||
TaskId taskId(); | ||
} |
46 changes: 46 additions & 0 deletions
46
...c/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,46 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.kafka.streams.errors; | ||
|
||
import org.apache.kafka.streams.processor.api.Record; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.util.Map; | ||
|
||
/** | ||
* Processing exception handler that logs a processing exception and then | ||
* signals the processing pipeline to continue processing more records. | ||
*/ | ||
public class LogAndContinueProcessingExceptionHandler implements ProcessingExceptionHandler { | ||
private static final Logger log = LoggerFactory.getLogger(LogAndContinueProcessingExceptionHandler.class); | ||
|
||
@Override | ||
public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record<?, ?> record, final Exception exception) { | ||
log.warn("Exception caught during message processing, " + | ||
"processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", | ||
context.processorNodeId(), context.taskId(), context.topic(), context.partition(), context.offset(), | ||
exception); | ||
|
||
return ProcessingHandlerResponse.CONTINUE; | ||
} | ||
|
||
@Override | ||
public void configure(final Map<String, ?> configs) { | ||
// ignore | ||
} | ||
} |
46 changes: 46 additions & 0 deletions
46
...s/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,46 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.kafka.streams.errors; | ||
|
||
import org.apache.kafka.streams.processor.api.Record; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.util.Map; | ||
|
||
/** | ||
* Processing exception handler that logs a processing exception and then | ||
* signals the processing pipeline to stop processing more records and fail. | ||
*/ | ||
public class LogAndFailProcessingExceptionHandler implements ProcessingExceptionHandler { | ||
private static final Logger log = LoggerFactory.getLogger(LogAndFailProcessingExceptionHandler.class); | ||
|
||
@Override | ||
public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record<?, ?> record, final Exception exception) { | ||
log.warn("Exception caught during message processing, " + | ||
"processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", | ||
context.processorNodeId(), context.taskId(), context.topic(), context.partition(), context.offset(), | ||
exception); | ||
|
||
return ProcessingHandlerResponse.FAIL; | ||
} | ||
|
||
@Override | ||
public void configure(final Map<String, ?> configs) { | ||
// ignore | ||
} | ||
} |
56 changes: 56 additions & 0 deletions
56
streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,56 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.kafka.streams.errors; | ||
|
||
import org.apache.kafka.common.Configurable; | ||
import org.apache.kafka.streams.processor.api.Record; | ||
|
||
/** | ||
* An interface that allows user code to inspect a record that has failed processing | ||
*/ | ||
public interface ProcessingExceptionHandler extends Configurable { | ||
/** | ||
* Inspect a record and the exception received | ||
* | ||
* @param context processing context metadata | ||
* @param record record where the exception occurred | ||
* @param exception the actual exception | ||
*/ | ||
ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record<?, ?> record, final Exception exception); | ||
|
||
enum ProcessingHandlerResponse { | ||
/* continue with processing */ | ||
CONTINUE(1, "CONTINUE"), | ||
/* fail the processing and stop */ | ||
FAIL(2, "FAIL"); | ||
|
||
/** | ||
* the permanent and immutable name of processing exception response | ||
*/ | ||
public final String name; | ||
|
||
/** | ||
* the permanent and immutable id of processing exception response | ||
*/ | ||
public final int id; | ||
|
||
ProcessingHandlerResponse(final int id, final String name) { | ||
this.id = id; | ||
this.name = name; | ||
} | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Could you please add unit tests in
StreamsConfigTest
?I think you need the following unit tests:
processingExceptionHandler()
.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.
hi,
It is done !