-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-16448: Catch and handle processing exceptions #16093
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
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 |
|---|---|---|
| @@ -0,0 +1,31 @@ | ||
| /* | ||
| * 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.internals; | ||
|
|
||
| import org.apache.kafka.common.KafkaException; | ||
|
|
||
| /** | ||
| * {@link FailedProcessingException} is the top-level exception type generated by Kafka Streams, and indicates errors have | ||
| * occurred during a {@link org.apache.kafka.streams.processor.internals.ProcessorNode ProcessorNode's} processing. | ||
| */ | ||
| public class FailedProcessingException extends KafkaException { | ||
|
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. Should we extend
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. Good point! Agreed! |
||
| private static final long serialVersionUID = 1L; | ||
|
|
||
| public FailedProcessingException(final Throwable throwable) { | ||
|
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. Do we really want to catch
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 missed that! Agreed! |
||
| super(throwable); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,22 +16,36 @@ | |
| */ | ||
| package org.apache.kafka.streams.processor.internals; | ||
|
|
||
| import org.apache.kafka.common.metrics.Sensor; | ||
| import org.apache.kafka.streams.errors.ErrorHandlerContext; | ||
| import org.apache.kafka.streams.errors.ProcessingExceptionHandler; | ||
| import org.apache.kafka.streams.errors.StreamsException; | ||
| import org.apache.kafka.streams.errors.TaskCorruptedException; | ||
| import org.apache.kafka.streams.errors.TaskMigratedException; | ||
| import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; | ||
| import org.apache.kafka.streams.errors.internals.FailedProcessingException; | ||
| import org.apache.kafka.streams.processor.Punctuator; | ||
| import org.apache.kafka.streams.processor.api.FixedKeyProcessor; | ||
| import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext; | ||
| import org.apache.kafka.streams.processor.api.InternalFixedKeyRecordFactory; | ||
| import org.apache.kafka.streams.processor.api.Processor; | ||
| import org.apache.kafka.streams.processor.api.Record; | ||
| import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; | ||
|
|
||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.HashMap; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Set; | ||
|
|
||
| import static org.apache.kafka.streams.StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG; | ||
|
|
||
| public class ProcessorNode<KIn, VIn, KOut, VOut> { | ||
|
|
||
| private final Logger log = LoggerFactory.getLogger(ProcessorNode.class); | ||
| private final List<ProcessorNode<KOut, VOut, ?, ?>> children; | ||
| private final Map<String, ProcessorNode<KOut, VOut, ?, ?>> childByName; | ||
|
|
||
|
|
@@ -40,12 +54,15 @@ public class ProcessorNode<KIn, VIn, KOut, VOut> { | |
| private final String name; | ||
|
|
||
| public final Set<String> stateStores; | ||
| private ProcessingExceptionHandler processingExceptionHandler; | ||
|
|
||
| private InternalProcessorContext<KOut, VOut> internalProcessorContext; | ||
| private String threadId; | ||
|
|
||
| private boolean closed = true; | ||
|
|
||
| private Sensor droppedRecordsSensor; | ||
|
|
||
| public ProcessorNode(final String name) { | ||
| this(name, (Processor<KIn, VIn, KOut, VOut>) null, null); | ||
| } | ||
|
|
@@ -98,6 +115,10 @@ public void init(final InternalProcessorContext<KOut, VOut> context) { | |
| try { | ||
| threadId = Thread.currentThread().getName(); | ||
| internalProcessorContext = context; | ||
| droppedRecordsSensor = TaskMetrics.droppedRecordsSensor(threadId, | ||
| internalProcessorContext.taskId().toString(), | ||
| internalProcessorContext.metrics()); | ||
|
|
||
| if (processor != null) { | ||
| processor.init(context); | ||
| } | ||
|
|
@@ -115,6 +136,11 @@ public void init(final InternalProcessorContext<KOut, VOut> context) { | |
| closed = false; | ||
| } | ||
|
|
||
| public void init(final InternalProcessorContext<KOut, VOut> context, final ProcessingExceptionHandler processingExceptionHandler) { | ||
| init(context); | ||
| this.processingExceptionHandler = processingExceptionHandler; | ||
| } | ||
|
|
||
| public void close() { | ||
| throwIfClosed(); | ||
|
|
||
|
|
@@ -174,6 +200,32 @@ public void process(final Record<KIn, VIn> record) { | |
| keyClass, | ||
| valueClass), | ||
| e); | ||
| } catch (final FailedProcessingException | TaskCorruptedException | TaskMigratedException e) { | ||
| // Rethrow exceptions that should not be handled here | ||
| throw e; | ||
| } catch (final Exception e) { | ||
| final ErrorHandlerContext errorHandlerContext = new DefaultErrorHandlerContext( | ||
| internalProcessorContext.topic(), | ||
| internalProcessorContext.partition(), | ||
| internalProcessorContext.offset(), | ||
| internalProcessorContext.headers(), | ||
| internalProcessorContext.recordContext().rawRecord().key(), | ||
| internalProcessorContext.recordContext().rawRecord().value(), | ||
| internalProcessorContext.currentNode().name(), | ||
| internalProcessorContext.taskId()); | ||
|
|
||
| final ProcessingExceptionHandler.ProcessingHandlerResponse response = processingExceptionHandler | ||
| .handle(errorHandlerContext, record, e); | ||
|
|
||
| if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { | ||
| log.error("Processing exception handler is set to fail upon" + | ||
| " a processing error. If you would rather have the streaming pipeline" + | ||
| " continue after a processing error, please set the " + | ||
| PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately."); | ||
| throw new FailedProcessingException(e); | ||
| } else { | ||
| droppedRecordsSensor.record(); | ||
|
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. Should we log here? How do we do this elsewhere? I know that the default handlers log already (so might be redundant), but a custom handler might not log...? Just curious.
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 also thought about logging here. But since we had quite some issues with log pressure due to dropped records, I thought, it would be better to let the handler log. |
||
| } | ||
| } | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,9 +20,11 @@ | |
| import org.apache.kafka.common.header.Headers; | ||
|
|
||
| public class StampedRecord extends Stamped<ConsumerRecord<?, ?>> { | ||
| private final ConsumerRecord<byte[], byte[]> rawRecord; | ||
|
|
||
| public StampedRecord(final ConsumerRecord<?, ?> record, final long timestamp) { | ||
| public StampedRecord(final ConsumerRecord<?, ?> record, final long timestamp, final ConsumerRecord<byte[], byte[]> rawRecord) { | ||
| super(record, timestamp); | ||
| this.rawRecord = rawRecord; | ||
| } | ||
|
|
||
| public String topic() { | ||
|
|
@@ -49,6 +51,20 @@ public Headers headers() { | |
| return value.headers(); | ||
| } | ||
|
|
||
| public ConsumerRecord<byte[], byte[]> rawRecord() { | ||
| return rawRecord; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean equals(final Object other) { | ||
|
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 add this overload? (Same for
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. We have already had that. See #16093 (comment)
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 see. Thanks. "Interesting" -- or should I rather say "annoying" 😁 |
||
| return super.equals(other); | ||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() { | ||
| return super.hashCode(); | ||
| } | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| return value.toString() + ", timestamp = " + timestamp; | ||
|
|
||
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.
Why do we need to use a supplier here?
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.
Because Streams creates a processing exception handler per task.
See
kafka/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java
Line 288 in a5bfc21
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.
Sure, but line 288 could just be
Not sure why we need the supplier indirection?
For the existing
timestampExtractorSupplieranddeserializationExceptionHandlerSupplieris a little different because we setup different suppliers for different cases (so using suppliers simplifies the code), but for the new one we have only one supplier and thus it seem unnecessary.No need to change anything. We follow an established pattern, but it just was confusing to me, as it seems (strictly speaking) unnecessary to have the layer of indirection.
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.
processingExceptionHandlerhas been created following thedeserializationExceptionHandler. Thus a supplier has been used.The supplier can be removed to make the code clearer if needed.
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.
Yes. Understood. -- I think it's not necessary to have a supplier, but also ok to keep the code as-is.