diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
index 70e4cba9625ac..5685fc4b40720 100644
--- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
+++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
@@ -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.ProcessingExceptionHandler;
+import org.apache.kafka.streams.errors.ProcessingLogAndFailExceptionHandler;
import org.apache.kafka.streams.errors.ProductionExceptionHandler;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.internals.StreamsConfigUtils;
@@ -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 org.apache.kafka.streams.errors.ProductionExceptionHandler 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 org.apache.kafka.streams.errors.ProcessingExceptionHandler interface.";
+
/** {@code default.dsl.store} */
@Deprecated
@SuppressWarnings("WeakerAccess")
@@ -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,
+ ProcessingLogAndFailExceptionHandler.class.getName(),
+ Importance.MEDIUM,
+ PROCESSING_EXCEPTION_HANDLER_CLASS_DOC)
.define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG,
Type.CLASS,
FailOnInvalidTimestamp.class.getName(),
@@ -1915,6 +1927,11 @@ public ProductionExceptionHandler defaultProductionExceptionHandler() {
return getConfiguredInstance(DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, ProductionExceptionHandler.class);
}
+ @SuppressWarnings("WeakerAccess")
+ public ProcessingExceptionHandler processingExceptionHandler() {
+ return getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class);
+ }
+
/**
* Override any client properties in the original configs with overrides
*
diff --git a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java
index 2d4157eba7885..b333de60bc7ad 100644
--- a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java
+++ b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java
@@ -22,15 +22,16 @@
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.errors.DeserializationExceptionHandler;
+import org.apache.kafka.streams.errors.ProcessingExceptionHandler;
import org.apache.kafka.streams.internals.StreamsConfigUtils;
import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.kstream.internals.MaterializedInternal;
import org.apache.kafka.streams.processor.TimestampExtractor;
-
import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper;
import org.apache.kafka.streams.state.DslStoreSuppliers;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+
import java.util.Optional;
import java.util.Properties;
import java.util.function.Supplier;
@@ -57,6 +58,7 @@
import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_DOC;
import static org.apache.kafka.streams.StreamsConfig.ROCKS_DB;
import static org.apache.kafka.streams.StreamsConfig.IN_MEMORY;
+import static org.apache.kafka.streams.StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG;
import static org.apache.kafka.streams.internals.StreamsConfigUtils.getTotalCacheSize;
/**
@@ -135,6 +137,7 @@ public class TopologyConfig extends AbstractConfig {
public final Class> dslStoreSuppliers;
public final Supplier timestampExtractorSupplier;
public final Supplier deserializationExceptionHandlerSupplier;
+ public final Supplier processingExceptionHandler;
public TopologyConfig(final StreamsConfig globalAppConfigs) {
this(null, globalAppConfigs, new Properties());
@@ -225,6 +228,13 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo
deserializationExceptionHandlerSupplier = () -> globalAppConfigs.getConfiguredInstance(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, DeserializationExceptionHandler.class);
}
+ if (isTopologyOverride(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, topologyOverrides)) {
+ processingExceptionHandler = () -> getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class);
+ log.info("Topology {} is overriding {} to {}", topologyName, PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, getClass(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG));
+ } else {
+ processingExceptionHandler = () -> globalAppConfigs.getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class);
+ }
+
if (isTopologyOverride(DEFAULT_DSL_STORE_CONFIG, topologyOverrides)) {
storeType = getString(DEFAULT_DSL_STORE_CONFIG);
log.info("Topology {} is overriding {} to {}", topologyName, DEFAULT_DSL_STORE_CONFIG, storeType);
@@ -280,7 +290,8 @@ public TaskConfig getTaskConfig() {
maxBufferedSize,
timestampExtractorSupplier.get(),
deserializationExceptionHandlerSupplier.get(),
- eosEnabled
+ eosEnabled,
+ processingExceptionHandler.get()
);
}
@@ -291,19 +302,22 @@ public static class TaskConfig {
public final TimestampExtractor timestampExtractor;
public final DeserializationExceptionHandler deserializationExceptionHandler;
public final boolean eosEnabled;
+ public final ProcessingExceptionHandler processingExceptionHandler;
private TaskConfig(final long maxTaskIdleMs,
final long taskTimeoutMs,
final int maxBufferedSize,
final TimestampExtractor timestampExtractor,
final DeserializationExceptionHandler deserializationExceptionHandler,
- final boolean eosEnabled) {
+ final boolean eosEnabled,
+ final ProcessingExceptionHandler processingExceptionHandler) {
this.maxTaskIdleMs = maxTaskIdleMs;
this.taskTimeoutMs = taskTimeoutMs;
this.maxBufferedSize = maxBufferedSize;
this.timestampExtractor = timestampExtractor;
this.deserializationExceptionHandler = deserializationExceptionHandler;
this.eosEnabled = eosEnabled;
+ this.processingExceptionHandler = processingExceptionHandler;
}
}
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/DefaultProductionExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/DefaultProductionExceptionHandler.java
index 4fdb1a3fc0a95..a976346141277 100644
--- a/streams/src/main/java/org/apache/kafka/streams/errors/DefaultProductionExceptionHandler.java
+++ b/streams/src/main/java/org/apache/kafka/streams/errors/DefaultProductionExceptionHandler.java
@@ -18,6 +18,7 @@
import java.util.Map;
import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.streams.processor.ErrorHandlerContext;
/**
* {@code ProductionExceptionHandler} that always instructs streams to fail when an exception
@@ -25,7 +26,8 @@
*/
public class DefaultProductionExceptionHandler implements ProductionExceptionHandler {
@Override
- public ProductionExceptionHandlerResponse handle(final ProducerRecord record,
+ public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context,
+ final ProducerRecord record,
final Exception exception) {
return ProductionExceptionHandlerResponse.FAIL;
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java
index 95ccfeced8e43..6b2cc335c34e8 100644
--- a/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java
+++ b/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java
@@ -16,10 +16,11 @@
*/
package org.apache.kafka.streams.errors;
-
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.common.Configurable;
+import org.apache.kafka.streams.processor.ErrorHandlerContext;
import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.internals.ErrorHandlerContextImpl;
/**
* Interface that specifies how an exception from source node deserialization
@@ -37,11 +38,28 @@ public interface DeserializationExceptionHandler extends Configurable {
* @param context processor context
* @param record record that failed deserialization
* @param exception the actual exception
+ * @deprecated Please {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)}
*/
- @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
- DeserializationHandlerResponse handle(final ProcessorContext context,
- final ConsumerRecord record,
- final Exception exception);
+ @Deprecated
+ default DeserializationHandlerResponse handle(final ProcessorContext context,
+ final ConsumerRecord record,
+ final Exception exception) {
+
+ throw new UnsupportedOperationException();
+ }
+ /**
+ * Inspect a record and the exception received.
+ *
+ * @param context error handler context
+ * @param record record that failed deserialization
+ * @param exception the actual exception
+ */
+ default DeserializationHandlerResponse handle(final ErrorHandlerContext context,
+ final ConsumerRecord record,
+ final Exception exception) {
+
+ return handle(((ErrorHandlerContextImpl) context).convertToProcessorContext(), record, exception);
+ }
/**
* Enumeration that describes the response from the exception handler.
diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java
index 4f9a0964405a2..0494b44500210 100644
--- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java
+++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java
@@ -18,6 +18,7 @@
import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.streams.processor.ErrorHandlerContext;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -32,6 +33,7 @@ public class LogAndContinueExceptionHandler implements DeserializationExceptionH
private static final Logger log = LoggerFactory.getLogger(LogAndContinueExceptionHandler.class);
@Override
+ @Deprecated
public DeserializationHandlerResponse handle(final ProcessorContext context,
final ConsumerRecord record,
final Exception exception) {
@@ -44,6 +46,19 @@ public DeserializationHandlerResponse handle(final ProcessorContext context,
return DeserializationHandlerResponse.CONTINUE;
}
+ @Override
+ public DeserializationHandlerResponse handle(final ErrorHandlerContext context,
+ final ConsumerRecord record,
+ final Exception exception) {
+
+ log.warn("Exception caught during Deserialization, " +
+ "taskId: {}, topic: {}, partition: {}, offset: {}",
+ context.taskId(), record.topic(), record.partition(), record.offset(),
+ exception);
+
+ return DeserializationHandlerResponse.CONTINUE;
+ }
+
@Override
public void configure(final Map configs) {
// ignore
diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java
index 61d210649ba9a..5fc197a10b03e 100644
--- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java
+++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java
@@ -17,6 +17,7 @@
package org.apache.kafka.streams.errors;
import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.streams.processor.ErrorHandlerContext;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -32,6 +33,7 @@ public class LogAndFailExceptionHandler implements DeserializationExceptionHandl
private static final Logger log = LoggerFactory.getLogger(LogAndFailExceptionHandler.class);
@Override
+ @Deprecated
public DeserializationHandlerResponse handle(final ProcessorContext context,
final ConsumerRecord record,
final Exception exception) {
@@ -44,6 +46,18 @@ public DeserializationHandlerResponse handle(final ProcessorContext context,
return DeserializationHandlerResponse.FAIL;
}
+ public DeserializationHandlerResponse handle(final ErrorHandlerContext context,
+ final ConsumerRecord record,
+ final Exception exception) {
+
+ log.error("Exception caught during Deserialization, " +
+ "taskId: {}, topic: {}, partition: {}, offset: {}",
+ context.taskId(), record.topic(), record.partition(), record.offset(),
+ exception);
+
+ return DeserializationHandlerResponse.FAIL;
+ }
+
@Override
public void configure(final Map configs) {
// ignore
diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java
new file mode 100644
index 0000000000000..acf4c58fbc481
--- /dev/null
+++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java
@@ -0,0 +1,57 @@
+/*
+ * 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.ErrorHandlerContext;
+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(ErrorHandlerContext context, Record, ?> record, 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;
+ }
+ }
+}
diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java
new file mode 100644
index 0000000000000..93d6a330c4bab
--- /dev/null
+++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java
@@ -0,0 +1,47 @@
+/*
+ * 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.ErrorHandlerContext;
+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 ProcessingLogAndContinueExceptionHandler implements ProcessingExceptionHandler {
+ private static final Logger log = LoggerFactory.getLogger(ProcessingLogAndContinueExceptionHandler.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 configs) {
+ // ignore
+ }
+}
diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java
new file mode 100644
index 0000000000000..facb300a9692b
--- /dev/null
+++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java
@@ -0,0 +1,47 @@
+/*
+ * 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.ErrorHandlerContext;
+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 ProcessingLogAndFailExceptionHandler implements ProcessingExceptionHandler {
+ private static final Logger log = LoggerFactory.getLogger(ProcessingLogAndFailExceptionHandler.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 configs) {
+ // ignore
+ }
+}
diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java
index 6ae0170bfc906..71b160b5d2d1c 100644
--- a/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java
+++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java
@@ -18,6 +18,7 @@
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.Configurable;
+import org.apache.kafka.streams.processor.ErrorHandlerContext;
/**
* Interface that specifies how an exception when attempting to produce a result to
@@ -30,22 +31,60 @@ public interface ProductionExceptionHandler extends Configurable {
*
* @param record The record that failed to produce
* @param exception The exception that occurred during production
+ * @deprecated Use {@link #handle(ErrorHandlerContext, ProducerRecord, Exception)} instead
*/
- ProductionExceptionHandlerResponse handle(final ProducerRecord record,
- final Exception exception);
+ @Deprecated
+ default ProductionExceptionHandlerResponse handle(final ProducerRecord record,
+ final Exception exception) {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Inspect a record that we attempted to produce, and the exception that resulted
+ * from attempting to produce it and determine whether or not to continue processing.
+ *
+ * @param context The error handler context metadata
+ * @param record The record that failed to produce
+ * @param exception The exception that occurred during production
+ */
+ @SuppressWarnings("deprecation")
+ default ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context,
+ final ProducerRecord record,
+ final Exception exception) {
+ return handle(record, exception);
+ }
/**
* Handles serialization exception and determine if the process should continue. The default implementation is to
* fail the process.
*
- * @param record the record that failed to serialize
- * @param exception the exception that occurred during serialization
+ * @param record the record that failed to serialize
+ * @param exception the exception that occurred during serialization
+ * @deprecated Use {@link #handle(ErrorHandlerContext, ProducerRecord, Exception)} instead
*/
+ @Deprecated
default ProductionExceptionHandlerResponse handleSerializationException(final ProducerRecord record,
final Exception exception) {
return ProductionExceptionHandlerResponse.FAIL;
}
+ /**
+ * Handles serialization exception and determine if the process should continue. The default implementation is to
+ * fail the process.
+ *
+ * @param context the error handler context metadata
+ * @param record the record that failed to serialize
+ * @param exception the exception that occurred during serialization
+ * @param origin the origin of the serialization exception
+ */
+ @SuppressWarnings("deprecation")
+ default ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context,
+ final ProducerRecord record,
+ final Exception exception,
+ final SerializationExceptionOrigin origin) {
+ return handleSerializationException(record, exception);
+ }
+
enum ProductionExceptionHandlerResponse {
/* continue processing */
CONTINUE(0, "CONTINUE"),
@@ -68,4 +107,11 @@ enum ProductionExceptionHandlerResponse {
this.name = name;
}
}
+
+ enum SerializationExceptionOrigin {
+ /* serialization exception occurred during serialization of the key */
+ KEY,
+ /* serialization exception occurred during serialization of the value */
+ VALUE
+ }
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java
new file mode 100644
index 0000000000000..5a852d415a94c
--- /dev/null
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java
@@ -0,0 +1,139 @@
+/*
+ * 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.processor;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier;
+
+
+/**
+ * ErrorHandlerContext interface
+ */
+public interface ErrorHandlerContext {
+
+ /**
+ * Return the topic name of the current input record; could be {@code null} if it is not
+ * available.
+ *
+ *
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.
+ *
+ *
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.
+ *
+ *
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.
+ *
+ *
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.
+ *
+ *
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 null.
+ *
+ *
If this method is invoked in a sub-topology due to a repartition, the returned key would be one sent
+ * to the repartition topic.
+ *
+ *
Always returns null if this method is invoked within a
+ * {@link org.apache.kafka.streams.errors.ProductionExceptionHandler#handle(ErrorHandlerContext, org.apache.kafka.clients.producer.ProducerRecord, Exception)}
+ *
+ * @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.
+ *
+ *
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 null.
+ *
+ *
If this method is invoked in a sub-topology due to a repartition, the returned value would be one sent
+ * to the repartition topic.
+ *
+ *
Always returns null if this method is invoked within a
+ * {@link org.apache.kafka.streams.errors.ProductionExceptionHandler#handle(ErrorHandlerContext, org.apache.kafka.clients.producer.ProducerRecord, Exception)}
+ *
+ * @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();
+}
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractPartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractPartitionGroup.java
index 12af39ca4dbd3..7183526d27ca0 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractPartitionGroup.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractPartitionGroup.java
@@ -55,6 +55,8 @@ abstract class AbstractPartitionGroup {
abstract Long headRecordOffset(final TopicPartition partition);
+ abstract ConsumerRecord rawHeadRecord();
+
abstract int numBuffered();
abstract int numBuffered(TopicPartition tp);
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java
new file mode 100644
index 0000000000000..0b4830cd13df6
--- /dev/null
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java
@@ -0,0 +1,110 @@
+/*
+ * 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.processor.internals;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.streams.processor.ErrorHandlerContext;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.TaskId;
+
+public class ErrorHandlerContextImpl implements ErrorHandlerContext {
+ private InternalProcessorContext processorContext;
+ private final String topic;
+ private final int partition;
+ private final long offset;
+ private final Headers headers;
+ private final byte[] sourceRawKey;
+ private final byte[] sourceRawValue;
+ private final String processorNodeId;
+ private final TaskId taskId;
+
+ public ErrorHandlerContextImpl(final String topic,
+ final int partition,
+ final long offset,
+ final Headers headers,
+ final byte[] sourceRawKey,
+ final byte[] sourceRawValue,
+ final String processorNodeId,
+ final TaskId taskId) {
+ this(null, topic, partition, offset, headers, sourceRawKey, sourceRawValue, processorNodeId, taskId);
+ }
+
+ public ErrorHandlerContextImpl(final InternalProcessorContext processorContext,
+ final String topic,
+ final int partition,
+ final long offset,
+ final Headers headers,
+ final byte[] sourceRawKey,
+ final byte[] sourceRawValue,
+ final String processorNodeId,
+ final TaskId taskId) {
+ this.processorContext = processorContext;
+ this.topic = topic;
+ this.partition = partition;
+ this.offset = offset;
+ this.headers = headers;
+ this.sourceRawKey = sourceRawKey;
+ this.sourceRawValue = sourceRawValue;
+ this.processorNodeId = processorNodeId;
+ this.taskId = taskId;
+ }
+
+ @Override
+ public String topic() {
+ return this.topic;
+ }
+
+ @Override
+ public int partition() {
+ return this.partition;
+ }
+
+ @Override
+ public long offset() {
+ return this.offset;
+ }
+
+ @Override
+ public Headers headers() {
+ return this.headers;
+ }
+
+ @Override
+ public byte[] sourceRawKey() {
+ return this.sourceRawKey;
+ }
+
+ @Override
+ public byte[] sourceRawValue() {
+ return this.sourceRawValue;
+ }
+
+ @Override
+ public String processorNodeId() {
+ return this.processorNodeId;
+ }
+
+ @Override
+ public TaskId taskId() {
+ return this.taskId;
+ }
+
+ public ProcessorContext convertToProcessorContext() {
+ return new RestrictiveProcessorContext(this.processorContext);
+
+ }
+}
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java
index d42e90bf33f4c..76a09985218af 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java
@@ -329,7 +329,8 @@ private void reprocessState(final List topicPartitions,
Thread.currentThread().getName(),
globalProcessorContext.taskId().toString(),
globalProcessorContext.metrics()
- )
+ ),
+ null
);
}
}
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java
index 4852ba97932e9..f52a1feb01785 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java
@@ -71,6 +71,7 @@ class PartitionGroup extends AbstractPartitionGroup {
private boolean allBuffered;
private final Map idlePartitionDeadlines = new HashMap<>();
private final Map fetchedLags = new HashMap<>();
+ private ConsumerRecord rawHeadRecord;
PartitionGroup(final LogContext logContext,
final Map partitionQueues,
@@ -248,6 +249,7 @@ StampedRecord nextRecord(final RecordInfo info, final long wallClockTime) {
if (queue != null) {
// get the first record from this queue.
+ rawHeadRecord = queue.rawHeadRecord();
record = queue.poll(wallClockTime);
if (record != null) {
@@ -321,6 +323,16 @@ Long headRecordOffset(final TopicPartition partition) {
return recordQueue.headRecordOffset();
}
+ /**
+ * Returns the raw head record
+ *
+ * @return the raw head record
+ */
+ @Override
+ ConsumerRecord rawHeadRecord() {
+ return rawHeadRecord;
+ }
+
/**
* @throws IllegalStateException if the record's partition does not belong to this partition group
*/
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
index b484d26f0fe87..7cbd643ae2f6f 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
@@ -20,10 +20,13 @@
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeader;
import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.errors.ProcessingExceptionHandler;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.processor.Cancellable;
+import org.apache.kafka.streams.processor.ErrorHandlerContext;
import org.apache.kafka.streams.processor.PunctuationType;
import org.apache.kafka.streams.processor.Punctuator;
import org.apache.kafka.streams.processor.StateStore;
@@ -33,10 +36,13 @@
import org.apache.kafka.streams.processor.api.Record;
import org.apache.kafka.streams.processor.internals.Task.TaskType;
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
import org.apache.kafka.streams.query.Position;
import org.apache.kafka.streams.state.internals.PositionSerde;
import org.apache.kafka.streams.state.internals.ThreadCache;
import org.apache.kafka.streams.state.internals.ThreadCache.DirtyEntryFlushListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import java.time.Duration;
import java.util.HashMap;
@@ -44,12 +50,15 @@
import java.util.Map;
import static org.apache.kafka.streams.StreamsConfig.InternalConfig.IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED;
+import static org.apache.kafka.streams.StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG;
import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix;
import static org.apache.kafka.streams.internals.ApiUtils.validateMillisecondDuration;
import static org.apache.kafka.streams.processor.internals.AbstractReadOnlyDecorator.getReadOnlyStore;
import static org.apache.kafka.streams.processor.internals.AbstractReadWriteDecorator.getReadWriteStore;
public class ProcessorContextImpl extends AbstractProcessorContext