Skip to content
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
Show file tree
Hide file tree
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 May 27, 2024
b957861
KAFKA-16448: Add ProcessingExceptionHandler in Streams configuration
loicgreffier May 27, 2024
9eef86d
KAFKA-16448: Add ProcessingExceptionHandler interface and implementat…
loicgreffier May 27, 2024
40fdac5
KAFKA-16448: Update from parent branch
loicgreffier May 28, 2024
3cb797b
KAFKA-16448: Fix indentation
loicgreffier May 28, 2024
c1c0d5c
KAFKA-16448: Remove unecessary classes
loicgreffier May 28, 2024
bea1fb8
KAFKA-16448: add Streams configuration test for ProcessingExceptionHa…
sebastienviale May 29, 2024
b7058d2
KAFKA-16448: add Streams configuration test for ProcessingExceptionHa…
sebastienviale May 29, 2024
0a54bae
Merge branch 'KAFKA-16448-Add-Processing-Exception-Handler-StreamsCon…
sebastienviale May 29, 2024
014b799
Merge branch 'KAFKA-16448-Add-Processing-Exception-Handler-StreamsCon…
sebastienviale May 29, 2024
10e69a9
Merge branch 'KAFKA-16448-Add-Processing-Exception-Handler-StreamsCon…
sebastienviale May 29, 2024
3c7de0c
KAFKA-16448: add Streams configuration test for invalid ProcessingExc…
sebastienviale May 29, 2024
5e7a656
KAFKA-16448: add error message test in case of invalid ProcessingExce…
sebastienviale May 29, 2024
60bba3b
KAFKA-16448: Add ProcessingExceptionHandler in Streams configuration
loicgreffier May 27, 2024
25115ce
KAFKA-16448: Add ProcessingExceptionHandler in Streams configuration
loicgreffier May 29, 2024
4b56def
Merge branch 'KAFKA-16448-Add-Processing-Exception-Handler-StreamsCon…
sebastienviale May 29, 2024
7d67257
KAFKA-16448: remove unused imports
sebastienviale May 29, 2024
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
16 changes: 16 additions & 0 deletions streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,8 @@
import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler;
import org.apache.kafka.streams.errors.DeserializationExceptionHandler;
import org.apache.kafka.streams.errors.LogAndFailExceptionHandler;
import org.apache.kafka.streams.errors.LogAndFailProcessingExceptionHandler;
import org.apache.kafka.streams.errors.ProcessingExceptionHandler;
import org.apache.kafka.streams.errors.ProductionExceptionHandler;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.internals.StreamsConfigUtils;
Expand Down Expand Up @@ -553,6 +555,11 @@ public class StreamsConfig extends AbstractConfig {
public static final String DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG = "default.production.exception.handler";
private static final String DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the <code>org.apache.kafka.streams.errors.ProductionExceptionHandler</code> interface.";

/** {@code processing.exception.handler} */
@SuppressWarnings("WeakerAccess")
public static final String PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG = "processing.exception.handler";
public static final String PROCESSING_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the <code>org.apache.kafka.streams.errors.ProcessingExceptionHandler</code> interface.";

/** {@code default.dsl.store} */
@Deprecated
@SuppressWarnings("WeakerAccess")
Expand Down Expand Up @@ -926,6 +933,11 @@ public class StreamsConfig extends AbstractConfig {
DefaultProductionExceptionHandler.class.getName(),
Importance.MEDIUM,
DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC)
.define(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG,
Type.CLASS,
LogAndFailProcessingExceptionHandler.class.getName(),
Importance.MEDIUM,
PROCESSING_EXCEPTION_HANDLER_CLASS_DOC)
Comment on lines +941 to +945
Copy link
Contributor

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:

  • One where config is not set to verify the default value
  • One where the config is set and you get the correct instance from processingExceptionHandler().

Copy link
Contributor

Choose a reason for hiding this comment

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

hi,
It is done !

.define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG,
Type.CLASS,
FailOnInvalidTimestamp.class.getName(),
Expand Down Expand Up @@ -1915,6 +1927,10 @@ public ProductionExceptionHandler defaultProductionExceptionHandler() {
return getConfiguredInstance(DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, ProductionExceptionHandler.class);
}

public ProcessingExceptionHandler processingExceptionHandler() {
return getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class);
}

/**
* Override any client properties in the original configs with overrides
*
Expand Down
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();
}
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
}
}
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
}
}
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;
}
}
}
Loading