From 0c23328e70b6cd2e84f57de24e9294edd3e7414e Mon Sep 17 00:00:00 2001 From: Damien Gasparina Date: Fri, 15 Nov 2024 17:20:00 +0100 Subject: [PATCH 01/16] KFK-16505 WIP Implementing KIP-1034 - adding new interfaces --- .../apache/kafka/streams/StreamsConfig.java | 9 ++ .../errors/CommonExceptionHandler.java | 93 +++++++++++++++++++ .../DefaultProductionExceptionHandler.java | 6 +- .../DeserializationExceptionHandler.java | 17 +++- .../LogAndContinueExceptionHandler.java | 6 +- ...AndContinueProcessingExceptionHandler.java | 6 +- .../errors/LogAndFailExceptionHandler.java | 6 +- .../LogAndFailProcessingExceptionHandler.java | 6 +- .../errors/ProcessingExceptionHandler.java | 18 ++++ .../errors/ProductionExceptionHandler.java | 17 ++++ .../processor/internals/ProcessorNode.java | 20 ++++ .../processor/internals/RecordCollector.java | 4 +- .../internals/RecordCollectorImpl.java | 10 +- .../internals/RecordDeserializer.java | 19 ++++ .../processor/internals/StreamTask.java | 19 ++++ .../state/KeyValueStoreTestDriver.java | 4 +- .../kafka/test/MockRecordCollector.java | 4 +- 17 files changed, 237 insertions(+), 27 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/CommonExceptionHandler.java 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 5e1eaff1162ba..018dd71972a4f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -619,6 +619,10 @@ public class StreamsConfig extends AbstractConfig { "support \"classic\" or \"streams\". If \"streams\" is specified, then the streams rebalance protocol will be " + "used. Otherwise, the classic group protocol will be used."; + public static final String ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG = "errors.dead.letter.queue.topic.name"; + public static final String ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_DOC = "If not null, the default exception handler will build and send a Dead Letter Queue record in the provided topic name if an error occurs.\n" + + "If a custom deserialization/production or processing exception handler is set, this parameter is ignored for this handler."; + /** {@code log.summary.interval.ms} */ public static final String LOG_SUMMARY_INTERVAL_MS_CONFIG = "log.summary.interval.ms"; private static final String LOG_SUMMARY_INTERVAL_MS_DOC = "The output interval in milliseconds for logging summary information.\n" + @@ -991,6 +995,11 @@ public class StreamsConfig extends AbstractConfig { LogAndFailExceptionHandler.class.getName(), Importance.MEDIUM, DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC) + .define(ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG, + Type.STRING, + null, + Importance.MEDIUM, + ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_DOC) .define(MAX_TASK_IDLE_MS_CONFIG, Type.LONG, 0L, diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/CommonExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/CommonExceptionHandler.java new file mode 100644 index 0000000000000..d3e7e14d882c7 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/CommonExceptionHandler.java @@ -0,0 +1,93 @@ +/* + * 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.clients.producer.ProducerRecord; +import org.apache.kafka.common.Configurable; +import org.apache.kafka.common.errors.InvalidConfigurationException; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.StreamsConfig; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.Collections; +import java.util.Map; + +public class CommonExceptionHandler implements Configurable { + protected String deadLetterQueueTopicName = null; + public static final String HEADER_ERRORS_EXCEPTION_NAME = "__streams.errors.exception"; + public static final String HEADER_ERRORS_STACKTRACE_NAME = "__streams.errors.stacktrace"; + public static final String HEADER_ERRORS_EXCEPTION_MESSAGE_NAME = "__streams.errors.message"; + public static final String HEADER_ERRORS_TOPIC_NAME = "__streams.errors.topic"; + public static final String HEADER_ERRORS_PARTITION_NAME = "__streams.errors.partition"; + public static final String HEADER_ERRORS_OFFSET_NAME = "__streams.errors.offset"; + + @Override + public void configure(final Map configs) { + if (configs.containsKey(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)) { + setDeadLetterQueueTopicName(String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG))); + } + } + + public boolean shouldBuildDeadLetterQueueRecord() { + return this.deadLetterQueueTopicName != null; + } + + public Iterable> maybeBuildDeadLetterQueueRecords(final byte[] key, + final byte[] value, + final ErrorHandlerContext context, + final Exception exception) { + if (!shouldBuildDeadLetterQueueRecord()) { + return Collections.emptyList(); + } + + return Collections.singleton(buildDeadLetterQueueRecord(key, value, context, exception)); + } + + public ProducerRecord buildDeadLetterQueueRecord(final byte[] key, + final byte[] value, + final ErrorHandlerContext context, + final Exception e) { + if (deadLetterQueueTopicName == null) { + throw new InvalidConfigurationException(String.format("%s can not be null while building DeadLetterQueue record", StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); + } + final ProducerRecord producerRecord = new ProducerRecord<>(deadLetterQueueTopicName, null, context.timestamp(), key, value); + final StringWriter stackStraceStringWriter = new StringWriter(); + final PrintWriter stackTracePrintWriter = new PrintWriter(stackStraceStringWriter); + e.printStackTrace(stackTracePrintWriter); + + try (final StringSerializer stringSerializer = new StringSerializer()) { + producerRecord.headers().add(HEADER_ERRORS_EXCEPTION_NAME, stringSerializer.serialize(null, e.toString())); + producerRecord.headers().add(HEADER_ERRORS_EXCEPTION_MESSAGE_NAME, stringSerializer.serialize(null, e.getMessage())); + producerRecord.headers().add(HEADER_ERRORS_STACKTRACE_NAME, stringSerializer.serialize(null, stackStraceStringWriter.toString())); + producerRecord.headers().add(HEADER_ERRORS_TOPIC_NAME, stringSerializer.serialize(null, context.topic())); + producerRecord.headers().add(HEADER_ERRORS_PARTITION_NAME, stringSerializer.serialize(null, String.valueOf(context.partition()))); + producerRecord.headers().add(HEADER_ERRORS_OFFSET_NAME, stringSerializer.serialize(null, String.valueOf(context.offset()))); + } + + return producerRecord; + } + + public String getDeadLetterQueueTopicName() { + return deadLetterQueueTopicName; + } + + public void setDeadLetterQueueTopicName(final String deadLetterQueueTopicName) { + this.deadLetterQueueTopicName = deadLetterQueueTopicName; + } +} \ No newline at end of file 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 5994326770c89..9d1a2636b2a9d 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 @@ -44,12 +44,12 @@ public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext conte final ProducerRecord record, final Exception exception) { return exception instanceof RetriableException ? - ProductionExceptionHandlerResponse.RETRY : - ProductionExceptionHandlerResponse.FAIL; + ProductionExceptionHandlerResponse.RETRY : + ProductionExceptionHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(null, null, context, exception)); } @Override public void configure(final Map configs) { - // ignore + super.configure(configs); } } \ No newline at end of file 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 0b44e04d79114..21f8f1c0f0d08 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 @@ -17,10 +17,14 @@ package org.apache.kafka.streams.errors; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Configurable; import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; import org.apache.kafka.streams.processor.ProcessorContext; +import java.util.LinkedList; +import java.util.List; + /** * Interface that specifies how an exception from source node deserialization * (e.g., reading from Kafka) should be handled. @@ -88,11 +92,22 @@ enum DeserializationHandlerResponse { * The permanent and immutable id for the used option. This can't change ever. */ public final int id; + public final List> deadLetterQueueRecords; DeserializationHandlerResponse(final int id, final String name) { this.id = id; this.name = name; + this.deadLetterQueueRecords = new LinkedList<>(); } - } + public DeserializationHandlerResponse andAddToDeadLetterQueue(final Iterable> deadLetterQueueRecords) { + if (deadLetterQueueRecords == null) { + return this; + } + for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { + this.deadLetterQueueRecords.add(deadLetterQueueRecord); + } + return this; + } + } } 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 6de997be98653..4531caafa97a7 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 @@ -28,7 +28,7 @@ * Deserialization handler that logs a deserialization exception and then * signals the processing pipeline to continue processing more records. */ -public class LogAndContinueExceptionHandler implements DeserializationExceptionHandler { +public class LogAndContinueExceptionHandler extends CommonExceptionHandler implements DeserializationExceptionHandler { private static final Logger log = LoggerFactory.getLogger(LogAndContinueExceptionHandler.class); /** @@ -67,11 +67,11 @@ public DeserializationHandlerResponse handle(final ErrorHandlerContext context, exception ); - return DeserializationHandlerResponse.CONTINUE; + return DeserializationHandlerResponse.CONTINUE.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(null, null, context, exception)); } @Override public void configure(final Map configs) { - // ignore + super.configure(configs); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java index c832ab142007c..2423fa9f073a6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java @@ -27,7 +27,7 @@ * Processing exception handler that logs a processing exception and then * signals the processing pipeline to continue processing more records. */ -public class LogAndContinueProcessingExceptionHandler implements ProcessingExceptionHandler { +public class LogAndContinueProcessingExceptionHandler extends CommonExceptionHandler implements ProcessingExceptionHandler { private static final Logger log = LoggerFactory.getLogger(LogAndContinueProcessingExceptionHandler.class); @Override @@ -42,11 +42,11 @@ public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final exception ); - return ProcessingHandlerResponse.CONTINUE; + return ProcessingHandlerResponse.CONTINUE.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(null, null, context, exception)); } @Override public void configure(final Map configs) { - // ignore + super.configure(configs); } } 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 20e6b9414de27..78ef2f6b8b314 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 @@ -28,7 +28,7 @@ * Deserialization handler that logs a deserialization exception and then * signals the processing pipeline to stop processing more records and fail. */ -public class LogAndFailExceptionHandler implements DeserializationExceptionHandler { +public class LogAndFailExceptionHandler extends CommonExceptionHandler implements DeserializationExceptionHandler { private static final Logger log = LoggerFactory.getLogger(LogAndFailExceptionHandler.class); /** @@ -67,11 +67,11 @@ public DeserializationHandlerResponse handle(final ErrorHandlerContext context, exception ); - return DeserializationHandlerResponse.FAIL; + return DeserializationHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(null, null, context, exception)); } @Override public void configure(final Map configs) { - // ignore + super.configure(configs); } } \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java index f592663a6c07a..0f7bab6065fa0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java @@ -27,7 +27,7 @@ * 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 { +public class LogAndFailProcessingExceptionHandler extends CommonExceptionHandler implements ProcessingExceptionHandler { private static final Logger log = LoggerFactory.getLogger(LogAndFailProcessingExceptionHandler.class); @Override @@ -42,11 +42,11 @@ public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final exception ); - return ProcessingHandlerResponse.FAIL; + return ProcessingHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(null, null, context, exception)); } @Override public void configure(final Map configs) { - // ignore + super.configure(configs); } } 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 index 7dc1b90bc2e9f..3732b8a2f57d7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java @@ -16,9 +16,13 @@ */ package org.apache.kafka.streams.errors; +import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Configurable; import org.apache.kafka.streams.processor.api.Record; +import java.util.LinkedList; +import java.util.List; + /** * An interface that allows user code to inspect a record that has failed processing */ @@ -53,9 +57,23 @@ enum ProcessingHandlerResponse { */ public final int id; + public final List> deadLetterQueueRecords; + ProcessingHandlerResponse(final int id, final String name) { this.id = id; this.name = name; + deadLetterQueueRecords = new LinkedList<>(); + } + + + public ProcessingExceptionHandler.ProcessingHandlerResponse andAddToDeadLetterQueue(final Iterable> deadLetterQueueRecords) { + if (deadLetterQueueRecords == null) { + return this; + } + for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { + this.deadLetterQueueRecords.add(deadLetterQueueRecord); + } + return this; } } } 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 ed6b38a5692f7..0a2ed69e0c423 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 @@ -19,6 +19,9 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Configurable; +import java.util.LinkedList; +import java.util.List; + /** * Interface that specifies how an exception when attempting to produce a result to * Kafka should be handled. @@ -140,11 +143,25 @@ enum ProductionExceptionHandlerResponse { */ public final int id; + public final List> deadLetterQueueRecords; + ProductionExceptionHandlerResponse(final int id, final String name) { this.id = id; this.name = name; + deadLetterQueueRecords = new LinkedList<>(); } + + public ProductionExceptionHandler.ProductionExceptionHandlerResponse andAddToDeadLetterQueue(final Iterable> deadLetterQueueRecords) { + if (deadLetterQueueRecords == null) { + return this; + } + for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { + this.deadLetterQueueRecords.add(deadLetterQueueRecord); + } + return this; + } + } enum SerializationExceptionOrigin { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index 1dddc55ca3c26..1760e4a5b2b81 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -16,7 +16,9 @@ */ package org.apache.kafka.streams.processor.internals; +import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.streams.errors.ErrorHandlerContext; import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; @@ -242,6 +244,24 @@ public void process(final Record record) { ); } + if (!response.deadLetterQueueRecords.isEmpty()) { + final RecordCollector collector = ((RecordCollector.Supplier) internalProcessorContext).recordCollector(); + for (final ProducerRecord deadLetterQueueRecord : response.deadLetterQueueRecords) { + collector.send( + deadLetterQueueRecord.topic(), + deadLetterQueueRecord.key(), + deadLetterQueueRecord.value(), + deadLetterQueueRecord.headers(), + null, + deadLetterQueueRecord.timestamp(), + new ByteArraySerializer(), + new ByteArraySerializer(), + name(), + internalProcessorContext); + } + + } + 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" + diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java index a48a671d46091..b43e2da87f71c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java @@ -35,7 +35,7 @@ void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context); + final InternalProcessorContext context); void send(final String topic, final K key, @@ -45,7 +45,7 @@ void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context, + final InternalProcessorContext context, final StreamPartitioner partitioner); /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index 89cbf4d4c7d4e..6508064ddbd9e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -139,7 +139,7 @@ public void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context, + final InternalProcessorContext context, final StreamPartitioner partitioner) { if (partitioner != null) { @@ -197,7 +197,7 @@ public void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context) { + final InternalProcessorContext context) { checkForException(); final byte[] keyBytes; @@ -329,7 +329,7 @@ private void handleException(final ProductionExceptionHandler.Serializati final Integer partition, final Long timestamp, final String processorNodeId, - final InternalProcessorContext context, + final InternalProcessorContext context, final Exception serializationException) { log.debug(String.format("Error serializing record for topic %s", topic), serializationException); @@ -385,7 +385,7 @@ private void handleException(final ProductionExceptionHandler.Serializati droppedRecordsSensor.record(); } - private DefaultErrorHandlerContext errorHandlerContext(final InternalProcessorContext context, + private DefaultErrorHandlerContext errorHandlerContext(final InternalProcessorContext context, final String processorNodeId) { final RecordContext recordContext = context != null ? context.recordContext() : null; @@ -442,7 +442,7 @@ private StreamsException createStreamsExceptionForClassCastException(final private void recordSendError(final String topic, final Exception productionException, final ProducerRecord serializedRecord, - final InternalProcessorContext context, + final InternalProcessorContext context, final String processorNodeId) { String errorMessage = String.format(SEND_EXCEPTION_MESSAGE, topic, taskId, productionException.toString()); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 153ca2e02f1ee..82d4d05d9a950 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -17,8 +17,10 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.DeserializationExceptionHandler.DeserializationHandlerResponse; @@ -118,6 +120,23 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa throw new StreamsException("Fatal user code error in deserialization error callback", fatalUserException); } + if (!response.deadLetterQueueRecords.isEmpty()) { + final RecordCollector collector = ((RecordCollector.Supplier) processorContext).recordCollector(); + for (final ProducerRecord deadLetterQueueRecord : response.deadLetterQueueRecords) { + collector.send( + deadLetterQueueRecord.topic(), + deadLetterQueueRecord.key(), + deadLetterQueueRecord.value(), + deadLetterQueueRecord.headers(), + null, + deadLetterQueueRecord.timestamp(), + new ByteArraySerializer(), + new ByteArraySerializer(), + sourceNodeName, + (InternalProcessorContext) processorContext); + } + } + if (response == DeserializationHandlerResponse.FAIL) { throw new StreamsException("Deserialization exception handler is set to fail upon" + " a deserialization error. If you would rather have the streaming pipeline" + diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 82e9c8d7fb110..fc31f7f9d9d74 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -20,11 +20,13 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.TopologyConfig.TaskConfig; @@ -963,6 +965,23 @@ record = null; throw new FailedProcessingException("Fatal user code error in processing error callback", node.name(), fatalUserException); } + if (!response.deadLetterQueueRecords.isEmpty()) { + final RecordCollector collector = ((RecordCollector.Supplier) processorContext).recordCollector(); + for (final ProducerRecord deadLetterQueueRecord : response.deadLetterQueueRecords) { + collector.send( + deadLetterQueueRecord.topic(), + deadLetterQueueRecord.key(), + deadLetterQueueRecord.value(), + deadLetterQueueRecord.headers(), + null, + deadLetterQueueRecord.timestamp(), + new ByteArraySerializer(), + new ByteArraySerializer(), + node.name(), + processorContext); + } + } + 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" + diff --git a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java index 78c6dedcbf45c..c1959b7de7f71 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java @@ -234,7 +234,7 @@ public void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context) { + final InternalProcessorContext context) { // for byte arrays we need to wrap it for comparison final byte[] keyBytes = keySerializer.serialize(topic, headers, key); @@ -255,7 +255,7 @@ public void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context, + final InternalProcessorContext context, final StreamPartitioner partitioner) { throw new UnsupportedOperationException(); } diff --git a/streams/src/test/java/org/apache/kafka/test/MockRecordCollector.java b/streams/src/test/java/org/apache/kafka/test/MockRecordCollector.java index 8a7f543496356..737173e4cb2f3 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockRecordCollector.java +++ b/streams/src/test/java/org/apache/kafka/test/MockRecordCollector.java @@ -49,7 +49,7 @@ public void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context) { + final InternalProcessorContext context) { collected.add(new ProducerRecord<>( topic, partition, @@ -69,7 +69,7 @@ public void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context, + final InternalProcessorContext context, final StreamPartitioner partitioner) { collected.add(new ProducerRecord<>( topic, From dc4069e4e9983a93f4fba6848a4add64bd1e3cfc Mon Sep 17 00:00:00 2001 From: Damien Gasparina Date: Mon, 25 Nov 2024 15:33:42 +0100 Subject: [PATCH 02/16] KAFKA-16505 Adding test for DLQ support # Conflicts: # streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java --- .../errors/CommonExceptionHandler.java | 32 ++++- .../DefaultProductionExceptionHandler.java | 8 ++ .../DeserializationExceptionHandler.java | 2 + .../errors/ProcessingExceptionHandler.java | 3 + .../errors/ProductionExceptionHandler.java | 3 + .../processor/internals/ProcessorNode.java | 12 +- .../processor/internals/RecordCollector.java | 11 +- .../internals/RecordCollectorImpl.java | 47 ++++++-- .../internals/RecordDeserializer.java | 11 +- .../processor/internals/StreamTask.java | 10 +- .../internals/ProcessorNodeTest.java | 52 ++++++++ .../internals/RecordCollectorTest.java | 61 +++++++++- .../internals/RecordDeserializerTest.java | 84 +++++++++++++ .../errors/CommonExceptionHandlerTest.java | 114 ++++++++++++++++++ .../kafka/test/MockRecordCollector.java | 17 +++ 15 files changed, 425 insertions(+), 42 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/errors/CommonExceptionHandlerTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/CommonExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/CommonExceptionHandler.java index d3e7e14d882c7..ee22ce62b1a29 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/CommonExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/CommonExceptionHandler.java @@ -28,6 +28,9 @@ import java.util.Collections; import java.util.Map; +/** + * {@code CommonExceptionHandler} Contains utilities method that could be used by all exception handlers + */ public class CommonExceptionHandler implements Configurable { protected String deadLetterQueueTopicName = null; public static final String HEADER_ERRORS_EXCEPTION_NAME = "__streams.errors.exception"; @@ -37,17 +40,19 @@ public class CommonExceptionHandler implements Configurable { public static final String HEADER_ERRORS_PARTITION_NAME = "__streams.errors.partition"; public static final String HEADER_ERRORS_OFFSET_NAME = "__streams.errors.offset"; - @Override - public void configure(final Map configs) { - if (configs.containsKey(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)) { - setDeadLetterQueueTopicName(String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG))); - } - } public boolean shouldBuildDeadLetterQueueRecord() { return this.deadLetterQueueTopicName != null; } + /** + * If required, return Dead Letter Queue records for the provided exception + * @param key Serialized key for the records + * @param value Serialized value for the records + * @param context ErrorHandlerContext of the exception + * @param exception Thrown exception + * @return A list of Dead Letter Queue records to produce + */ public Iterable> maybeBuildDeadLetterQueueRecords(final byte[] key, final byte[] value, final ErrorHandlerContext context, @@ -59,6 +64,14 @@ public Iterable> maybeBuildDeadLetterQueueRecords return Collections.singleton(buildDeadLetterQueueRecord(key, value, context, exception)); } + + /** + * Build Dead Letter Queue records for the provided exception + * @param key Serialized key for the records + * @param value Serialized value for the records + * @param context ErrorHandlerContext of the exception + * @return A list of Dead Letter Queue records to produce + */ public ProducerRecord buildDeadLetterQueueRecord(final byte[] key, final byte[] value, final ErrorHandlerContext context, @@ -90,4 +103,11 @@ public String getDeadLetterQueueTopicName() { public void setDeadLetterQueueTopicName(final String deadLetterQueueTopicName) { this.deadLetterQueueTopicName = deadLetterQueueTopicName; } + + @Override + public void configure(final Map configs) { + if (configs.containsKey(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)) { + setDeadLetterQueueTopicName(String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG))); + } + } } \ No newline at end of file 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 9d1a2636b2a9d..a9e16a7616aba 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 @@ -48,6 +48,14 @@ public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext conte ProductionExceptionHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(null, null, context, exception)); } + @Override + public ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception, + final SerializationExceptionOrigin origin) { + return ProductionExceptionHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(null, null, context, exception)); + } + @Override public void configure(final Map configs) { super.configure(configs); 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 21f8f1c0f0d08..f516c9e28a010 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 @@ -92,6 +92,8 @@ enum DeserializationHandlerResponse { * The permanent and immutable id for the used option. This can't change ever. */ public final int id; + + /** a list of Kafka records to publish, e.g. in a Dead Letter Queue topic */ public final List> deadLetterQueueRecords; DeserializationHandlerResponse(final int id, final String name) { 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 index 3732b8a2f57d7..4d2dc0c8386f2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java @@ -57,6 +57,9 @@ enum ProcessingHandlerResponse { */ public final int id; + /** + * a list of Kafka records to publish, e.g. in a Dead Letter Queue topic + */ public final List> deadLetterQueueRecords; ProcessingHandlerResponse(final int id, final String name) { 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 0a2ed69e0c423..317c43fed5144 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 @@ -143,6 +143,9 @@ enum ProductionExceptionHandlerResponse { */ public final int id; + /** + * a list of Kafka records to publish, e.g. in a Dead Letter Queue topic + */ public final List> deadLetterQueueRecords; ProductionExceptionHandlerResponse(final int id, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index 1760e4a5b2b81..d560227839298 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.streams.errors.ErrorHandlerContext; import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; @@ -248,18 +247,13 @@ public void process(final Record record) { final RecordCollector collector = ((RecordCollector.Supplier) internalProcessorContext).recordCollector(); for (final ProducerRecord deadLetterQueueRecord : response.deadLetterQueueRecords) { collector.send( - deadLetterQueueRecord.topic(), deadLetterQueueRecord.key(), deadLetterQueueRecord.value(), - deadLetterQueueRecord.headers(), - null, - deadLetterQueueRecord.timestamp(), - new ByteArraySerializer(), - new ByteArraySerializer(), name(), - internalProcessorContext); + internalProcessorContext, + deadLetterQueueRecord + ); } - } if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java index b43e2da87f71c..2c14200f41324 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.serialization.Serializer; @@ -35,7 +36,7 @@ void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context); + final InternalProcessorContext context); void send(final String topic, final K key, @@ -45,9 +46,15 @@ void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context, + final InternalProcessorContext context, final StreamPartitioner partitioner); + void send(K key, + V value, + String processorNodeId, + InternalProcessorContext context, + ProducerRecord serializedRecord); + /** * Initialize the internal {@link Producer}; note this function should be made idempotent * diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index 6508064ddbd9e..dcc6bf336b732 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -139,7 +139,7 @@ public void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context, + final InternalProcessorContext context, final StreamPartitioner partitioner) { if (partitioner != null) { @@ -197,7 +197,7 @@ public void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context) { + final InternalProcessorContext context) { checkForException(); final byte[] keyBytes; @@ -263,6 +263,15 @@ public void send(final String topic, // freeing raw records in the context to reduce memory pressure freeRawInputRecordFromContext(context); + send(key, value, processorNodeId, context, serializedRecord); + } + + public void send(final K key, + final V value, + final String processorNodeId, + final InternalProcessorContext context, + final ProducerRecord serializedRecord) { + streamsProducer.send(serializedRecord, (metadata, exception) -> { try { // if there's already an exception record, skip logging offsets or new exceptions @@ -278,16 +287,16 @@ public void send(final String topic, log.warn("Received offset={} in produce response for {}", metadata.offset(), tp); } - if (!topic.endsWith("-changelog")) { + if (!serializedRecord.topic().endsWith("-changelog")) { // we may not have created a sensor during initialization if the node uses dynamic topic routing, // as all topics are not known up front, so create the sensor for this topic if absent final Sensor topicProducedSensor = producedSensorByTopic.computeIfAbsent( - topic, + serializedRecord.topic(), t -> TopicMetrics.producedSensor( Thread.currentThread().getName(), taskId.toString(), processorNodeId, - topic, + serializedRecord.topic(), context.metrics() ) ); @@ -299,7 +308,7 @@ public void send(final String topic, } } else { recordSendError( - topic, + serializedRecord.topic(), exception, serializedRecord, context, @@ -307,7 +316,7 @@ public void send(final String topic, ); // KAFKA-7510 only put message key and value in TRACE level log so we don't leak data by default - log.trace("Failed record: (key {} value {} timestamp {}) topic=[{}] partition=[{}]", key, value, timestamp, topic, partition); + log.trace("Failed record: (key {} value {} timestamp {}) topic=[{}] partition=[{}]", key, value, serializedRecord.timestamp(), serializedRecord.topic(), serializedRecord.partition()); } } catch (final RuntimeException fatal) { sendException.set(new StreamsException("Producer.send `Callback` failed", fatal)); @@ -365,6 +374,18 @@ private void handleException(final ProductionExceptionHandler.Serializati ); } + if (!response.deadLetterQueueRecords.isEmpty()) { + for (final ProducerRecord deadLetterQueueRecord : response.deadLetterQueueRecords) { + this.send( + deadLetterQueueRecord.key(), + deadLetterQueueRecord.value(), + processorNodeId, + context, + deadLetterQueueRecord + ); + } + } + if (maybeFailResponse(response) == ProductionExceptionHandlerResponse.FAIL) { throw new StreamsException( String.format( @@ -490,6 +511,18 @@ private void recordSendError(final String topic, return; } + if (!response.deadLetterQueueRecords.isEmpty()) { + for (final ProducerRecord deadLetterQueueRecord : response.deadLetterQueueRecords) { + this.send( + deadLetterQueueRecord.key(), + deadLetterQueueRecord.value(), + processorNodeId, + context, + deadLetterQueueRecord + ); + } + } + if (productionException instanceof RetriableException && response == ProductionExceptionHandlerResponse.RETRY) { errorMessage += "\nThe broker is either slow or in bad state (like not having enough replicas) in responding the request, " + "or the connection to broker was interrupted sending the request or receiving the response. " + diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 82d4d05d9a950..0a2ccd6bf69d3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -20,7 +20,6 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.record.TimestampType; -import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.DeserializationExceptionHandler.DeserializationHandlerResponse; @@ -124,16 +123,12 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa final RecordCollector collector = ((RecordCollector.Supplier) processorContext).recordCollector(); for (final ProducerRecord deadLetterQueueRecord : response.deadLetterQueueRecords) { collector.send( - deadLetterQueueRecord.topic(), deadLetterQueueRecord.key(), deadLetterQueueRecord.value(), - deadLetterQueueRecord.headers(), - null, - deadLetterQueueRecord.timestamp(), - new ByteArraySerializer(), - new ByteArraySerializer(), sourceNodeName, - (InternalProcessorContext) processorContext); + (InternalProcessorContext) processorContext, + deadLetterQueueRecord + ); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index fc31f7f9d9d74..80cfbf72d6127 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.TopologyConfig.TaskConfig; @@ -969,16 +968,11 @@ record = null; final RecordCollector collector = ((RecordCollector.Supplier) processorContext).recordCollector(); for (final ProducerRecord deadLetterQueueRecord : response.deadLetterQueueRecords) { collector.send( - deadLetterQueueRecord.topic(), deadLetterQueueRecord.key(), deadLetterQueueRecord.value(), - deadLetterQueueRecord.headers(), - null, - deadLetterQueueRecord.timestamp(), - new ByteArraySerializer(), - new ByteArraySerializer(), node.name(), - processorContext); + processorContext, + deadLetterQueueRecord); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index e9669ac39f442..76dda6ebbcd7c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -29,6 +29,8 @@ import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.errors.ErrorHandlerContext; +import org.apache.kafka.streams.errors.LogAndContinueProcessingExceptionHandler; +import org.apache.kafka.streams.errors.LogAndFailProcessingExceptionHandler; import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; @@ -39,7 +41,9 @@ import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.test.InternalMockProcessorContext; +import org.apache.kafka.test.MockRecordCollector; import org.apache.kafka.test.StreamsTestUtils; import org.junit.jupiter.api.Test; @@ -166,6 +170,54 @@ public void shouldThrowFailedProcessingExceptionWhenProcessingExceptionHandlerTh assertEquals(NAME, failedProcessingException.failedProcessorNodeName()); } + + @Test + public void shouldBuildDeadLetterQueueRecordsInDefaultProcessingExceptionHandler() { + final ProcessorNode node = new ProcessorNode<>("processor", + (Processor) record -> { + throw new NullPointerException("Oopsie!"); + }, Collections.emptySet()); + + final MockRecordCollector collector = new MockRecordCollector(); + final InternalProcessorContext internalProcessorContext = + new InternalMockProcessorContext<>( + new StateSerdes<>("sink", Serdes.ByteArray(), Serdes.ByteArray()), + collector + ); + final ProcessingExceptionHandler processingExceptionHandler = new LogAndFailProcessingExceptionHandler(); + processingExceptionHandler.configure(Collections.singletonMap(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG, "dlq")); + node.init(internalProcessorContext, processingExceptionHandler); + + assertThrows(RuntimeException.class, + () -> node.process(new Record<>("hello", "world", 1L))); + + assertEquals(1, collector.collected().size()); + assertEquals("dlq", collector.collected().get(0).topic()); + } + + @Test + public void shouldBuildDeadLetterQueueRecordsInLogAndContinueProcessingExceptionHandler() { + final ProcessorNode node = new ProcessorNode<>("processor", + (Processor) record -> { + throw new NullPointerException("Oopsie!"); + }, Collections.emptySet()); + + final MockRecordCollector collector = new MockRecordCollector(); + final InternalProcessorContext internalProcessorContext = + new InternalMockProcessorContext<>( + new StateSerdes<>("sink", Serdes.ByteArray(), Serdes.ByteArray()), + collector + ); + final ProcessingExceptionHandler processingExceptionHandler = new LogAndContinueProcessingExceptionHandler(); + processingExceptionHandler.configure(Collections.singletonMap(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG, "dlq")); + node.init(internalProcessorContext, processingExceptionHandler); + + node.process(new Record<>("hello", "world", 0L)); + + assertEquals(1, collector.collected().size()); + assertEquals("dlq", collector.collected().get(0).topic()); + } + private static class ExceptionalProcessor implements Processor { @Override public void init(final ProcessorContext context) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index ebc92c651841c..dca6ad4e84ea1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -47,6 +47,7 @@ import org.apache.kafka.common.utils.LogCaptureAppender; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; import org.apache.kafka.streams.errors.ErrorHandlerContext; import org.apache.kafka.streams.errors.ProductionExceptionHandler; @@ -1834,6 +1835,58 @@ public void shouldSwallowTransactionAbortedExceptionAndNotCallProductionExceptio collector.flush(); // need to call flush() to check for internal exceptions } + @Test + public void shouldBuildDeadLetterQueueRecordsInDefaultExceptionHandlerDuringDeserialization() { + try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { + final DefaultProductionExceptionHandler productionExceptionHandler = new DefaultProductionExceptionHandler(); + productionExceptionHandler.configure(Collections.singletonMap( + StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG, + "dlq" + )); + final RecordCollector collector = newRecordCollector(productionExceptionHandler); + collector.initialize(); + + assertThat(mockProducer.history().isEmpty(), equalTo(true)); + final StreamsException error = assertThrows( + StreamsException.class, + () -> + collector.send(topic, "hello", "world", null, 0, null, errorSerializer, stringSerializer, sinkNodeName, context) + ); + + assertEquals(1, mockProducer.history().size()); + assertEquals("dlq", mockProducer.history().get(0).topic()); + } + } + + + @Test + public void shouldBuildDeadLetterQueueRecordsInDefaultExceptionHandler() { + final KafkaException exception = new KafkaException("KABOOM!"); + final StreamsProducer streamProducer = getExceptionalStreamsProducerOnSend(exception); + final MockProducer mockProducer = (MockProducer) streamProducer.kafkaProducer(); + final DefaultProductionExceptionHandler productionExceptionHandler = new DefaultProductionExceptionHandler(); + productionExceptionHandler.configure(Collections.singletonMap( + StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG, + "dlq" + )); + final RecordCollector collector = new RecordCollectorImpl( + logContext, + taskId, + streamProducer, + productionExceptionHandler, + streamsMetrics, + topology + ); + + collector.initialize(); + + collector.send(topic, "hello", "world", null, 0, null, stringSerializer, stringSerializer, sinkNodeName, context); + assertThrows(StreamsException.class, collector::flush); + + assertEquals(1, mockProducer.history().size()); + assertEquals("dlq", mockProducer.history().get(0).topic()); + } + @Test public void shouldNotSendIfSendOfOtherTaskFailedInCallback() { final TaskId taskId1 = new TaskId(0, 0); @@ -1978,8 +2031,12 @@ private StreamsProducer getExceptionalStreamsProducerOnSend(final Exception exce new MockProducer<>(cluster, true, null, byteArraySerializer, byteArraySerializer) { @Override public synchronized Future send(final ProducerRecord record, final Callback callback) { - callback.onCompletion(null, exception); - return null; + if (record.topic().equals("dlq")) { + return super.send(record, callback); + } else { + callback.onCompletion(null, exception); + return null; + } } }, AT_LEAST_ONCE, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java index aa3bb57c7a66c..9847fedb25149 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java @@ -23,18 +23,26 @@ import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.DeserializationExceptionHandler.DeserializationHandlerResponse; import org.apache.kafka.streams.errors.ErrorHandlerContext; +import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler; +import org.apache.kafka.streams.errors.LogAndFailExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.test.InternalMockProcessorContext; +import org.apache.kafka.test.MockRecordCollector; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; +import java.nio.charset.StandardCharsets; +import java.util.Collections; import java.util.Map; import java.util.Optional; @@ -222,6 +230,82 @@ public void shouldFailWhenDeserializationFailsAndExceptionHandlerThrows() { } } + + @Test + public void shouldBuildDeadLetterQueueRecordsInDefaultDeserializationException() { + try (Metrics metrics = new Metrics()) { + final MockRecordCollector collector = new MockRecordCollector(); + final InternalProcessorContext internalProcessorContext = + new InternalMockProcessorContext<>( + new StateSerdes<>("sink", Serdes.ByteArray(), Serdes.ByteArray()), + collector + ); + final DeserializationExceptionHandler deserializationExceptionHandler = new LogAndFailExceptionHandler(); + deserializationExceptionHandler.configure(Collections.singletonMap(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG, "dlq")); + + assertThrows(StreamsException.class, () -> RecordDeserializer.handleDeserializationFailure( + deserializationExceptionHandler, + internalProcessorContext, + new RuntimeException(new NullPointerException("Oopsie")), + new ConsumerRecord<>("source", + 0, + 0, + 123, + TimestampType.CREATE_TIME, + -1, + -1, + "hello".getBytes(StandardCharsets.UTF_8), + "world".getBytes(StandardCharsets.UTF_8), + new RecordHeaders(), + Optional.empty()), + new LogContext().logger(this.getClass()), + metrics.sensor("dropped-records"), + "sourceNode" + )); + + assertEquals(1, collector.collected().size()); + assertEquals("dlq", collector.collected().get(0).topic()); + } + } + + + @Test + public void shouldBuildDeadLetterQueueRecordsInLogAndContinueDeserializationException() { + try (Metrics metrics = new Metrics()) { + final MockRecordCollector collector = new MockRecordCollector(); + final InternalProcessorContext internalProcessorContext = + new InternalMockProcessorContext<>( + new StateSerdes<>("sink", Serdes.ByteArray(), Serdes.ByteArray()), + collector + ); + final DeserializationExceptionHandler deserializationExceptionHandler = new LogAndContinueExceptionHandler(); + deserializationExceptionHandler.configure(Collections.singletonMap(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG, "dlq")); + + RecordDeserializer.handleDeserializationFailure( + deserializationExceptionHandler, + internalProcessorContext, + new RuntimeException(new NullPointerException("Oopsie")), + new ConsumerRecord<>("source", + 0, + 0, + 123, + TimestampType.CREATE_TIME, + -1, + -1, + "hello".getBytes(StandardCharsets.UTF_8), + "world".getBytes(StandardCharsets.UTF_8), + new RecordHeaders(), + Optional.empty()), + new LogContext().logger(this.getClass()), + metrics.sensor("dropped-records"), + "sourceNode" + ); + + assertEquals(1, collector.collected().size()); + assertEquals("dlq", collector.collected().get(0).topic()); + } + } + static class TheSourceNode extends SourceNode { private final boolean keyThrowsException; private final boolean valueThrowsException; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/errors/CommonExceptionHandlerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/errors/CommonExceptionHandlerTest.java new file mode 100644 index 0000000000000..d0d09f5126bf4 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/errors/CommonExceptionHandlerTest.java @@ -0,0 +1,114 @@ +/* + * 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.errors; + +import org.apache.kafka.clients.producer.ProducerRecord; +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.serialization.Serdes; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.CommonExceptionHandler; +import org.apache.kafka.streams.errors.ErrorHandlerContext; +import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.test.InternalMockProcessorContext; +import org.apache.kafka.test.MockRecordCollector; +import org.apache.kafka.test.StreamsTestUtils; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.util.Collections; +import java.util.Iterator; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + + +@ExtendWith(MockitoExtension.class) +public class CommonExceptionHandlerTest { + @Test + public void checkDealLetterQueueRecords() { + final StringSerializer stringSerializer = new StringSerializer(); + final StringDeserializer stringDeserializer = new StringDeserializer(); + final MockRecordCollector collector = new MockRecordCollector(); + final InternalProcessorContext internalProcessorContext = new InternalMockProcessorContext<>( + new StateSerdes<>("sink", Serdes.ByteArray(), Serdes.ByteArray()), + collector + ); + internalProcessorContext.setRecordContext(new ProcessorRecordContext( + 1L, + 2, + 3, + "source", + new RecordHeaders(Collections.singletonList( + new RecordHeader("sourceHeader", stringSerializer.serialize(null, "hello world")))) + )); + final ErrorHandlerContext errorHandlerContext = getErrorHandlerContext(internalProcessorContext); + + final CommonExceptionHandler commonExceptionHandler = new CommonExceptionHandler(); + commonExceptionHandler.setDeadLetterQueueTopicName("dlq"); + final NullPointerException exception = new NullPointerException("Oopsie!"); + final Iterable> dlqRecords = commonExceptionHandler.maybeBuildDeadLetterQueueRecords(null, null, errorHandlerContext, exception); + final Iterator> iterator = dlqRecords.iterator(); + + assertTrue(iterator.hasNext()); + final ProducerRecord dlqRecord = iterator.next(); + final Headers headers = dlqRecord.headers(); + assertFalse(iterator.hasNext()); // There should be only one record + + assertEquals("dlq", dlqRecord.topic()); + assertEquals(errorHandlerContext.timestamp(), dlqRecord.timestamp()); + assertEquals(1, dlqRecord.timestamp()); + assertEquals(exception.toString(), stringDeserializer.deserialize(null, headers.lastHeader(CommonExceptionHandler.HEADER_ERRORS_EXCEPTION_NAME).value())); + assertEquals(exception.getMessage(), stringDeserializer.deserialize(null, headers.lastHeader(CommonExceptionHandler.HEADER_ERRORS_EXCEPTION_MESSAGE_NAME).value())); + assertEquals("source", stringDeserializer.deserialize(null, headers.lastHeader(CommonExceptionHandler.HEADER_ERRORS_TOPIC_NAME).value())); + assertEquals("3", stringDeserializer.deserialize(null, headers.lastHeader(CommonExceptionHandler.HEADER_ERRORS_PARTITION_NAME).value())); + assertEquals("2", stringDeserializer.deserialize(null, headers.lastHeader(CommonExceptionHandler.HEADER_ERRORS_OFFSET_NAME).value())); + } + + @Test + public void doNotBuildDeadLetterQueueRecordsIfNotConfigured() { + final CommonExceptionHandler commonExceptionHandler = new CommonExceptionHandler(); + final StreamsConfig streamsConfig = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); + commonExceptionHandler.configure(streamsConfig.originals()); + final NullPointerException exception = new NullPointerException("Oopsie!"); + final Iterable> dlqRecords = commonExceptionHandler.maybeBuildDeadLetterQueueRecords(null, null, null, exception); + final Iterator> iterator = dlqRecords.iterator(); + + assertFalse(iterator.hasNext()); + } + + private static DefaultErrorHandlerContext getErrorHandlerContext(final InternalProcessorContext internalProcessorContext) { + return new DefaultErrorHandlerContext( + null, + internalProcessorContext.topic(), + internalProcessorContext.partition(), + internalProcessorContext.offset(), + internalProcessorContext.headers(), + internalProcessorContext.currentNode().name(), + internalProcessorContext.taskId(), + internalProcessorContext.timestamp()); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/test/MockRecordCollector.java b/streams/src/test/java/org/apache/kafka/test/MockRecordCollector.java index 737173e4cb2f3..643d5d88d32c8 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockRecordCollector.java +++ b/streams/src/test/java/org/apache/kafka/test/MockRecordCollector.java @@ -81,6 +81,23 @@ public void send(final String topic, ); } + @Override + public void send(final K key, + final V value, + final String processorNodeId, + final InternalProcessorContext context, + final ProducerRecord serializedRecord) { + // Building a new ProducerRecord for key & value type conversion + collected.add(new ProducerRecord<>( + serializedRecord.topic(), + serializedRecord.partition(), + serializedRecord.timestamp(), + key, + value, + serializedRecord.headers()) + ); + } + @Override public void initialize() {} From 6a14db059a048489c85fb682e49f0aead7534c3c Mon Sep 17 00:00:00 2001 From: Damien Gasparina Date: Mon, 25 Nov 2024 16:56:29 +0100 Subject: [PATCH 03/16] KAFKA-16505 Adding test for DLQ support --- .../DeserializationExceptionHandler.java | 20 ++++++++++++++++--- .../errors/ProcessingExceptionHandler.java | 20 ++++++++++++++++--- .../errors/ProductionExceptionHandler.java | 19 +++++++++++++++--- .../processor/internals/ProcessorNode.java | 5 +++-- .../internals/RecordCollectorImpl.java | 10 ++++++---- .../internals/RecordDeserializer.java | 6 ++++-- .../processor/internals/StreamTask.java | 5 +++-- 7 files changed, 66 insertions(+), 19 deletions(-) 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 f516c9e28a010..d1dc22b248500 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 @@ -24,6 +24,8 @@ import java.util.LinkedList; import java.util.List; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; /** * Interface that specifies how an exception from source node deserialization @@ -94,12 +96,12 @@ enum DeserializationHandlerResponse { public final int id; /** a list of Kafka records to publish, e.g. in a Dead Letter Queue topic */ - public final List> deadLetterQueueRecords; + private final Queue> deadLetterQueueRecordsQueue; DeserializationHandlerResponse(final int id, final String name) { this.id = id; this.name = name; - this.deadLetterQueueRecords = new LinkedList<>(); + this.deadLetterQueueRecordsQueue = new ConcurrentLinkedQueue<>(); } public DeserializationHandlerResponse andAddToDeadLetterQueue(final Iterable> deadLetterQueueRecords) { @@ -107,9 +109,21 @@ public DeserializationHandlerResponse andAddToDeadLetterQueue(final Iterable deadLetterQueueRecord : deadLetterQueueRecords) { - this.deadLetterQueueRecords.add(deadLetterQueueRecord); + this.deadLetterQueueRecordsQueue.add(deadLetterQueueRecord); } return this; } + + public List> drainDeadLetterQueueRecords() { + final LinkedList> deadLetterQueueRecords = new LinkedList<>(); + while (true) { + final ProducerRecord record = this.deadLetterQueueRecordsQueue.poll(); + if (record == null) { + break; + } + deadLetterQueueRecords.add(record); + } + return deadLetterQueueRecords; + } } } 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 index 4d2dc0c8386f2..4b730d32b1f23 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java @@ -22,6 +22,8 @@ import java.util.LinkedList; import java.util.List; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; /** * An interface that allows user code to inspect a record that has failed processing @@ -60,12 +62,12 @@ enum ProcessingHandlerResponse { /** * a list of Kafka records to publish, e.g. in a Dead Letter Queue topic */ - public final List> deadLetterQueueRecords; + private final Queue> deadLetterQueueRecordsQueue; ProcessingHandlerResponse(final int id, final String name) { this.id = id; this.name = name; - deadLetterQueueRecords = new LinkedList<>(); + deadLetterQueueRecordsQueue = new ConcurrentLinkedQueue<>(); } @@ -74,9 +76,21 @@ public ProcessingExceptionHandler.ProcessingHandlerResponse andAddToDeadLetterQu return this; } for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { - this.deadLetterQueueRecords.add(deadLetterQueueRecord); + this.deadLetterQueueRecordsQueue.add(deadLetterQueueRecord); } return this; } + + public List> drainDeadLetterQueueRecords() { + final LinkedList> deadLetterQueueRecords = new LinkedList<>(); + while (true) { + final ProducerRecord record = this.deadLetterQueueRecordsQueue.poll(); + if (record == null) { + break; + } + deadLetterQueueRecords.add(record); + } + return deadLetterQueueRecords; + } } } 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 317c43fed5144..71958d64f017d 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 @@ -21,6 +21,8 @@ import java.util.LinkedList; import java.util.List; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; /** * Interface that specifies how an exception when attempting to produce a result to @@ -146,13 +148,13 @@ enum ProductionExceptionHandlerResponse { /** * a list of Kafka records to publish, e.g. in a Dead Letter Queue topic */ - public final List> deadLetterQueueRecords; + private final Queue> deadLetterQueueRecordsQueue; ProductionExceptionHandlerResponse(final int id, final String name) { this.id = id; this.name = name; - deadLetterQueueRecords = new LinkedList<>(); + this.deadLetterQueueRecordsQueue = new ConcurrentLinkedQueue<>(); } public ProductionExceptionHandler.ProductionExceptionHandlerResponse andAddToDeadLetterQueue(final Iterable> deadLetterQueueRecords) { @@ -160,11 +162,22 @@ public ProductionExceptionHandler.ProductionExceptionHandlerResponse andAddToDea return this; } for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { - this.deadLetterQueueRecords.add(deadLetterQueueRecord); + this.deadLetterQueueRecordsQueue.add(deadLetterQueueRecord); } return this; } + public List> drainDeadLetterQueueRecords() { + final LinkedList> deadLetterQueueRecords = new LinkedList<>(); + while (true) { + final ProducerRecord record = this.deadLetterQueueRecordsQueue.poll(); + if (record == null) { + break; + } + deadLetterQueueRecords.add(record); + } + return deadLetterQueueRecords; + } } enum SerializationExceptionOrigin { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index d560227839298..4af05331cf312 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -243,9 +243,10 @@ public void process(final Record record) { ); } - if (!response.deadLetterQueueRecords.isEmpty()) { + final List> deadLetterQueueRecords = response.drainDeadLetterQueueRecords(); + if (!deadLetterQueueRecords.isEmpty()) { final RecordCollector collector = ((RecordCollector.Supplier) internalProcessorContext).recordCollector(); - for (final ProducerRecord deadLetterQueueRecord : response.deadLetterQueueRecords) { + for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { collector.send( deadLetterQueueRecord.key(), deadLetterQueueRecord.value(), diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index dcc6bf336b732..4f0294f3ce0da 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -374,8 +374,9 @@ private void handleException(final ProductionExceptionHandler.Serializati ); } - if (!response.deadLetterQueueRecords.isEmpty()) { - for (final ProducerRecord deadLetterQueueRecord : response.deadLetterQueueRecords) { + final List> deadLetterQueueRecords = response.drainDeadLetterQueueRecords(); + if (!deadLetterQueueRecords.isEmpty()) { + for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { this.send( deadLetterQueueRecord.key(), deadLetterQueueRecord.value(), @@ -511,8 +512,9 @@ private void recordSendError(final String topic, return; } - if (!response.deadLetterQueueRecords.isEmpty()) { - for (final ProducerRecord deadLetterQueueRecord : response.deadLetterQueueRecords) { + final List> deadLetterQueueRecords = response.drainDeadLetterQueueRecords(); + if (!deadLetterQueueRecords.isEmpty()) { + for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { this.send( deadLetterQueueRecord.key(), deadLetterQueueRecord.value(), diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 0a2ccd6bf69d3..418ae3911fece 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -29,6 +29,7 @@ import org.slf4j.Logger; +import java.util.List; import java.util.Objects; import static org.apache.kafka.streams.StreamsConfig.DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG; @@ -119,9 +120,10 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa throw new StreamsException("Fatal user code error in deserialization error callback", fatalUserException); } - if (!response.deadLetterQueueRecords.isEmpty()) { + final List> deadLetterQueueRecords = response.drainDeadLetterQueueRecords(); + if (!deadLetterQueueRecords.isEmpty()) { final RecordCollector collector = ((RecordCollector.Supplier) processorContext).recordCollector(); - for (final ProducerRecord deadLetterQueueRecord : response.deadLetterQueueRecords) { + for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { collector.send( deadLetterQueueRecord.key(), deadLetterQueueRecord.value(), diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 80cfbf72d6127..b1d3b98bdc39b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -964,9 +964,10 @@ record = null; throw new FailedProcessingException("Fatal user code error in processing error callback", node.name(), fatalUserException); } - if (!response.deadLetterQueueRecords.isEmpty()) { + final List> deadLetterQueueRecords = response.drainDeadLetterQueueRecords(); + if (!deadLetterQueueRecords.isEmpty()) { final RecordCollector collector = ((RecordCollector.Supplier) processorContext).recordCollector(); - for (final ProducerRecord deadLetterQueueRecord : response.deadLetterQueueRecords) { + for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { collector.send( deadLetterQueueRecord.key(), deadLetterQueueRecord.value(), From 3cee49cc780dea027ab9d52634a94c632aeeb455 Mon Sep 17 00:00:00 2001 From: Damien Gasparina Date: Wed, 27 Nov 2024 17:08:17 +0100 Subject: [PATCH 04/16] KAFKA-16505: add source raw bytes in processorContex --- .../DefaultProductionExceptionHandler.java | 12 +++-- .../DeserializationExceptionHandler.java | 2 +- ...andler.java => ExceptionHandlerUtils.java} | 52 +++++++------------ .../LogAndContinueExceptionHandler.java | 10 ++-- ...AndContinueProcessingExceptionHandler.java | 10 ++-- .../errors/LogAndFailExceptionHandler.java | 10 ++-- .../LogAndFailProcessingExceptionHandler.java | 10 ++-- .../errors/ProcessingExceptionHandler.java | 2 +- .../errors/ProductionExceptionHandler.java | 2 +- .../processor/internals/ProcessorNode.java | 2 +- .../internals/RecordCollectorImpl.java | 4 +- .../internals/RecordDeserializer.java | 2 +- .../processor/internals/StreamTask.java | 2 +- .../kafka/streams/StreamsConfigTest.java | 5 ++ .../ExceptionHandlerUtilsTest.java} | 27 ++++------ 15 files changed, 77 insertions(+), 75 deletions(-) rename streams/src/main/java/org/apache/kafka/streams/errors/{CommonExceptionHandler.java => ExceptionHandlerUtils.java} (70%) rename streams/src/test/java/org/apache/kafka/streams/{processor/internals/errors/CommonExceptionHandlerTest.java => errors/ExceptionHandlerUtilsTest.java} (78%) 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 a9e16a7616aba..ef6e3fcc89163 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,14 +18,20 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.errors.RetriableException; +import org.apache.kafka.streams.StreamsConfig; import java.util.Map; +import static org.apache.kafka.streams.errors.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; + /** * {@code ProductionExceptionHandler} that always instructs streams to fail when an exception * happens while attempting to produce result records. */ public class DefaultProductionExceptionHandler implements ProductionExceptionHandler { + + private String deadLetterQueueTopic; + /** * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ProducerRecord, Exception)} instead. */ @@ -45,7 +51,7 @@ public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext conte final Exception exception) { return exception instanceof RetriableException ? ProductionExceptionHandlerResponse.RETRY : - ProductionExceptionHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(null, null, context, exception)); + ProductionExceptionHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } @Override @@ -53,11 +59,11 @@ public ProductionExceptionHandlerResponse handleSerializationException(final Err final ProducerRecord record, final Exception exception, final SerializationExceptionOrigin origin) { - return ProductionExceptionHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(null, null, context, exception)); + return ProductionExceptionHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } @Override public void configure(final Map configs) { - super.configure(configs); + deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); } } \ No newline at end of file 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 d1dc22b248500..bb74d71114e4d 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 @@ -114,7 +114,7 @@ public DeserializationHandlerResponse andAddToDeadLetterQueue(final Iterable> drainDeadLetterQueueRecords() { + public List> deadLetterQueueRecords() { final LinkedList> deadLetterQueueRecords = new LinkedList<>(); while (true) { final ProducerRecord record = this.deadLetterQueueRecordsQueue.poll(); diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/CommonExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ExceptionHandlerUtils.java similarity index 70% rename from streams/src/main/java/org/apache/kafka/streams/errors/CommonExceptionHandler.java rename to streams/src/main/java/org/apache/kafka/streams/errors/ExceptionHandlerUtils.java index ee22ce62b1a29..eb0c72896ef75 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/CommonExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ExceptionHandlerUtils.java @@ -18,7 +18,6 @@ package org.apache.kafka.streams.errors; import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.Configurable; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.StreamsConfig; @@ -26,23 +25,21 @@ import java.io.PrintWriter; import java.io.StringWriter; import java.util.Collections; -import java.util.Map; /** * {@code CommonExceptionHandler} Contains utilities method that could be used by all exception handlers */ -public class CommonExceptionHandler implements Configurable { - protected String deadLetterQueueTopicName = null; - public static final String HEADER_ERRORS_EXCEPTION_NAME = "__streams.errors.exception"; - public static final String HEADER_ERRORS_STACKTRACE_NAME = "__streams.errors.stacktrace"; - public static final String HEADER_ERRORS_EXCEPTION_MESSAGE_NAME = "__streams.errors.message"; - public static final String HEADER_ERRORS_TOPIC_NAME = "__streams.errors.topic"; - public static final String HEADER_ERRORS_PARTITION_NAME = "__streams.errors.partition"; - public static final String HEADER_ERRORS_OFFSET_NAME = "__streams.errors.offset"; +class ExceptionHandlerUtils { + static final String HEADER_ERRORS_EXCEPTION_NAME = "__streams.errors.exception"; + static final String HEADER_ERRORS_STACKTRACE_NAME = "__streams.errors.stacktrace"; + static final String HEADER_ERRORS_EXCEPTION_MESSAGE_NAME = "__streams.errors.message"; + static final String HEADER_ERRORS_TOPIC_NAME = "__streams.errors.topic"; + static final String HEADER_ERRORS_PARTITION_NAME = "__streams.errors.partition"; + static final String HEADER_ERRORS_OFFSET_NAME = "__streams.errors.offset"; - public boolean shouldBuildDeadLetterQueueRecord() { - return this.deadLetterQueueTopicName != null; + static boolean shouldBuildDeadLetterQueueRecord(final String deadLetterQueueTopicName) { + return deadLetterQueueTopicName != null; } /** @@ -53,15 +50,16 @@ public boolean shouldBuildDeadLetterQueueRecord() { * @param exception Thrown exception * @return A list of Dead Letter Queue records to produce */ - public Iterable> maybeBuildDeadLetterQueueRecords(final byte[] key, + static Iterable> maybeBuildDeadLetterQueueRecords(final String deadLetterQueueTopicName, + final byte[] key, final byte[] value, final ErrorHandlerContext context, final Exception exception) { - if (!shouldBuildDeadLetterQueueRecord()) { + if (!shouldBuildDeadLetterQueueRecord(deadLetterQueueTopicName)) { return Collections.emptyList(); } - return Collections.singleton(buildDeadLetterQueueRecord(key, value, context, exception)); + return Collections.singleton(buildDeadLetterQueueRecord(deadLetterQueueTopicName, key, value, context, exception)); } @@ -72,10 +70,11 @@ public Iterable> maybeBuildDeadLetterQueueRecords * @param context ErrorHandlerContext of the exception * @return A list of Dead Letter Queue records to produce */ - public ProducerRecord buildDeadLetterQueueRecord(final byte[] key, - final byte[] value, - final ErrorHandlerContext context, - final Exception e) { + static ProducerRecord buildDeadLetterQueueRecord(final String deadLetterQueueTopicName, + final byte[] key, + final byte[] value, + final ErrorHandlerContext context, + final Exception e) { if (deadLetterQueueTopicName == null) { throw new InvalidConfigurationException(String.format("%s can not be null while building DeadLetterQueue record", StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); } @@ -95,19 +94,4 @@ public ProducerRecord buildDeadLetterQueueRecord(final byte[] ke return producerRecord; } - - public String getDeadLetterQueueTopicName() { - return deadLetterQueueTopicName; - } - - public void setDeadLetterQueueTopicName(final String deadLetterQueueTopicName) { - this.deadLetterQueueTopicName = deadLetterQueueTopicName; - } - - @Override - public void configure(final Map configs) { - if (configs.containsKey(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)) { - setDeadLetterQueueTopicName(String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG))); - } - } } \ No newline at end of file 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 4531caafa97a7..187d765353b57 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 @@ -17,6 +17,7 @@ package org.apache.kafka.streams.errors; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.ProcessorContext; import org.slf4j.Logger; @@ -24,12 +25,15 @@ import java.util.Map; +import static org.apache.kafka.streams.errors.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; + /** * Deserialization handler that logs a deserialization exception and then * signals the processing pipeline to continue processing more records. */ -public class LogAndContinueExceptionHandler extends CommonExceptionHandler implements DeserializationExceptionHandler { +public class LogAndContinueExceptionHandler implements DeserializationExceptionHandler { private static final Logger log = LoggerFactory.getLogger(LogAndContinueExceptionHandler.class); + private String deadLetterQueueTopic; /** * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)} instead. @@ -67,11 +71,11 @@ public DeserializationHandlerResponse handle(final ErrorHandlerContext context, exception ); - return DeserializationHandlerResponse.CONTINUE.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(null, null, context, exception)); + return DeserializationHandlerResponse.CONTINUE.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } @Override public void configure(final Map configs) { - super.configure(configs); + deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java index 2423fa9f073a6..709e423360d37 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.errors; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.api.Record; import org.slf4j.Logger; @@ -23,12 +24,15 @@ import java.util.Map; +import static org.apache.kafka.streams.errors.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; + /** * Processing exception handler that logs a processing exception and then * signals the processing pipeline to continue processing more records. */ -public class LogAndContinueProcessingExceptionHandler extends CommonExceptionHandler implements ProcessingExceptionHandler { +public class LogAndContinueProcessingExceptionHandler implements ProcessingExceptionHandler { private static final Logger log = LoggerFactory.getLogger(LogAndContinueProcessingExceptionHandler.class); + private String deadLetterQueueTopic; @Override public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { @@ -42,11 +46,11 @@ public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final exception ); - return ProcessingHandlerResponse.CONTINUE.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(null, null, context, exception)); + return ProcessingHandlerResponse.CONTINUE.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } @Override public void configure(final Map configs) { - super.configure(configs); + deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); } } 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 78ef2f6b8b314..8a242d39f1f39 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.StreamsConfig; import org.apache.kafka.streams.processor.ProcessorContext; import org.slf4j.Logger; @@ -24,12 +25,15 @@ import java.util.Map; +import static org.apache.kafka.streams.errors.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; + /** * Deserialization handler that logs a deserialization exception and then * signals the processing pipeline to stop processing more records and fail. */ -public class LogAndFailExceptionHandler extends CommonExceptionHandler implements DeserializationExceptionHandler { +public class LogAndFailExceptionHandler implements DeserializationExceptionHandler { private static final Logger log = LoggerFactory.getLogger(LogAndFailExceptionHandler.class); + private String deadLetterQueueTopic; /** * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)} instead. @@ -67,11 +71,11 @@ public DeserializationHandlerResponse handle(final ErrorHandlerContext context, exception ); - return DeserializationHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(null, null, context, exception)); + return DeserializationHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } @Override public void configure(final Map configs) { - super.configure(configs); + deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); } } \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java index 0f7bab6065fa0..10f8bd76232d7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.errors; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.api.Record; import org.slf4j.Logger; @@ -23,12 +24,15 @@ import java.util.Map; +import static org.apache.kafka.streams.errors.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; + /** * Processing exception handler that logs a processing exception and then * signals the processing pipeline to stop processing more records and fail. */ -public class LogAndFailProcessingExceptionHandler extends CommonExceptionHandler implements ProcessingExceptionHandler { +public class LogAndFailProcessingExceptionHandler implements ProcessingExceptionHandler { private static final Logger log = LoggerFactory.getLogger(LogAndFailProcessingExceptionHandler.class); + private String deadLetterQueueTopic; @Override public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { @@ -42,11 +46,11 @@ public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final exception ); - return ProcessingHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(null, null, context, exception)); + return ProcessingHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } @Override public void configure(final Map configs) { - super.configure(configs); + deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); } } 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 index 4b730d32b1f23..e41169bb7e403 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java @@ -81,7 +81,7 @@ public ProcessingExceptionHandler.ProcessingHandlerResponse andAddToDeadLetterQu return this; } - public List> drainDeadLetterQueueRecords() { + public List> deadLetterQueueRecords() { final LinkedList> deadLetterQueueRecords = new LinkedList<>(); while (true) { final ProducerRecord record = this.deadLetterQueueRecordsQueue.poll(); 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 71958d64f017d..14b40ca82bee4 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 @@ -167,7 +167,7 @@ public ProductionExceptionHandler.ProductionExceptionHandlerResponse andAddToDea return this; } - public List> drainDeadLetterQueueRecords() { + public List> deadLetterQueueRecords() { final LinkedList> deadLetterQueueRecords = new LinkedList<>(); while (true) { final ProducerRecord record = this.deadLetterQueueRecordsQueue.poll(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index 4af05331cf312..2e104060d49e6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -243,7 +243,7 @@ public void process(final Record record) { ); } - final List> deadLetterQueueRecords = response.drainDeadLetterQueueRecords(); + final List> deadLetterQueueRecords = response.deadLetterQueueRecords(); if (!deadLetterQueueRecords.isEmpty()) { final RecordCollector collector = ((RecordCollector.Supplier) internalProcessorContext).recordCollector(); for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index 4f0294f3ce0da..a75d87ae3434a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -374,7 +374,7 @@ private void handleException(final ProductionExceptionHandler.Serializati ); } - final List> deadLetterQueueRecords = response.drainDeadLetterQueueRecords(); + final List> deadLetterQueueRecords = response.deadLetterQueueRecords(); if (!deadLetterQueueRecords.isEmpty()) { for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { this.send( @@ -512,7 +512,7 @@ private void recordSendError(final String topic, return; } - final List> deadLetterQueueRecords = response.drainDeadLetterQueueRecords(); + final List> deadLetterQueueRecords = response.deadLetterQueueRecords(); if (!deadLetterQueueRecords.isEmpty()) { for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { this.send( diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 418ae3911fece..14a70117a19d0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -120,7 +120,7 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa throw new StreamsException("Fatal user code error in deserialization error callback", fatalUserException); } - final List> deadLetterQueueRecords = response.drainDeadLetterQueueRecords(); + final List> deadLetterQueueRecords = response.deadLetterQueueRecords(); if (!deadLetterQueueRecords.isEmpty()) { final RecordCollector collector = ((RecordCollector.Supplier) processorContext).recordCollector(); for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index b1d3b98bdc39b..3e99fce82adee 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -964,7 +964,7 @@ record = null; throw new FailedProcessingException("Fatal user code error in processing error callback", node.name(), fatalUserException); } - final List> deadLetterQueueRecords = response.drainDeadLetterQueueRecords(); + final List> deadLetterQueueRecords = response.deadLetterQueueRecords(); if (!deadLetterQueueRecords.isEmpty()) { final RecordCollector collector = ((RecordCollector.Supplier) processorContext).recordCollector(); for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index 6505cde08ed6e..bd8002782d275 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -1683,6 +1683,11 @@ public void shouldThrowConfigExceptionWhenStreamsProtocolUsedWithStaticMembershi "Please set group.protocol=classic or remove group.instance.id from the configuration.")); } + public void shouldSetDefaultDeadLetterQueue() { + final StreamsConfig config = new StreamsConfig(props); + assertNull(config.getString(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); + } + static class MisconfiguredSerde implements Serde { @Override public void configure(final Map configs, final boolean isKey) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/errors/CommonExceptionHandlerTest.java b/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java similarity index 78% rename from streams/src/test/java/org/apache/kafka/streams/processor/internals/errors/CommonExceptionHandlerTest.java rename to streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java index d0d09f5126bf4..e84a76427e61b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/errors/CommonExceptionHandlerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.streams.processor.internals.errors; +package org.apache.kafka.streams.errors; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.header.Headers; @@ -23,16 +23,12 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.errors.CommonExceptionHandler; -import org.apache.kafka.streams.errors.ErrorHandlerContext; import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.test.InternalMockProcessorContext; import org.apache.kafka.test.MockRecordCollector; -import org.apache.kafka.test.StreamsTestUtils; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -47,7 +43,7 @@ @ExtendWith(MockitoExtension.class) -public class CommonExceptionHandlerTest { +public class ExceptionHandlerUtilsTest { @Test public void checkDealLetterQueueRecords() { final StringSerializer stringSerializer = new StringSerializer(); @@ -67,10 +63,8 @@ public void checkDealLetterQueueRecords() { )); final ErrorHandlerContext errorHandlerContext = getErrorHandlerContext(internalProcessorContext); - final CommonExceptionHandler commonExceptionHandler = new CommonExceptionHandler(); - commonExceptionHandler.setDeadLetterQueueTopicName("dlq"); final NullPointerException exception = new NullPointerException("Oopsie!"); - final Iterable> dlqRecords = commonExceptionHandler.maybeBuildDeadLetterQueueRecords(null, null, errorHandlerContext, exception); + final Iterable> dlqRecords = ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords("dlq", null, null, errorHandlerContext, exception); final Iterator> iterator = dlqRecords.iterator(); assertTrue(iterator.hasNext()); @@ -81,20 +75,17 @@ public void checkDealLetterQueueRecords() { assertEquals("dlq", dlqRecord.topic()); assertEquals(errorHandlerContext.timestamp(), dlqRecord.timestamp()); assertEquals(1, dlqRecord.timestamp()); - assertEquals(exception.toString(), stringDeserializer.deserialize(null, headers.lastHeader(CommonExceptionHandler.HEADER_ERRORS_EXCEPTION_NAME).value())); - assertEquals(exception.getMessage(), stringDeserializer.deserialize(null, headers.lastHeader(CommonExceptionHandler.HEADER_ERRORS_EXCEPTION_MESSAGE_NAME).value())); - assertEquals("source", stringDeserializer.deserialize(null, headers.lastHeader(CommonExceptionHandler.HEADER_ERRORS_TOPIC_NAME).value())); - assertEquals("3", stringDeserializer.deserialize(null, headers.lastHeader(CommonExceptionHandler.HEADER_ERRORS_PARTITION_NAME).value())); - assertEquals("2", stringDeserializer.deserialize(null, headers.lastHeader(CommonExceptionHandler.HEADER_ERRORS_OFFSET_NAME).value())); + assertEquals(exception.toString(), stringDeserializer.deserialize(null, headers.lastHeader(ExceptionHandlerUtils.HEADER_ERRORS_EXCEPTION_NAME).value())); + assertEquals(exception.getMessage(), stringDeserializer.deserialize(null, headers.lastHeader(ExceptionHandlerUtils.HEADER_ERRORS_EXCEPTION_MESSAGE_NAME).value())); + assertEquals("source", stringDeserializer.deserialize(null, headers.lastHeader(ExceptionHandlerUtils.HEADER_ERRORS_TOPIC_NAME).value())); + assertEquals("3", stringDeserializer.deserialize(null, headers.lastHeader(ExceptionHandlerUtils.HEADER_ERRORS_PARTITION_NAME).value())); + assertEquals("2", stringDeserializer.deserialize(null, headers.lastHeader(ExceptionHandlerUtils.HEADER_ERRORS_OFFSET_NAME).value())); } @Test public void doNotBuildDeadLetterQueueRecordsIfNotConfigured() { - final CommonExceptionHandler commonExceptionHandler = new CommonExceptionHandler(); - final StreamsConfig streamsConfig = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); - commonExceptionHandler.configure(streamsConfig.originals()); final NullPointerException exception = new NullPointerException("Oopsie!"); - final Iterable> dlqRecords = commonExceptionHandler.maybeBuildDeadLetterQueueRecords(null, null, null, exception); + final Iterable> dlqRecords = ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords(null, null, null, null, exception); final Iterator> iterator = dlqRecords.iterator(); assertFalse(iterator.hasNext()); From 6c75500ca209dbc042c129cafea61f32e63653f8 Mon Sep 17 00:00:00 2001 From: Damien Gasparina Date: Wed, 27 Nov 2024 17:54:59 +0100 Subject: [PATCH 05/16] Implementing KIP-1034 - adding new interfaces --- .../streams/errors/DefaultProductionExceptionHandler.java | 5 +++-- .../kafka/streams/errors/LogAndContinueExceptionHandler.java | 5 +++-- .../errors/LogAndContinueProcessingExceptionHandler.java | 5 +++-- .../kafka/streams/errors/LogAndFailExceptionHandler.java | 5 +++-- .../streams/errors/LogAndFailProcessingExceptionHandler.java | 5 +++-- 5 files changed, 15 insertions(+), 10 deletions(-) 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 ef6e3fcc89163..845dbbd982135 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 @@ -30,7 +30,7 @@ */ public class DefaultProductionExceptionHandler implements ProductionExceptionHandler { - private String deadLetterQueueTopic; + private String deadLetterQueueTopic = null; /** * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ProducerRecord, Exception)} instead. @@ -64,6 +64,7 @@ public ProductionExceptionHandlerResponse handleSerializationException(final Err @Override public void configure(final Map configs) { - deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); + if (configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG) != null) + deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); } } \ No newline at end of file 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 187d765353b57..1354877e1104f 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 @@ -33,7 +33,7 @@ */ public class LogAndContinueExceptionHandler implements DeserializationExceptionHandler { private static final Logger log = LoggerFactory.getLogger(LogAndContinueExceptionHandler.class); - private String deadLetterQueueTopic; + private String deadLetterQueueTopic = null; /** * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)} instead. @@ -76,6 +76,7 @@ public DeserializationHandlerResponse handle(final ErrorHandlerContext context, @Override public void configure(final Map configs) { - deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); + if (configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG) != null) + deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java index 709e423360d37..b5123ef120e39 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java @@ -32,7 +32,7 @@ */ public class LogAndContinueProcessingExceptionHandler implements ProcessingExceptionHandler { private static final Logger log = LoggerFactory.getLogger(LogAndContinueProcessingExceptionHandler.class); - private String deadLetterQueueTopic; + private String deadLetterQueueTopic = null; @Override public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { @@ -51,6 +51,7 @@ public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final @Override public void configure(final Map configs) { - deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); + if (configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG) != null) + deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); } } 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 8a242d39f1f39..5a8102691bbb8 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 @@ -33,7 +33,7 @@ */ public class LogAndFailExceptionHandler implements DeserializationExceptionHandler { private static final Logger log = LoggerFactory.getLogger(LogAndFailExceptionHandler.class); - private String deadLetterQueueTopic; + private String deadLetterQueueTopic = null; /** * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)} instead. @@ -76,6 +76,7 @@ public DeserializationHandlerResponse handle(final ErrorHandlerContext context, @Override public void configure(final Map configs) { - deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); + if (configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG) != null) + deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); } } \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java index 10f8bd76232d7..cede3eb35d014 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java @@ -32,7 +32,7 @@ */ public class LogAndFailProcessingExceptionHandler implements ProcessingExceptionHandler { private static final Logger log = LoggerFactory.getLogger(LogAndFailProcessingExceptionHandler.class); - private String deadLetterQueueTopic; + private String deadLetterQueueTopic = null; @Override public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { @@ -51,6 +51,7 @@ public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final @Override public void configure(final Map configs) { - deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); + if (configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG) != null) + deadLetterQueueTopic = String.valueOf(configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); } } From d9c13fa4ba4b26ed36f021270f24b19734ca5b13 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Tue, 10 Dec 2024 09:59:46 +0100 Subject: [PATCH 06/16] KIP-1034 - adding new interfaces --- .../DefaultProductionExceptionHandler.java | 33 ++- .../DeserializationExceptionHandler.java | 130 +++++++++--- .../streams/errors/ExceptionHandlerUtils.java | 13 +- .../LogAndContinueExceptionHandler.java | 20 +- ...AndContinueProcessingExceptionHandler.java | 24 ++- .../errors/LogAndFailExceptionHandler.java | 20 +- .../LogAndFailProcessingExceptionHandler.java | 23 ++- .../errors/ProcessingExceptionHandler.java | 137 +++++++++--- .../errors/ProductionExceptionHandler.java | 195 +++++++++++++++--- .../processor/internals/ProcessorNode.java | 12 +- .../internals/RecordCollectorImpl.java | 21 +- .../internals/RecordDeserializer.java | 6 +- .../processor/internals/StreamTask.java | 10 +- 13 files changed, 520 insertions(+), 124 deletions(-) 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 845dbbd982135..02d4111a261dc 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 @@ -41,27 +41,50 @@ public class DefaultProductionExceptionHandler implements ProductionExceptionHan public ProductionExceptionHandlerResponse handle(final ProducerRecord record, final Exception exception) { return exception instanceof RetriableException ? - ProductionExceptionHandlerResponse.RETRY : - ProductionExceptionHandlerResponse.FAIL; + ProductionExceptionHandler.ProductionExceptionHandlerResponse.RETRY : + ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL; } + @SuppressWarnings("deprecation") + @Deprecated @Override public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, final ProducerRecord record, final Exception exception) { return exception instanceof RetriableException ? - ProductionExceptionHandlerResponse.RETRY : - ProductionExceptionHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + ProductionExceptionHandler.ProductionExceptionHandlerResponse.RETRY : + ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL; } + @Override + public ProductionExceptionResponse handleError(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception) { + return exception instanceof RetriableException ? + ProductionExceptionResponse.retryProcessing() : + ProductionExceptionResponse.continueProcessing(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + } + + + @SuppressWarnings("deprecation") + @Deprecated @Override public ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context, final ProducerRecord record, final Exception exception, final SerializationExceptionOrigin origin) { - return ProductionExceptionHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + return ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL; + } + + @Override + public ProductionExceptionResponse handleSerializationError(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception, + final SerializationExceptionOrigin origin) { + return ProductionExceptionResponse.failProcessing(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } + @Override public void configure(final Map configs) { if (configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG) != null) 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 bb74d71114e4d..370a101f86845 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 @@ -22,10 +22,8 @@ import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; import org.apache.kafka.streams.processor.ProcessorContext; -import java.util.LinkedList; +import java.util.Collections; import java.util.List; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; /** * Interface that specifies how an exception from source node deserialization @@ -69,13 +67,34 @@ default DeserializationHandlerResponse handle(final ProcessorContext context, * The actual exception. * * @return Whether to continue or stop processing. + * + * @deprecated Use {@link #handleError(ErrorHandlerContext, ConsumerRecord, Exception)} instead. */ + @Deprecated default DeserializationHandlerResponse handle(final ErrorHandlerContext context, final ConsumerRecord record, final Exception exception) { return handle(((DefaultErrorHandlerContext) context).processorContext().orElse(null), record, exception); } + /** + * Inspects a record and the exception received during deserialization. + * + * @param context + * Error handler context. + * @param record + * Record that failed deserialization. + * @param exception + * The actual exception. + * + * @return a {@link DeserializationExceptionResponse} object + */ + default DeserializationExceptionResponse handleError(final ErrorHandlerContext context, final ConsumerRecord record, final Exception exception) { + if (DeserializationHandlerResponse.FAIL == handle(context, record, exception)) { + return DeserializationExceptionResponse.failProcessing(Collections.emptyList()); + } + return DeserializationExceptionResponse.continueProcessing(Collections.emptyList()); + } /** * Enumeration that describes the response from the exception handler. */ @@ -95,35 +114,100 @@ enum DeserializationHandlerResponse { */ public final int id; - /** a list of Kafka records to publish, e.g. in a Dead Letter Queue topic */ - private final Queue> deadLetterQueueRecordsQueue; - DeserializationHandlerResponse(final int id, final String name) { this.id = id; this.name = name; - this.deadLetterQueueRecordsQueue = new ConcurrentLinkedQueue<>(); } + } - public DeserializationHandlerResponse andAddToDeadLetterQueue(final Iterable> deadLetterQueueRecords) { - if (deadLetterQueueRecords == null) { - return this; - } - for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { - this.deadLetterQueueRecordsQueue.add(deadLetterQueueRecord); - } - return this; + /** + * Represents the result of handling a deserialization exception. + *

+ * The {@code Response} class encapsulates a {@link ProcessingExceptionHandler.ProcessingHandlerResponse}, + * indicating whether processing should continue or fail, along with an optional list of + * {@link ProducerRecord} instances to be sent to a dead letter queue. + *

+ */ + class DeserializationExceptionResponse { + + private DeserializationHandlerResponse deserializationHandlerResponse; + + private List> deadLetterQueueRecords; + + /** + * Constructs a new {@code DeserializationExceptionResponse} object. + * + * @param deserializationHandlerResponse the response indicating whether processing should continue or fail; + * must not be {@code null}. + * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. + */ + private DeserializationExceptionResponse(final DeserializationHandlerResponse deserializationHandlerResponse, + final List> deadLetterQueueRecords) { + this.deserializationHandlerResponse = deserializationHandlerResponse; + this.deadLetterQueueRecords = deadLetterQueueRecords; + } + + /** + * Creates a {@code DeserializationExceptionResponse} indicating that processing should fail. + * + * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. + * @return a {@code DeserializationExceptionResponse} with a {@link DeserializationHandlerResponse#FAIL} status. + */ + public static DeserializationExceptionResponse failProcessing(final List> deadLetterQueueRecords) { + return new DeserializationExceptionResponse(DeserializationHandlerResponse.FAIL, deadLetterQueueRecords); + } + + /** + * Creates a {@code DeserializationExceptionResponse} indicating that processing should fail. + * + * @return a {@code DeserializationExceptionResponse} with a {@link DeserializationHandlerResponse#FAIL} status. + */ + public static DeserializationExceptionResponse failProcessing() { + return failProcessing(Collections.emptyList()); + } + + /** + * Creates a {@code DeserializationExceptionResponse} indicating that processing should continue. + * + * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. + * @return a {@code DeserializationExceptionResponse} with a {@link DeserializationHandlerResponse#CONTINUE} status. + */ + public static DeserializationExceptionResponse continueProcessing(final List> deadLetterQueueRecords) { + return new DeserializationExceptionResponse(DeserializationHandlerResponse.CONTINUE, deadLetterQueueRecords); + } + + /** + * Creates a {@code DeserializationExceptionResponse} indicating that processing should continue. + * + * @return a {@code DeserializationExceptionResponse} with a {@link DeserializationHandlerResponse#CONTINUE} status. + */ + public static DeserializationExceptionResponse continueProcessing() { + return continueProcessing(Collections.emptyList()); + } + + /** + * Retrieves the deserialization handler response. + * + * @return the {@link DeserializationHandlerResponse} indicating whether processing should continue or fail. + */ + public DeserializationHandlerResponse response() { + return deserializationHandlerResponse; } + /** + * Retrieves an unmodifiable list of records to be sent to the dead letter queue. + *

+ * If the list is {@code null}, an empty list is returned. + *

+ * + * @return an unmodifiable list of {@link ProducerRecord} instances + * for the dead letter queue, or an empty list if no records are available. + */ public List> deadLetterQueueRecords() { - final LinkedList> deadLetterQueueRecords = new LinkedList<>(); - while (true) { - final ProducerRecord record = this.deadLetterQueueRecordsQueue.poll(); - if (record == null) { - break; - } - deadLetterQueueRecords.add(record); + if (deadLetterQueueRecords == null) { + return Collections.emptyList(); } - return deadLetterQueueRecords; + return Collections.unmodifiableList(deadLetterQueueRecords); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ExceptionHandlerUtils.java b/streams/src/main/java/org/apache/kafka/streams/errors/ExceptionHandlerUtils.java index eb0c72896ef75..c8a8cfdddc64d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ExceptionHandlerUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ExceptionHandlerUtils.java @@ -25,6 +25,7 @@ import java.io.PrintWriter; import java.io.StringWriter; import java.util.Collections; +import java.util.List; /** * {@code CommonExceptionHandler} Contains utilities method that could be used by all exception handlers @@ -50,16 +51,16 @@ static boolean shouldBuildDeadLetterQueueRecord(final String deadLetterQueueTopi * @param exception Thrown exception * @return A list of Dead Letter Queue records to produce */ - static Iterable> maybeBuildDeadLetterQueueRecords(final String deadLetterQueueTopicName, - final byte[] key, - final byte[] value, - final ErrorHandlerContext context, - final Exception exception) { + static List> maybeBuildDeadLetterQueueRecords(final String deadLetterQueueTopicName, + final byte[] key, + final byte[] value, + final ErrorHandlerContext context, + final Exception exception) { if (!shouldBuildDeadLetterQueueRecord(deadLetterQueueTopicName)) { return Collections.emptyList(); } - return Collections.singleton(buildDeadLetterQueueRecord(deadLetterQueueTopicName, key, value, context, exception)); + return Collections.singletonList(buildDeadLetterQueueRecord(deadLetterQueueTopicName, key, value, context, exception)); } 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 1354877e1104f..c67c773ac4bd8 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 @@ -57,6 +57,8 @@ public DeserializationHandlerResponse handle(final ProcessorContext context, return DeserializationHandlerResponse.CONTINUE; } + @SuppressWarnings("deprecation") + @Deprecated @Override public DeserializationHandlerResponse handle(final ErrorHandlerContext context, final ConsumerRecord record, @@ -71,7 +73,23 @@ public DeserializationHandlerResponse handle(final ErrorHandlerContext context, exception ); - return DeserializationHandlerResponse.CONTINUE.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + return DeserializationHandlerResponse.CONTINUE; + } + + @Override + public DeserializationExceptionResponse handleError(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 DeserializationExceptionResponse.continueProcessing(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java index b5123ef120e39..9057731a870ad 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java @@ -34,8 +34,11 @@ public class LogAndContinueProcessingExceptionHandler implements ProcessingExcep private static final Logger log = LoggerFactory.getLogger(LogAndContinueProcessingExceptionHandler.class); private String deadLetterQueueTopic = null; + @Deprecated @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + 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(), @@ -46,9 +49,26 @@ public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final exception ); - return ProcessingHandlerResponse.CONTINUE.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + return ProcessingHandlerResponse.CONTINUE; } + @Override + public ProcessingExceptionResponse handleError(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 ProcessingExceptionResponse.continueProcessing(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + } + + @Override public void configure(final Map configs) { if (configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG) != null) 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 5a8102691bbb8..c06ccfba015b8 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 @@ -57,6 +57,8 @@ public DeserializationHandlerResponse handle(final ProcessorContext context, return DeserializationHandlerResponse.FAIL; } + @SuppressWarnings("deprecation") + @Deprecated @Override public DeserializationHandlerResponse handle(final ErrorHandlerContext context, final ConsumerRecord record, @@ -71,7 +73,23 @@ public DeserializationHandlerResponse handle(final ErrorHandlerContext context, exception ); - return DeserializationHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + return DeserializationHandlerResponse.FAIL; + } + + @Override + public DeserializationExceptionResponse handleError(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 DeserializationExceptionResponse.failProcessing(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java index cede3eb35d014..393e30b8e7a81 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java @@ -34,8 +34,10 @@ public class LogAndFailProcessingExceptionHandler implements ProcessingException private static final Logger log = LoggerFactory.getLogger(LogAndFailProcessingExceptionHandler.class); private String deadLetterQueueTopic = null; + @Deprecated @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, + final Record record, final Exception exception) { log.error( "Exception caught during message processing, processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", context.processorNodeId(), @@ -46,7 +48,24 @@ public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final exception ); - return ProcessingHandlerResponse.FAIL.andAddToDeadLetterQueue(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + return ProcessingHandlerResponse.FAIL; + } + + @Override + public ProcessingExceptionResponse handleError(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 ProcessingExceptionResponse.failProcessing(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } @Override 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 index e41169bb7e403..da3055d907758 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java @@ -20,15 +20,14 @@ import org.apache.kafka.common.Configurable; import org.apache.kafka.streams.processor.api.Record; -import java.util.LinkedList; +import java.util.Collections; import java.util.List; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; /** * 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 * @@ -40,8 +39,32 @@ public interface ProcessingExceptionHandler extends Configurable { * The actual exception. * * @return Whether to continue or stop processing. + * @deprecated Use {@link #handleError(ErrorHandlerContext, Record, Exception)} instead. */ - ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception); + @Deprecated + default ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + throw new UnsupportedOperationException(); + }; + + /** + * Inspects a record and the exception received during processing. + * + * @param context + * Processing context metadata. + * @param record + * Record where the exception occurred. + * @param exception + * The actual exception. + * + * @return a {@link ProcessingExceptionResponse} object + */ + default ProcessingExceptionResponse handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { + if (ProcessingHandlerResponse.FAIL == handle(context, record, exception)) { + return ProcessingExceptionResponse.failProcessing(); + } + return ProcessingExceptionResponse.continueProcessing(); + } + enum ProcessingHandlerResponse { /** Continue processing. */ @@ -59,38 +82,100 @@ enum ProcessingHandlerResponse { */ public final int id; - /** - * a list of Kafka records to publish, e.g. in a Dead Letter Queue topic - */ - private final Queue> deadLetterQueueRecordsQueue; - ProcessingHandlerResponse(final int id, final String name) { this.id = id; this.name = name; - deadLetterQueueRecordsQueue = new ConcurrentLinkedQueue<>(); } + } + /** + * Represents the result of handling a processing exception. + *

+ * The {@code Response} class encapsulates a {@link ProcessingHandlerResponse}, + * indicating whether processing should continue or fail, along with an optional list of + * {@link org.apache.kafka.clients.producer.ProducerRecord} instances to be sent to a dead letter queue. + *

+ */ + class ProcessingExceptionResponse { - public ProcessingExceptionHandler.ProcessingHandlerResponse andAddToDeadLetterQueue(final Iterable> deadLetterQueueRecords) { - if (deadLetterQueueRecords == null) { - return this; - } - for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { - this.deadLetterQueueRecordsQueue.add(deadLetterQueueRecord); - } - return this; + private ProcessingHandlerResponse processingHandlerResponse; + + private List> deadLetterQueueRecords; + + /** + * Constructs a new {@code ProcessingExceptionResponse} object. + * + * @param processingHandlerResponse the response indicating whether processing should continue or fail; + * must not be {@code null}. + * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. + */ + private ProcessingExceptionResponse(final ProcessingHandlerResponse processingHandlerResponse, + final List> deadLetterQueueRecords) { + this.processingHandlerResponse = processingHandlerResponse; + this.deadLetterQueueRecords = deadLetterQueueRecords; } + /** + * Creates a {@code ProcessingExceptionResponse} indicating that processing should fail. + * + * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. + * @return a {@code ProcessingExceptionResponse} with a {@link ProcessingHandlerResponse#FAIL} status. + */ + public static ProcessingExceptionResponse failProcessing(final List> deadLetterQueueRecords) { + return new ProcessingExceptionResponse(ProcessingHandlerResponse.FAIL, deadLetterQueueRecords); + } + + /** + * Creates a {@code ProcessingExceptionResponse} indicating that processing should fail. + * + * @return a {@code ProcessingExceptionResponse} with a {@link ProcessingHandlerResponse#FAIL} status. + */ + public static ProcessingExceptionResponse failProcessing() { + return failProcessing(Collections.emptyList()); + } + + /** + * Creates a {@code ProcessingExceptionResponse} indicating that processing should continue. + * + * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. + * @return a {@code Response} with a {@link ProcessingHandlerResponse#CONTINUE} status. + */ + public static ProcessingExceptionResponse continueProcessing(final List> deadLetterQueueRecords) { + return new ProcessingExceptionResponse(ProcessingHandlerResponse.CONTINUE, deadLetterQueueRecords); + } + + /** + * Creates a {@code ProcessingExceptionResponse} indicating that processing should continue. + * + * @return a {@code ProcessingExceptionResponse} with a {@link ProcessingHandlerResponse#CONTINUE} status. + */ + public static ProcessingExceptionResponse continueProcessing() { + return continueProcessing(Collections.emptyList()); + } + + /** + * Retrieves the processing handler response. + * + * @return the {@link ProcessingHandlerResponse} indicating whether processing should continue or fail. + */ + public ProcessingHandlerResponse response() { + return processingHandlerResponse; + } + + /** + * Retrieves an unmodifiable list of records to be sent to the dead letter queue. + *

+ * If the list is {@code null}, an empty list is returned. + *

+ * + * @return an unmodifiable list of {@link ProducerRecord} instances + * for the dead letter queue, or an empty list if no records are available. + */ public List> deadLetterQueueRecords() { - final LinkedList> deadLetterQueueRecords = new LinkedList<>(); - while (true) { - final ProducerRecord record = this.deadLetterQueueRecordsQueue.poll(); - if (record == null) { - break; - } - deadLetterQueueRecords.add(record); + if (deadLetterQueueRecords == null) { + return Collections.emptyList(); } - return deadLetterQueueRecords; + return Collections.unmodifiableList(deadLetterQueueRecords); } } } 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 14b40ca82bee4..893412fc55226 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 @@ -19,10 +19,8 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Configurable; -import java.util.LinkedList; +import java.util.Collections; import java.util.List; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; /** * Interface that specifies how an exception when attempting to produce a result to @@ -60,11 +58,38 @@ default ProductionExceptionHandlerResponse handle(final ProducerRecord record, final Exception exception) { - return handle(record, exception); + throw new UnsupportedOperationException(); + } + + /** + * Inspect a record that we attempted to produce, and the exception that resulted + * from attempting to produce it and determine to continue or stop processing. + * + * @param context + * The error handler context metadata. + * @param record + * The record that failed to produce. + * @param exception + * The exception that occurred during production. + * + * @return a {@link ProductionExceptionResponse} object + */ + default ProductionExceptionResponse handleError(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception) { + final ProductionExceptionHandlerResponse response = handle(context, record, exception); + if (ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL == response) { + return ProductionExceptionResponse.failProcessing(); + } else if (ProductionExceptionHandler.ProductionExceptionHandlerResponse.RETRY == response) { + return ProductionExceptionResponse.retryProcessing(); + } + return ProductionExceptionResponse.continueProcessing(); } /** @@ -84,7 +109,7 @@ default ProductionExceptionHandlerResponse handle(final ErrorHandlerContext cont @Deprecated default ProductionExceptionHandlerResponse handleSerializationException(final ProducerRecord record, final Exception exception) { - return ProductionExceptionHandlerResponse.FAIL; + return ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL; } /** @@ -101,8 +126,11 @@ default ProductionExceptionHandlerResponse handleSerializationException(final Pr * The origin of the serialization exception. * * @return Whether to continue or stop processing, or retry the failed operation. + * + * @deprecated Use {@link #handleSerializationError(ErrorHandlerContext, ProducerRecord, Exception, SerializationExceptionOrigin)} instead. */ @SuppressWarnings("rawtypes") + @Deprecated default ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context, final ProducerRecord record, final Exception exception, @@ -110,6 +138,35 @@ default ProductionExceptionHandlerResponse handleSerializationException(final Er return handleSerializationException(record, exception); } + /** + * 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. + * + * @return a {@link ProductionExceptionResponse} object + */ + @SuppressWarnings("rawtypes") + default ProductionExceptionResponse handleSerializationError(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception, + final SerializationExceptionOrigin origin) { + final ProductionExceptionHandlerResponse response = handleSerializationException(context, record, exception, origin); + if (ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL == response) { + return ProductionExceptionResponse.failProcessing(); + } else if (ProductionExceptionHandler.ProductionExceptionHandlerResponse.RETRY == response) { + return ProductionExceptionResponse.retryProcessing(); + } + return ProductionExceptionResponse.continueProcessing(); + } + enum ProductionExceptionHandlerResponse { /** Continue processing. * @@ -145,38 +202,10 @@ enum ProductionExceptionHandlerResponse { */ public final int id; - /** - * a list of Kafka records to publish, e.g. in a Dead Letter Queue topic - */ - private final Queue> deadLetterQueueRecordsQueue; - ProductionExceptionHandlerResponse(final int id, final String name) { this.id = id; this.name = name; - this.deadLetterQueueRecordsQueue = new ConcurrentLinkedQueue<>(); - } - - public ProductionExceptionHandler.ProductionExceptionHandlerResponse andAddToDeadLetterQueue(final Iterable> deadLetterQueueRecords) { - if (deadLetterQueueRecords == null) { - return this; - } - for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { - this.deadLetterQueueRecordsQueue.add(deadLetterQueueRecord); - } - return this; - } - - public List> deadLetterQueueRecords() { - final LinkedList> deadLetterQueueRecords = new LinkedList<>(); - while (true) { - final ProducerRecord record = this.deadLetterQueueRecordsQueue.poll(); - if (record == null) { - break; - } - deadLetterQueueRecords.add(record); - } - return deadLetterQueueRecords; } } @@ -186,4 +215,104 @@ enum SerializationExceptionOrigin { /** Serialization exception occurred during serialization of the value. */ VALUE } + + /** + * Represents the result of handling a production exception. + *

+ * The {@code Response} class encapsulates a {@link ProductionExceptionHandlerResponse}, + * indicating whether processing should continue or fail, along with an optional list of + * {@link ProducerRecord} instances to be sent to a dead letter queue. + *

+ */ + class ProductionExceptionResponse { + + private ProductionExceptionHandlerResponse productionExceptionHandlerResponse; + + private List> deadLetterQueueRecords; + + /** + * Constructs a new {@code ProductionExceptionResponse} object. + * + * @param productionExceptionHandlerResponse the response indicating whether processing should continue or fail; + * must not be {@code null}. + * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. + */ + private ProductionExceptionResponse(final ProductionExceptionHandlerResponse productionExceptionHandlerResponse, + final List> deadLetterQueueRecords) { + this.productionExceptionHandlerResponse = productionExceptionHandlerResponse; + this.deadLetterQueueRecords = deadLetterQueueRecords; + } + + /** + * Creates a {@code ProductionExceptionResponse} indicating that processing should fail. + * + * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. + * @return a {@code ProductionExceptionResponse} with a {@link DeserializationExceptionHandler.DeserializationHandlerResponse#FAIL} status. + */ + public static ProductionExceptionResponse failProcessing(final List> deadLetterQueueRecords) { + return new ProductionExceptionResponse(ProductionExceptionHandlerResponse.FAIL, deadLetterQueueRecords); + } + + /** + * Creates a {@code ProductionExceptionResponse} indicating that processing should fail. + * + * @return a {@code ProductionExceptionResponse} with a {@link DeserializationExceptionHandler.DeserializationHandlerResponse#FAIL} status. + */ + public static ProductionExceptionResponse failProcessing() { + return new ProductionExceptionResponse(ProductionExceptionHandlerResponse.FAIL, Collections.emptyList()); + } + + /** + * Creates a {@code ProductionExceptionResponse} indicating that processing should continue. + * + * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. + * @return a {@code ProductionExceptionResponse} with a {@link DeserializationExceptionHandler.DeserializationHandlerResponse#CONTINUE} status. + */ + public static ProductionExceptionResponse continueProcessing(final List> deadLetterQueueRecords) { + return new ProductionExceptionResponse(ProductionExceptionHandlerResponse.CONTINUE, deadLetterQueueRecords); + } + + /** + * Creates a {@code ProductionExceptionResponse} indicating that processing should continue. + * + * @return a {@code ProductionExceptionResponse} with a {@link DeserializationExceptionHandler.DeserializationHandlerResponse#CONTINUE} status. + */ + public static ProductionExceptionResponse continueProcessing() { + return new ProductionExceptionResponse(ProductionExceptionHandlerResponse.CONTINUE, Collections.emptyList()); + } + + /** + * Creates a {@code ProductionExceptionResponse} indicating that processing should retry. + * + * @return a {@code ProductionExceptionResponse} with a {@link DeserializationExceptionHandler.DeserializationHandlerResponse#CONTINUE} status. + */ + public static ProductionExceptionResponse retryProcessing() { + return new ProductionExceptionResponse(ProductionExceptionHandlerResponse.RETRY, Collections.emptyList()); + } + + /** + * Retrieves the production exception handler response. + * + * @return the {@link ProductionExceptionHandlerResponse} indicating whether processing should continue or fail. + */ + public ProductionExceptionHandlerResponse response() { + return productionExceptionHandlerResponse; + } + + /** + * Retrieves an unmodifiable list of records to be sent to the dead letter queue. + *

+ * If the list is {@code null}, an empty list is returned. + *

+ * + * @return an unmodifiable list of {@link ProducerRecord} instances + * for the dead letter queue, or an empty list if no records are available. + */ + public List> deadLetterQueueRecords() { + if (deadLetterQueueRecords == null) { + return Collections.emptyList(); + } + return Collections.unmodifiableList(deadLetterQueueRecords); + } + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index 2e104060d49e6..4902420cd08b3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -221,11 +221,11 @@ public void process(final Record record) { internalProcessorContext.recordContext().sourceRawValue() ); - final ProcessingExceptionHandler.ProcessingHandlerResponse response; + final ProcessingExceptionHandler.ProcessingExceptionResponse processingExceptionResponse; try { - response = Objects.requireNonNull( - processingExceptionHandler.handle(errorHandlerContext, record, processingException), - "Invalid ProductionExceptionHandler response." + processingExceptionResponse = Objects.requireNonNull( + processingExceptionHandler.handleError(errorHandlerContext, record, processingException), + "Invalid ProcessingResponse response." ); } catch (final Exception fatalUserException) { // while Java distinguishes checked vs unchecked exceptions, other languages @@ -243,7 +243,7 @@ public void process(final Record record) { ); } - final List> deadLetterQueueRecords = response.deadLetterQueueRecords(); + final List> deadLetterQueueRecords = processingExceptionResponse.deadLetterQueueRecords(); if (!deadLetterQueueRecords.isEmpty()) { final RecordCollector collector = ((RecordCollector.Supplier) internalProcessorContext).recordCollector(); for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { @@ -257,7 +257,7 @@ public void process(final Record record) { } } - if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { + if (processingExceptionResponse.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 " + diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index a75d87ae3434a..437613c61d6e7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -41,7 +41,6 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.errors.ProductionExceptionHandler; -import org.apache.kafka.streams.errors.ProductionExceptionHandler.ProductionExceptionHandlerResponse; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.errors.TaskMigratedException; @@ -344,10 +343,10 @@ private void handleException(final ProductionExceptionHandler.Serializati final ProducerRecord record = new ProducerRecord<>(topic, partition, timestamp, key, value, headers); - final ProductionExceptionHandlerResponse response; + final ProductionExceptionHandler.ProductionExceptionResponse response; try { response = Objects.requireNonNull( - productionExceptionHandler.handleSerializationException( + productionExceptionHandler.handleSerializationError( errorHandlerContext(context, processorNodeId), record, serializationException, @@ -387,7 +386,7 @@ private void handleException(final ProductionExceptionHandler.Serializati } } - if (maybeFailResponse(response) == ProductionExceptionHandlerResponse.FAIL) { + if (maybeFailResponse(response.response()) == ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL) { throw new StreamsException( String.format( "Unable to serialize record. ProducerRecord(topic=[%s], partition=[%d], timestamp=[%d]", @@ -484,10 +483,10 @@ private void recordSendError(final String topic, // TransactionAbortedException is only thrown after `abortTransaction()` was called, // so it's only a followup error, and Kafka Streams is already handling the original error } else { - final ProductionExceptionHandlerResponse response; + final ProductionExceptionHandler.ProductionExceptionResponse response; try { response = Objects.requireNonNull( - productionExceptionHandler.handle( + productionExceptionHandler.handleError( errorHandlerContext(context, processorNodeId), serializedRecord, productionException @@ -525,14 +524,14 @@ private void recordSendError(final String topic, } } - if (productionException instanceof RetriableException && response == ProductionExceptionHandlerResponse.RETRY) { + if (productionException instanceof RetriableException && response.response() == ProductionExceptionHandler.ProductionExceptionHandlerResponse.RETRY) { errorMessage += "\nThe broker is either slow or in bad state (like not having enough replicas) in responding the request, " + "or the connection to broker was interrupted sending the request or receiving the response. " + "\nConsider overwriting `max.block.ms` and /or " + "`delivery.timeout.ms` to a larger value to wait longer for such scenarios and avoid timeout errors"; sendException.set(new TaskCorruptedException(Collections.singleton(taskId))); } else { - if (maybeFailResponse(response) == ProductionExceptionHandlerResponse.FAIL) { + if (maybeFailResponse(response.response()) == ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL) { errorMessage += "\nException handler choose to FAIL the processing, no more records would be sent."; sendException.set(new StreamsException(errorMessage, productionException)); } else { @@ -545,10 +544,10 @@ private void recordSendError(final String topic, log.error(errorMessage, productionException); } - private ProductionExceptionHandlerResponse maybeFailResponse(final ProductionExceptionHandlerResponse response) { - if (response == ProductionExceptionHandlerResponse.RETRY) { + private ProductionExceptionHandler.ProductionExceptionHandlerResponse maybeFailResponse(final ProductionExceptionHandler.ProductionExceptionHandlerResponse response) { + if (response == ProductionExceptionHandler.ProductionExceptionHandlerResponse.RETRY) { log.warn("ProductionExceptionHandler returned RETRY for a non-retriable exception. Will treat it as FAIL."); - return ProductionExceptionHandlerResponse.FAIL; + return ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL; } else { return response; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 14a70117a19d0..78c699cf88353 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -102,10 +102,10 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa rawRecord.value() ); - final DeserializationHandlerResponse response; + final DeserializationExceptionHandler.DeserializationExceptionResponse response; try { response = Objects.requireNonNull( - deserializationExceptionHandler.handle(errorHandlerContext, rawRecord, deserializationException), + deserializationExceptionHandler.handleError(errorHandlerContext, rawRecord, deserializationException), "Invalid DeserializationExceptionHandler response." ); } catch (final Exception fatalUserException) { @@ -134,7 +134,7 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa } } - if (response == DeserializationHandlerResponse.FAIL) { + if (response.response() == DeserializationHandlerResponse.FAIL) { throw new StreamsException("Deserialization exception handler is set to fail upon" + " a deserialization error. If you would rather have the streaming pipeline" + " continue after a deserialization error, please set the " + diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 3e99fce82adee..510acef221675 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -946,10 +946,10 @@ record = null; recordContext.sourceRawValue() ); - final ProcessingExceptionHandler.ProcessingHandlerResponse response; + final ProcessingExceptionHandler.ProcessingExceptionResponse processingExceptionResponse; try { - response = Objects.requireNonNull( - processingExceptionHandler.handle(errorHandlerContext, null, processingException), + processingExceptionResponse = Objects.requireNonNull( + processingExceptionHandler.handleError(errorHandlerContext, null, processingException), "Invalid ProcessingExceptionHandler response." ); } catch (final Exception fatalUserException) { @@ -964,7 +964,7 @@ record = null; throw new FailedProcessingException("Fatal user code error in processing error callback", node.name(), fatalUserException); } - final List> deadLetterQueueRecords = response.deadLetterQueueRecords(); + final List> deadLetterQueueRecords = processingExceptionResponse.deadLetterQueueRecords(); if (!deadLetterQueueRecords.isEmpty()) { final RecordCollector collector = ((RecordCollector.Supplier) processorContext).recordCollector(); for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { @@ -977,7 +977,7 @@ record = null; } } - if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { + if (processingExceptionResponse.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 " + From 012c3a423bcfb02753e09450d96b504be396b341 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Wed, 11 Dec 2024 10:20:22 +0100 Subject: [PATCH 07/16] KAFKA-16505 fixing unit test --- ...essingExceptionHandlerIntegrationTest.java | 12 +++--- ...allowUnknownTopicErrorIntegrationTest.java | 6 +-- .../DefaultProductionExceptionHandler.java | 2 +- .../errors/ProductionExceptionHandler.java | 4 +- .../processor/internals/ProcessorNode.java | 2 +- .../internals/RecordCollectorImpl.java | 4 +- .../internals/RecordDeserializer.java | 2 +- .../internals/ProcessorNodeTest.java | 14 +++---- .../internals/RecordCollectorTest.java | 38 +++++++++---------- .../internals/RecordDeserializerTest.java | 13 +++---- .../processor/internals/StreamTaskTest.java | 4 +- 11 files changed, 50 insertions(+), 51 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java index 38711093ff8af..cd212b9f623e5 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java @@ -357,7 +357,7 @@ public void shouldStopProcessingWhenProcessingExceptionHandlerReturnsNull() { final StreamsException e = assertThrows(StreamsException.class, () -> inputTopic.pipeInput(eventError.key, eventError.value, Instant.EPOCH)); assertEquals("Fatal user code error in processing error callback", e.getMessage()); assertInstanceOf(NullPointerException.class, e.getCause()); - assertEquals("Invalid ProductionExceptionHandler response.", e.getCause().getMessage()); + assertEquals("Invalid ProcessingExceptionResponse response.", e.getCause().getMessage()); assertFalse(isExecuted.get()); } } @@ -524,7 +524,7 @@ public void shouldVerifySourceRawKeyAndSourceRawValuePresentOrNotInErrorHandlerC public static class ContinueProcessingExceptionHandlerMockTest implements ProcessingExceptionHandler { @Override - public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + public ProcessingExceptionResponse handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { if (((String) record.key()).contains("FATAL")) { throw new RuntimeException("KABOOM!"); } @@ -532,7 +532,7 @@ public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHa return null; } assertProcessingExceptionHandlerInputs(context, record, exception); - return ProcessingExceptionHandler.ProcessingHandlerResponse.CONTINUE; + return ProcessingExceptionResponse.continueProcessing(); } @Override @@ -542,10 +542,10 @@ public void configure(final Map configs) { } public static class FailProcessingExceptionHandlerMockTest implements ProcessingExceptionHandler { - @Override - public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + @Override + public ProcessingExceptionResponse handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { assertProcessingExceptionHandlerInputs(context, record, exception); - return ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL; + return ProcessingExceptionResponse.failProcessing(); } @Override diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SwallowUnknownTopicErrorIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SwallowUnknownTopicErrorIntegrationTest.java index a82e832e21c5a..853ce9c7cc332 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SwallowUnknownTopicErrorIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SwallowUnknownTopicErrorIntegrationTest.java @@ -156,15 +156,15 @@ public TestHandler() { } public void configure(final Map configs) { } @Override - public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, + public ProductionExceptionResponse handleError(final ErrorHandlerContext context, final ProducerRecord record, final Exception exception) { if (exception instanceof TimeoutException && exception.getCause() != null && exception.getCause() instanceof UnknownTopicOrPartitionException) { - return ProductionExceptionHandlerResponse.CONTINUE; + return ProductionExceptionResponse.continueProcessing(); } - return ProductionExceptionHandler.super.handle(context, record, exception); + return ProductionExceptionHandler.super.handleError(context, record, exception); } } 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 02d4111a261dc..97fd23e970fcc 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 @@ -62,7 +62,7 @@ public ProductionExceptionResponse handleError(final ErrorHandlerContext context final Exception exception) { return exception instanceof RetriableException ? ProductionExceptionResponse.retryProcessing() : - ProductionExceptionResponse.continueProcessing(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + ProductionExceptionResponse.failProcessing(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } 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 893412fc55226..31541c62d9929 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 @@ -259,7 +259,7 @@ public static ProductionExceptionResponse failProcessing(final List record) { try { processingExceptionResponse = Objects.requireNonNull( processingExceptionHandler.handleError(errorHandlerContext, record, processingException), - "Invalid ProcessingResponse response." + "Invalid ProcessingExceptionResponse response." ); } catch (final Exception fatalUserException) { // while Java distinguishes checked vs unchecked exceptions, other languages diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index 437613c61d6e7..b46d32401f940 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -352,7 +352,7 @@ private void handleException(final ProductionExceptionHandler.Serializati serializationException, origin ), - "Invalid ProductionExceptionHandler response." + "Invalid ProductionExceptionResponse response." ); } catch (final Exception fatalUserException) { // while Java distinguishes checked vs unchecked exceptions, other languages @@ -491,7 +491,7 @@ private void recordSendError(final String topic, serializedRecord, productionException ), - "Invalid ProductionExceptionHandler response." + "Invalid ProductionExceptionResponse response." ); } catch (final Exception fatalUserException) { // while Java distinguishes checked vs unchecked exceptions, other languages diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 78c699cf88353..b298934ffec4a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -106,7 +106,7 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa try { response = Objects.requireNonNull( deserializationExceptionHandler.handleError(errorHandlerContext, rawRecord, deserializationException), - "Invalid DeserializationExceptionHandler response." + "Invalid DeserializationExceptionResponse response." ); } catch (final Exception fatalUserException) { // while Java distinguishes checked vs unchecked exceptions, other languages diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index 76dda6ebbcd7c..f13b9c46c449c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -107,7 +107,7 @@ public void shouldThrowFailedProcessingExceptionWhenProcessingExceptionHandlerRe new ProcessorNode<>(NAME, new IgnoredInternalExceptionsProcessor(), Collections.emptySet()); final InternalProcessorContext internalProcessorContext = mockInternalProcessorContext(); - node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL, internalProcessorContext, false)); + node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingExceptionResponse.failProcessing(), internalProcessorContext, false)); final FailedProcessingException failedProcessingException = assertThrows(FailedProcessingException.class, () -> node.process(new Record<>(KEY, VALUE, TIMESTAMP))); @@ -124,7 +124,7 @@ public void shouldNotThrowFailedProcessingExceptionWhenProcessingExceptionHandle new ProcessorNode<>(NAME, new IgnoredInternalExceptionsProcessor(), Collections.emptySet()); final InternalProcessorContext internalProcessorContext = mockInternalProcessorContext(); - node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingHandlerResponse.CONTINUE, internalProcessorContext, false)); + node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingExceptionResponse.continueProcessing(), internalProcessorContext, false)); assertDoesNotThrow(() -> node.process(new Record<>(KEY, VALUE, TIMESTAMP))); } @@ -151,7 +151,7 @@ public void shouldNotHandleInternalExceptionsThrownDuringProcessing(final String assertEquals(ignoredExceptionCause, runtimeException.getCause().getClass()); assertEquals(ignoredExceptionCauseMessage, runtimeException.getCause().getMessage()); - verify(processingExceptionHandler, never()).handle(any(), any(), any()); + verify(processingExceptionHandler, never()).handleError(any(), any(), any()); } @Test @@ -160,7 +160,7 @@ public void shouldThrowFailedProcessingExceptionWhenProcessingExceptionHandlerTh new ProcessorNode<>(NAME, new IgnoredInternalExceptionsProcessor(), Collections.emptySet()); final InternalProcessorContext internalProcessorContext = mockInternalProcessorContext(); - node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingHandlerResponse.CONTINUE, internalProcessorContext, true)); + node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingExceptionResponse.continueProcessing(), internalProcessorContext, true)); final FailedProcessingException failedProcessingException = assertThrows(FailedProcessingException.class, () -> node.process(new Record<>(KEY, VALUE, TIMESTAMP))); @@ -394,12 +394,12 @@ private InternalProcessorContext mockInternalProcessorContext() } public static class ProcessingExceptionHandlerMock implements ProcessingExceptionHandler { - private final ProcessingExceptionHandler.ProcessingHandlerResponse response; + private final ProcessingExceptionHandler.ProcessingExceptionResponse response; private final InternalProcessorContext internalProcessorContext; private final boolean shouldThrowException; - public ProcessingExceptionHandlerMock(final ProcessingExceptionHandler.ProcessingHandlerResponse response, + public ProcessingExceptionHandlerMock(final ProcessingExceptionHandler.ProcessingExceptionResponse response, final InternalProcessorContext internalProcessorContext, final boolean shouldThrowException) { this.response = response; @@ -408,7 +408,7 @@ public ProcessingExceptionHandlerMock(final ProcessingExceptionHandler.Processin } @Override - public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + public ProcessingExceptionResponse handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { assertEquals(internalProcessorContext.topic(), context.topic()); assertEquals(internalProcessorContext.partition(), context.partition()); assertEquals(internalProcessorContext.offset(), context.offset()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index dca6ad4e84ea1..7c395836c8b64 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -1202,7 +1202,7 @@ public void shouldThrowStreamsExceptionOnSubsequentSendIfFatalEvenWithContinueEx logContext, taskId, getExceptionalStreamsProducerOnSend(exception), - new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandlerResponse.CONTINUE)), + new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.continueProcessing())), streamsMetrics, topology ); @@ -1229,7 +1229,7 @@ public void shouldThrowStreamsExceptionOnSubsequentFlushIfFatalEvenWithContinueE logContext, taskId, getExceptionalStreamsProducerOnSend(exception), - new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandlerResponse.CONTINUE)), + new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.continueProcessing())), streamsMetrics, topology ); @@ -1253,7 +1253,7 @@ public void shouldThrowStreamsExceptionOnSubsequentCloseIfFatalEvenWithContinueE logContext, taskId, getExceptionalStreamsProducerOnSend(exception), - new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandlerResponse.CONTINUE)), + new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.continueProcessing())), streamsMetrics, topology ); @@ -1277,7 +1277,7 @@ public void shouldNotThrowStreamsExceptionOnSubsequentCallIfASendFailsWithContin taskId, getExceptionalStreamsProducerOnSend(new RuntimeException("KABOOM!")), new ProductionExceptionHandlerMock( - Optional.of(ProductionExceptionHandlerResponse.CONTINUE), + Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.continueProcessing()), context, sinkNodeName, taskId @@ -1348,7 +1348,7 @@ public void shouldThrowStreamsExceptionOnUnknownTopicOrPartitionExceptionWhenExc taskId, getExceptionalStreamsProducerOnSend(exception), new ProductionExceptionHandlerMock( - Optional.of(ProductionExceptionHandlerResponse.FAIL), + Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.failProcessing()), context, sinkNodeName, taskId @@ -1378,7 +1378,7 @@ public void shouldNotThrowTaskCorruptedExceptionOnUnknownTopicOrPartitionExcepti taskId, getExceptionalStreamsProducerOnSend(exception), new ProductionExceptionHandlerMock( - Optional.of(ProductionExceptionHandlerResponse.CONTINUE), + Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.continueProcessing()), context, sinkNodeName, taskId @@ -1401,7 +1401,7 @@ public void shouldTreatRetryAsFailForNonRetriableException() { taskId, getExceptionalStreamsProducerOnSend(exception), new ProductionExceptionHandlerMock( - Optional.of(ProductionExceptionHandlerResponse.RETRY), + Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.retryProcessing()), context, sinkNodeName, taskId @@ -1536,7 +1536,7 @@ public void shouldThrowStreamsExceptionUsingDefaultExceptionHandler() { public void shouldDropRecordExceptionUsingAlwaysContinueExceptionHandler() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock( - Optional.of(ProductionExceptionHandlerResponse.CONTINUE), + Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.continueProcessing()), context, sinkNodeName, taskId, @@ -1565,7 +1565,7 @@ public void shouldDropRecordExceptionUsingAlwaysContinueExceptionHandler() { public void shouldThrowStreamsExceptionWhenValueSerializationFailedAndProductionExceptionHandlerRepliesWithFail() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock( - Optional.of(ProductionExceptionHandlerResponse.FAIL), + Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.failProcessing()), context, sinkNodeName, taskId, @@ -1586,7 +1586,7 @@ public void shouldThrowStreamsExceptionWhenValueSerializationFailedAndProduction public void shouldThrowStreamsExceptionWhenKeySerializationFailedAndProductionExceptionHandlerRepliesWithFail() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock( - Optional.of(ProductionExceptionHandlerResponse.FAIL), + Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.failProcessing()), context, sinkNodeName, taskId, @@ -1714,7 +1714,7 @@ public void shouldThrowStreamsExceptionWhenSerializationFailedAndProductionExcep assertEquals("Fatal user code error in production error callback", exception.getMessage()); assertInstanceOf(NullPointerException.class, exception.getCause()); - assertEquals("Invalid ProductionExceptionHandler response.", exception.getCause().getMessage()); + assertEquals("Invalid ProductionExceptionResponse response.", exception.getCause().getMessage()); } } @@ -1763,7 +1763,7 @@ public void shouldThrowStreamsExceptionOnSubsequentFlushIfASendFailsAndProductio final StreamsException thrown = assertThrows(StreamsException.class, collector::flush); assertEquals("Fatal user code error in production error callback", thrown.getMessage()); assertInstanceOf(NullPointerException.class, thrown.getCause()); - assertEquals("Invalid ProductionExceptionHandler response.", thrown.getCause().getMessage()); + assertEquals("Invalid ProductionExceptionResponse response.", thrown.getCause().getMessage()); } @Test @@ -1796,7 +1796,7 @@ public void shouldThrowStreamsExceptionOnSubsequentFlushIfASendFailsAndProductio public void shouldNotCallProductionExceptionHandlerOnClassCastException() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { final RecordCollector collector = newRecordCollector( - new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandlerResponse.CONTINUE)) + new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.continueProcessing())) ); collector.initialize(); @@ -2080,7 +2080,7 @@ public byte[] serialize(final String topic, final Headers headers, final String } public static class ProductionExceptionHandlerMock implements ProductionExceptionHandler { - private final Optional response; + private final Optional response; private boolean shouldThrowException; private InternalProcessorContext expectedContext; private String expectedProcessorNodeId; @@ -2097,11 +2097,11 @@ public ProductionExceptionHandlerMock() { this.expectedSerializationExceptionOrigin = null; } - public ProductionExceptionHandlerMock(final Optional response) { + public ProductionExceptionHandlerMock(final Optional response) { this.response = response; } - public ProductionExceptionHandlerMock(final Optional response, + public ProductionExceptionHandlerMock(final Optional response, final InternalProcessorContext context, final String processorNodeId, final TaskId taskId) { @@ -2121,7 +2121,7 @@ public ProductionExceptionHandlerMock(final boolean shouldThrowException, this.shouldThrowException = shouldThrowException; } - public ProductionExceptionHandlerMock(final Optional response, + public ProductionExceptionHandlerMock(final Optional response, final InternalProcessorContext context, final String processorNodeId, final TaskId taskId, @@ -2132,7 +2132,7 @@ public ProductionExceptionHandlerMock(final Optional record, final Exception exception) { assertInputs(context, exception); @@ -2144,7 +2144,7 @@ public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext conte @SuppressWarnings("rawtypes") @Override - public ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context, + public ProductionExceptionResponse handleSerializationError(final ErrorHandlerContext context, final ProducerRecord record, final Exception exception, final SerializationExceptionOrigin origin) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java index 9847fedb25149..e924f770e995b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java @@ -27,7 +27,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; -import org.apache.kafka.streams.errors.DeserializationExceptionHandler.DeserializationHandlerResponse; import org.apache.kafka.streams.errors.ErrorHandlerContext; import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler; import org.apache.kafka.streams.errors.LogAndFailExceptionHandler; @@ -116,7 +115,7 @@ public void shouldThrowStreamsExceptionWhenDeserializationFailsAndExceptionHandl "value" ), new DeserializationExceptionHandlerMock( - Optional.of(DeserializationHandlerResponse.FAIL), + Optional.of(DeserializationExceptionHandler.DeserializationExceptionResponse.failProcessing()), rawRecord, sourceNodeName, taskId @@ -155,7 +154,7 @@ public void shouldNotThrowStreamsExceptionWhenDeserializationFailsAndExceptionHa "value" ), new DeserializationExceptionHandlerMock( - Optional.of(DeserializationHandlerResponse.CONTINUE), + Optional.of(DeserializationExceptionHandler.DeserializationExceptionResponse.continueProcessing()), rawRecord, sourceNodeName, taskId @@ -196,7 +195,7 @@ public void shouldFailWhenDeserializationFailsAndExceptionHandlerReturnsNull() { ); assertEquals("Fatal user code error in deserialization error callback", exception.getMessage()); assertInstanceOf(NullPointerException.class, exception.getCause()); - assertEquals("Invalid DeserializationExceptionHandler response.", exception.getCause().getMessage()); + assertEquals("Invalid DeserializationExceptionResponse response.", exception.getCause().getMessage()); } } @@ -342,12 +341,12 @@ public Object deserializeValue(final String topic, final Headers headers, final } public static class DeserializationExceptionHandlerMock implements DeserializationExceptionHandler { - private final Optional response; + private final Optional response; private final ConsumerRecord expectedRecord; private final String expectedProcessorNodeId; private final TaskId expectedTaskId; - public DeserializationExceptionHandlerMock(final Optional response, + public DeserializationExceptionHandlerMock(final Optional response, final ConsumerRecord record, final String processorNodeId, final TaskId taskId) { @@ -358,7 +357,7 @@ public DeserializationExceptionHandlerMock(final Optional record, final Exception exception) { assertEquals(expectedRecord.topic(), context.topic()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 701f38eda0c4e..7f416f3409ca1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -3032,7 +3032,7 @@ public void punctuateShouldThrowFailedProcessingExceptionWhenProcessingException public static class CrashingProcessingExceptionHandler implements ProcessingExceptionHandler { @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + public ProcessingExceptionResponse handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { throw new RuntimeException("KABOOM from ProcessingExceptionHandlerMock!"); } @@ -3044,7 +3044,7 @@ public void configure(final Map configs) { public static class NullProcessingExceptionHandler implements ProcessingExceptionHandler { @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + public ProcessingExceptionResponse handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { return null; } From ace6816fc51c55a82172374bf5b505d624e513c8 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Wed, 11 Dec 2024 10:41:18 +0100 Subject: [PATCH 08/16] KAFKA-16505 simplifying default implementations --- .../errors/DeserializationExceptionHandler.java | 5 +---- .../errors/ProcessingExceptionHandler.java | 5 +---- .../errors/ProductionExceptionHandler.java | 16 ++-------------- 3 files changed, 4 insertions(+), 22 deletions(-) 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 370a101f86845..85d5574e40b5a 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 @@ -90,10 +90,7 @@ default DeserializationHandlerResponse handle(final ErrorHandlerContext context, * @return a {@link DeserializationExceptionResponse} object */ default DeserializationExceptionResponse handleError(final ErrorHandlerContext context, final ConsumerRecord record, final Exception exception) { - if (DeserializationHandlerResponse.FAIL == handle(context, record, exception)) { - return DeserializationExceptionResponse.failProcessing(Collections.emptyList()); - } - return DeserializationExceptionResponse.continueProcessing(Collections.emptyList()); + return new DeserializationExceptionResponse(handle(context, record, exception), Collections.emptyList()); } /** * Enumeration that describes the response from the exception handler. 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 index da3055d907758..ecebce158648d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java @@ -59,10 +59,7 @@ default ProcessingHandlerResponse handle(final ErrorHandlerContext context, fina * @return a {@link ProcessingExceptionResponse} object */ default ProcessingExceptionResponse handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { - if (ProcessingHandlerResponse.FAIL == handle(context, record, exception)) { - return ProcessingExceptionResponse.failProcessing(); - } - return ProcessingExceptionResponse.continueProcessing(); + return new ProcessingExceptionResponse(handle(context, record, exception), Collections.emptyList()); } 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 31541c62d9929..4f0fcf579f7d0 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 @@ -83,13 +83,7 @@ default ProductionExceptionHandlerResponse handle(final ErrorHandlerContext cont default ProductionExceptionResponse handleError(final ErrorHandlerContext context, final ProducerRecord record, final Exception exception) { - final ProductionExceptionHandlerResponse response = handle(context, record, exception); - if (ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL == response) { - return ProductionExceptionResponse.failProcessing(); - } else if (ProductionExceptionHandler.ProductionExceptionHandlerResponse.RETRY == response) { - return ProductionExceptionResponse.retryProcessing(); - } - return ProductionExceptionResponse.continueProcessing(); + return new ProductionExceptionResponse(handle(context, record, exception), Collections.emptyList()); } /** @@ -158,13 +152,7 @@ default ProductionExceptionResponse handleSerializationError(final ErrorHandlerC final ProducerRecord record, final Exception exception, final SerializationExceptionOrigin origin) { - final ProductionExceptionHandlerResponse response = handleSerializationException(context, record, exception, origin); - if (ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL == response) { - return ProductionExceptionResponse.failProcessing(); - } else if (ProductionExceptionHandler.ProductionExceptionHandlerResponse.RETRY == response) { - return ProductionExceptionResponse.retryProcessing(); - } - return ProductionExceptionResponse.continueProcessing(); + return new ProductionExceptionResponse(handleSerializationException(context, record, exception, origin), Collections.emptyList()); } enum ProductionExceptionHandlerResponse { From 4546d9162fb7c8fed999e1712a89f80464ec2330 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Mon, 16 Dec 2024 11:36:07 +0100 Subject: [PATCH 09/16] KAFKA-16505 deprecated old handler response enums --- ...essingExceptionHandlerIntegrationTest.java | 8 +- ...allowUnknownTopicErrorIntegrationTest.java | 8 +- .../DefaultProductionExceptionHandler.java | 20 +-- .../DeserializationExceptionHandler.java | 105 ++++++++---- .../LogAndContinueExceptionHandler.java | 8 +- ...AndContinueProcessingExceptionHandler.java | 9 +- .../errors/LogAndFailExceptionHandler.java | 8 +- .../LogAndFailProcessingExceptionHandler.java | 8 +- .../errors/ProcessingExceptionHandler.java | 106 +++++++++---- .../errors/ProductionExceptionHandler.java | 150 +++++++++++++----- .../processor/internals/ProcessorNode.java | 4 +- .../internals/RecordCollectorImpl.java | 18 +-- .../internals/RecordDeserializer.java | 7 +- .../processor/internals/StreamTask.java | 4 +- .../internals/ProcessorNodeTest.java | 12 +- .../internals/RecordCollectorTest.java | 45 +++--- .../internals/RecordDeserializerTest.java | 14 +- .../processor/internals/StreamTaskTest.java | 4 +- 18 files changed, 345 insertions(+), 193 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java index cd212b9f623e5..9a83babde6214 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java @@ -524,7 +524,7 @@ public void shouldVerifySourceRawKeyAndSourceRawValuePresentOrNotInErrorHandlerC public static class ContinueProcessingExceptionHandlerMockTest implements ProcessingExceptionHandler { @Override - public ProcessingExceptionResponse handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { + public Response handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { if (((String) record.key()).contains("FATAL")) { throw new RuntimeException("KABOOM!"); } @@ -532,7 +532,7 @@ public ProcessingExceptionResponse handleError(final ErrorHandlerContext context return null; } assertProcessingExceptionHandlerInputs(context, record, exception); - return ProcessingExceptionResponse.continueProcessing(); + return Response.resume(); } @Override @@ -543,9 +543,9 @@ public void configure(final Map configs) { public static class FailProcessingExceptionHandlerMockTest implements ProcessingExceptionHandler { @Override - public ProcessingExceptionResponse handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { + public Response handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { assertProcessingExceptionHandlerInputs(context, record, exception); - return ProcessingExceptionResponse.failProcessing(); + return Response.fail(); } @Override diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SwallowUnknownTopicErrorIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SwallowUnknownTopicErrorIntegrationTest.java index 853ce9c7cc332..0535dd2465e4d 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SwallowUnknownTopicErrorIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/SwallowUnknownTopicErrorIntegrationTest.java @@ -156,13 +156,13 @@ public TestHandler() { } public void configure(final Map configs) { } @Override - public ProductionExceptionResponse handleError(final ErrorHandlerContext context, - final ProducerRecord record, - final Exception exception) { + public Response handleError(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception) { if (exception instanceof TimeoutException && exception.getCause() != null && exception.getCause() instanceof UnknownTopicOrPartitionException) { - return ProductionExceptionResponse.continueProcessing(); + return Response.resume(); } return ProductionExceptionHandler.super.handleError(context, record, exception); } 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 97fd23e970fcc..04529291b141d 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 @@ -57,12 +57,12 @@ public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext conte } @Override - public ProductionExceptionResponse handleError(final ErrorHandlerContext context, - final ProducerRecord record, - final Exception exception) { + public Response handleError(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception) { return exception instanceof RetriableException ? - ProductionExceptionResponse.retryProcessing() : - ProductionExceptionResponse.failProcessing(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + Response.retry() : + Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } @@ -77,11 +77,11 @@ public ProductionExceptionHandlerResponse handleSerializationException(final Err } @Override - public ProductionExceptionResponse handleSerializationError(final ErrorHandlerContext context, - final ProducerRecord record, - final Exception exception, - final SerializationExceptionOrigin origin) { - return ProductionExceptionResponse.failProcessing(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + public Response handleSerializationError(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception, + final SerializationExceptionOrigin origin) { + return Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } 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 85d5574e40b5a..64e1255443268 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 @@ -87,14 +87,15 @@ default DeserializationHandlerResponse handle(final ErrorHandlerContext context, * @param exception * The actual exception. * - * @return a {@link DeserializationExceptionResponse} object + * @return a {@link Response} object */ - default DeserializationExceptionResponse handleError(final ErrorHandlerContext context, final ConsumerRecord record, final Exception exception) { - return new DeserializationExceptionResponse(handle(context, record, exception), Collections.emptyList()); + default Response handleError(final ErrorHandlerContext context, final ConsumerRecord record, final Exception exception) { + return new Response(Result.from(handle(context, record, exception)), Collections.emptyList()); } /** * Enumeration that describes the response from the exception handler. */ + @Deprecated enum DeserializationHandlerResponse { /** Continue processing. */ CONTINUE(0, "CONTINUE"), @@ -117,78 +118,122 @@ enum DeserializationHandlerResponse { } } + /** + * Enumeration that describes the response from the exception handler. + */ + enum Result { + /** Continue processing. */ + RESUME(0, "RESUME"), + /** Fail processing. */ + FAIL(1, "FAIL"); + + /** + * An english description for the used option. This is for debugging only and may change. + */ + public final String name; + + /** + * The permanent and immutable id for the used option. This can't change ever. + */ + public final int id; + + Result(final int id, final String name) { + this.id = id; + this.name = name; + } + + /** + * Converts the deprecated enum DeserializationHandlerResponse into the new Result enum. + * + * @param value the old DeserializationHandlerResponse enum value + * @return a {@link Result} enum value + * @throws IllegalArgumentException if the provided value does not map to a valid {@link Result} + */ + @Deprecated + public static DeserializationExceptionHandler.Result from(final DeserializationHandlerResponse value) { + switch (value) { + case FAIL: + return Result.FAIL; + case CONTINUE: + return Result.RESUME; + default: + throw new IllegalArgumentException("No Result enum found for old value: " + value); + } + } + } + /** * Represents the result of handling a deserialization exception. *

- * The {@code Response} class encapsulates a {@link ProcessingExceptionHandler.ProcessingHandlerResponse}, + * The {@code Response} class encapsulates a {@link ProcessingExceptionHandler.Result}, * indicating whether processing should continue or fail, along with an optional list of * {@link ProducerRecord} instances to be sent to a dead letter queue. *

*/ - class DeserializationExceptionResponse { + class Response { - private DeserializationHandlerResponse deserializationHandlerResponse; + private Result result; private List> deadLetterQueueRecords; /** * Constructs a new {@code DeserializationExceptionResponse} object. * - * @param deserializationHandlerResponse the response indicating whether processing should continue or fail; + * @param result the result indicating whether processing should continue or fail; * must not be {@code null}. * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. */ - private DeserializationExceptionResponse(final DeserializationHandlerResponse deserializationHandlerResponse, - final List> deadLetterQueueRecords) { - this.deserializationHandlerResponse = deserializationHandlerResponse; + private Response(final Result result, + final List> deadLetterQueueRecords) { + this.result = result; this.deadLetterQueueRecords = deadLetterQueueRecords; } /** - * Creates a {@code DeserializationExceptionResponse} indicating that processing should fail. + * Creates a {@code Response} indicating that processing should fail. * * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. - * @return a {@code DeserializationExceptionResponse} with a {@link DeserializationHandlerResponse#FAIL} status. + * @return a {@code Response} with a {@link DeserializationExceptionHandler.Result#FAIL} status. */ - public static DeserializationExceptionResponse failProcessing(final List> deadLetterQueueRecords) { - return new DeserializationExceptionResponse(DeserializationHandlerResponse.FAIL, deadLetterQueueRecords); + public static Response fail(final List> deadLetterQueueRecords) { + return new Response(Result.FAIL, deadLetterQueueRecords); } /** - * Creates a {@code DeserializationExceptionResponse} indicating that processing should fail. + * Creates a {@code Response} indicating that processing should fail. * - * @return a {@code DeserializationExceptionResponse} with a {@link DeserializationHandlerResponse#FAIL} status. + * @return a {@code Response} with a {@link DeserializationExceptionHandler.Result#FAIL} status. */ - public static DeserializationExceptionResponse failProcessing() { - return failProcessing(Collections.emptyList()); + public static Response fail() { + return fail(Collections.emptyList()); } /** - * Creates a {@code DeserializationExceptionResponse} indicating that processing should continue. + * Creates a {@code Response} indicating that processing should continue. * * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. - * @return a {@code DeserializationExceptionResponse} with a {@link DeserializationHandlerResponse#CONTINUE} status. + * @return a {@code Response} with a {@link DeserializationExceptionHandler.Result#RESUME} status. */ - public static DeserializationExceptionResponse continueProcessing(final List> deadLetterQueueRecords) { - return new DeserializationExceptionResponse(DeserializationHandlerResponse.CONTINUE, deadLetterQueueRecords); + public static Response resume(final List> deadLetterQueueRecords) { + return new Response(Result.RESUME, deadLetterQueueRecords); } /** - * Creates a {@code DeserializationExceptionResponse} indicating that processing should continue. + * Creates a {@code Response} indicating that processing should continue. * - * @return a {@code DeserializationExceptionResponse} with a {@link DeserializationHandlerResponse#CONTINUE} status. + * @return a {@code Response} with a {@link DeserializationHandlerResponse#CONTINUE} status. */ - public static DeserializationExceptionResponse continueProcessing() { - return continueProcessing(Collections.emptyList()); + public static Response resume() { + return resume(Collections.emptyList()); } /** - * Retrieves the deserialization handler response. + * Retrieves the deserialization handler result. * - * @return the {@link DeserializationHandlerResponse} indicating whether processing should continue or fail. + * @return the {@link Result} indicating whether processing should continue or fail. */ - public DeserializationHandlerResponse response() { - return deserializationHandlerResponse; + public Result result() { + return result; } /** 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 c67c773ac4bd8..34e7615dbc58a 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 @@ -77,9 +77,9 @@ public DeserializationHandlerResponse handle(final ErrorHandlerContext context, } @Override - public DeserializationExceptionResponse handleError(final ErrorHandlerContext context, - final ConsumerRecord record, - final Exception exception) { + public Response handleError(final ErrorHandlerContext context, + final ConsumerRecord record, + final Exception exception) { log.warn( "Exception caught during Deserialization, taskId: {}, topic: {}, partition: {}, offset: {}", context.taskId(), @@ -89,7 +89,7 @@ public DeserializationExceptionResponse handleError(final ErrorHandlerContext co exception ); - return DeserializationExceptionResponse.continueProcessing(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + return Response.resume(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java index 9057731a870ad..671bc89bcab4d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java @@ -53,9 +53,9 @@ public ProcessingHandlerResponse handle(final ErrorHandlerContext context, } @Override - public ProcessingExceptionResponse handleError(final ErrorHandlerContext context, - final Record record, - final Exception exception) { + public Response handleError(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(), @@ -65,10 +65,9 @@ public ProcessingExceptionResponse handleError(final ErrorHandlerContext context context.offset(), exception ); - return ProcessingExceptionResponse.continueProcessing(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + return Response.resume(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } - @Override public void configure(final Map configs) { if (configs.get(StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG) != null) 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 c06ccfba015b8..5b3d5ca123bed 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 @@ -77,9 +77,9 @@ public DeserializationHandlerResponse handle(final ErrorHandlerContext context, } @Override - public DeserializationExceptionResponse handleError(final ErrorHandlerContext context, - final ConsumerRecord record, - final Exception exception) { + public Response handleError(final ErrorHandlerContext context, + final ConsumerRecord record, + final Exception exception) { log.warn( "Exception caught during Deserialization, taskId: {}, topic: {}, partition: {}, offset: {}", context.taskId(), @@ -89,7 +89,7 @@ public DeserializationExceptionResponse handleError(final ErrorHandlerContext co exception ); - return DeserializationExceptionResponse.failProcessing(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + return Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java index 393e30b8e7a81..eb35021f8f86b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java @@ -52,9 +52,9 @@ public ProcessingHandlerResponse handle(final ErrorHandlerContext context, } @Override - public ProcessingExceptionResponse handleError(final ErrorHandlerContext context, - final Record record, - final Exception exception) { + public Response handleError(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(), @@ -65,7 +65,7 @@ public ProcessingExceptionResponse handleError(final ErrorHandlerContext context exception ); - return ProcessingExceptionResponse.failProcessing(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + return Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } @Override 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 index ecebce158648d..ce6e1e31e6c04 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java @@ -56,13 +56,13 @@ default ProcessingHandlerResponse handle(final ErrorHandlerContext context, fina * @param exception * The actual exception. * - * @return a {@link ProcessingExceptionResponse} object + * @return a {@link Response} object */ - default ProcessingExceptionResponse handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { - return new ProcessingExceptionResponse(handle(context, record, exception), Collections.emptyList()); + default Response handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { + return new Response(ProcessingExceptionHandler.Result.from(handle(context, record, exception)), Collections.emptyList()); } - + @Deprecated enum ProcessingHandlerResponse { /** Continue processing. */ CONTINUE(1, "CONTINUE"), @@ -85,78 +85,122 @@ enum ProcessingHandlerResponse { } } + /** + * Enumeration that describes the response from the exception handler. + */ + enum Result { + /** Resume processing. */ + RESUME(1, "RESUME"), + /** Fail processing. */ + FAIL(2, "FAIL"); + + /** + * An english description for the used option. This is for debugging only and may change. + */ + public final String name; + + /** + * The permanent and immutable id for the used option. This can't change ever. + */ + public final int id; + + Result(final int id, final String name) { + this.id = id; + this.name = name; + } + + /** + * Converts the deprecated enum ProcessingHandlerResponse into the new Result enum. + * + * @param value the old DeserializationHandlerResponse enum value + * @return a {@link ProcessingExceptionHandler.Result} enum value + * @throws IllegalArgumentException if the provided value does not map to a valid {@link ProcessingExceptionHandler.Result} + */ + @Deprecated + public static ProcessingExceptionHandler.Result from(final ProcessingHandlerResponse value) { + switch (value) { + case FAIL: + return Result.FAIL; + case CONTINUE: + return Result.RESUME; + default: + throw new IllegalArgumentException("No Result enum found for old value: " + value); + } + } + } + /** * Represents the result of handling a processing exception. *

- * The {@code Response} class encapsulates a {@link ProcessingHandlerResponse}, + * The {@code Response} class encapsulates a {@link Result}, * indicating whether processing should continue or fail, along with an optional list of * {@link org.apache.kafka.clients.producer.ProducerRecord} instances to be sent to a dead letter queue. *

*/ - class ProcessingExceptionResponse { + class Response { - private ProcessingHandlerResponse processingHandlerResponse; + private Result result; private List> deadLetterQueueRecords; /** * Constructs a new {@code ProcessingExceptionResponse} object. * - * @param processingHandlerResponse the response indicating whether processing should continue or fail; + * @param result the result indicating whether processing should continue or fail; * must not be {@code null}. * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. */ - private ProcessingExceptionResponse(final ProcessingHandlerResponse processingHandlerResponse, - final List> deadLetterQueueRecords) { - this.processingHandlerResponse = processingHandlerResponse; + private Response(final Result result, + final List> deadLetterQueueRecords) { + this.result = result; this.deadLetterQueueRecords = deadLetterQueueRecords; } /** - * Creates a {@code ProcessingExceptionResponse} indicating that processing should fail. + * Creates a {@code Response} indicating that processing should fail. * * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. - * @return a {@code ProcessingExceptionResponse} with a {@link ProcessingHandlerResponse#FAIL} status. + * @return a {@code Response} with a {@link ProcessingExceptionHandler.Result#FAIL} status. */ - public static ProcessingExceptionResponse failProcessing(final List> deadLetterQueueRecords) { - return new ProcessingExceptionResponse(ProcessingHandlerResponse.FAIL, deadLetterQueueRecords); + public static Response fail(final List> deadLetterQueueRecords) { + return new Response(Result.FAIL, deadLetterQueueRecords); } /** - * Creates a {@code ProcessingExceptionResponse} indicating that processing should fail. + * Creates a {@code Response} indicating that processing should fail. * - * @return a {@code ProcessingExceptionResponse} with a {@link ProcessingHandlerResponse#FAIL} status. + * @return a {@code Response} with a {@link ProcessingExceptionHandler.Result#FAIL} status. */ - public static ProcessingExceptionResponse failProcessing() { - return failProcessing(Collections.emptyList()); + public static Response fail() { + return fail(Collections.emptyList()); } /** - * Creates a {@code ProcessingExceptionResponse} indicating that processing should continue. + * Creates a {@code Response} indicating that processing should continue. * * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. - * @return a {@code Response} with a {@link ProcessingHandlerResponse#CONTINUE} status. + * @return a {@code Response} with a {@link ProcessingExceptionHandler.Result#RESUME} status. */ - public static ProcessingExceptionResponse continueProcessing(final List> deadLetterQueueRecords) { - return new ProcessingExceptionResponse(ProcessingHandlerResponse.CONTINUE, deadLetterQueueRecords); + public static Response resume(final List> deadLetterQueueRecords) { + return new Response(Result.RESUME, deadLetterQueueRecords); } /** - * Creates a {@code ProcessingExceptionResponse} indicating that processing should continue. + * Creates a {@code Response} indicating that processing should continue. * - * @return a {@code ProcessingExceptionResponse} with a {@link ProcessingHandlerResponse#CONTINUE} status. + * @return a {@code Response} with a {@link ProcessingExceptionHandler.Result#RESUME} status. */ - public static ProcessingExceptionResponse continueProcessing() { - return continueProcessing(Collections.emptyList()); + public static Response resume() { + return resume(Collections.emptyList()); } /** - * Retrieves the processing handler response. + * Retrieves the processing handler result. * - * @return the {@link ProcessingHandlerResponse} indicating whether processing should continue or fail. + * @return the {@link Result} indicating whether processing should continue or fail. */ - public ProcessingHandlerResponse response() { - return processingHandlerResponse; + public Result result() { + return result; } /** 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 4f0fcf579f7d0..0493e029bece0 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 @@ -78,12 +78,12 @@ default ProductionExceptionHandlerResponse handle(final ErrorHandlerContext cont * @param exception * The exception that occurred during production. * - * @return a {@link ProductionExceptionResponse} object + * @return a {@link Response} object */ - default ProductionExceptionResponse handleError(final ErrorHandlerContext context, - final ProducerRecord record, - final Exception exception) { - return new ProductionExceptionResponse(handle(context, record, exception), Collections.emptyList()); + default Response handleError(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception) { + return new Response(Result.from(handle(context, record, exception)), Collections.emptyList()); } /** @@ -145,16 +145,17 @@ default ProductionExceptionHandlerResponse handleSerializationException(final Er * @param origin * The origin of the serialization exception. * - * @return a {@link ProductionExceptionResponse} object + * @return a {@link Response} object */ @SuppressWarnings("rawtypes") - default ProductionExceptionResponse handleSerializationError(final ErrorHandlerContext context, - final ProducerRecord record, - final Exception exception, - final SerializationExceptionOrigin origin) { - return new ProductionExceptionResponse(handleSerializationException(context, record, exception, origin), Collections.emptyList()); + default Response handleSerializationError(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception, + final SerializationExceptionOrigin origin) { + return new Response(Result.from(handleSerializationException(context, record, exception, origin)), Collections.emptyList()); } + @Deprecated enum ProductionExceptionHandlerResponse { /** Continue processing. * @@ -197,6 +198,71 @@ enum ProductionExceptionHandlerResponse { } } + /** + * Enumeration that describes the response from the exception handler. + */ + enum Result { + /** Resume processing. + * + *

For this case, output records which could not be written successfully are lost. + * Use this option only if you can tolerate data loss. + */ + RESUME(0, "RESUME"), + /** Fail processing. + * + *

Kafka Streams will raise an exception and the {@code StreamsThread} will fail. + * No offsets (for {@link org.apache.kafka.streams.StreamsConfig#AT_LEAST_ONCE at-least-once}) or transactions + * (for {@link org.apache.kafka.streams.StreamsConfig#EXACTLY_ONCE_V2 exactly-once}) will be committed. + */ + FAIL(1, "FAIL"), + /** Retry the failed operation. + * + *

Retrying might imply that a {@link TaskCorruptedException} exception is thrown, and that the retry + * is started from the last committed offset. + * + *

NOTE: {@code RETRY} is only a valid return value for + * {@link org.apache.kafka.common.errors.RetriableException retriable exceptions}. + * If {@code RETRY} is returned for a non-retriable exception it will be interpreted as {@link #FAIL}. + */ + RETRY(2, "RETRY"); + + /** + * An english description for the used option. This is for debugging only and may change. + */ + public final String name; + + /** + * The permanent and immutable id for the used option. This can't change ever. + */ + public final int id; + + Result(final int id, final String name) { + this.id = id; + this.name = name; + } + + /** + * Converts the deprecated enum ProductionExceptionHandlerResponse into the new Result enum. + * + * @param value the old ProductionExceptionHandlerResponse enum value + * @return a {@link ProductionExceptionHandler.Result} enum value + * @throws IllegalArgumentException if the provided value does not map to a valid {@link ProductionExceptionHandler.Result} + */ + @Deprecated + public static ProductionExceptionHandler.Result from(final ProductionExceptionHandlerResponse value) { + switch (value) { + case FAIL: + return Result.FAIL; + case CONTINUE: + return Result.RESUME; + case RETRY: + return Result.RETRY; + default: + throw new IllegalArgumentException("No Result enum found for old value: " + value); + } + } + } + enum SerializationExceptionOrigin { /** Serialization exception occurred during serialization of the key. */ KEY, @@ -212,79 +278,79 @@ enum SerializationExceptionOrigin { * {@link ProducerRecord} instances to be sent to a dead letter queue. *

*/ - class ProductionExceptionResponse { + class Response { - private ProductionExceptionHandlerResponse productionExceptionHandlerResponse; + private Result result; private List> deadLetterQueueRecords; /** - * Constructs a new {@code ProductionExceptionResponse} object. + * Constructs a new {@code Response} object. * - * @param productionExceptionHandlerResponse the response indicating whether processing should continue or fail; + * @param result the result indicating whether processing should continue or fail; * must not be {@code null}. * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. */ - private ProductionExceptionResponse(final ProductionExceptionHandlerResponse productionExceptionHandlerResponse, - final List> deadLetterQueueRecords) { - this.productionExceptionHandlerResponse = productionExceptionHandlerResponse; + private Response(final Result result, + final List> deadLetterQueueRecords) { + this.result = result; this.deadLetterQueueRecords = deadLetterQueueRecords; } /** - * Creates a {@code ProductionExceptionResponse} indicating that processing should fail. + * Creates a {@code Response} indicating that processing should fail. * * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. - * @return a {@code ProductionExceptionResponse} with a {@link DeserializationExceptionHandler.DeserializationHandlerResponse#FAIL} status. + * @return a {@code Response} with a {@link ProductionExceptionHandler.Result#FAIL} status. */ - public static ProductionExceptionResponse failProcessing(final List> deadLetterQueueRecords) { - return new ProductionExceptionResponse(ProductionExceptionHandlerResponse.FAIL, deadLetterQueueRecords); + public static Response fail(final List> deadLetterQueueRecords) { + return new Response(Result.FAIL, deadLetterQueueRecords); } /** - * Creates a {@code ProductionExceptionResponse} indicating that processing should fail. + * Creates a {@code Response} indicating that processing should fail. * - * @return a {@code ProductionExceptionResponse} with a {@link DeserializationExceptionHandler.DeserializationHandlerResponse#FAIL} status. + * @return a {@code Response} with a {@link ProductionExceptionHandler.Result#FAIL} status. */ - public static ProductionExceptionResponse failProcessing() { - return failProcessing(Collections.emptyList()); + public static Response fail() { + return fail(Collections.emptyList()); } /** - * Creates a {@code ProductionExceptionResponse} indicating that processing should continue. + * Creates a {@code Response} indicating that processing should continue. * * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. - * @return a {@code ProductionExceptionResponse} with a {@link DeserializationExceptionHandler.DeserializationHandlerResponse#CONTINUE} status. + * @return a {@code Response} with a {@link ProductionExceptionHandler.Result#RESUME} status. */ - public static ProductionExceptionResponse continueProcessing(final List> deadLetterQueueRecords) { - return new ProductionExceptionResponse(ProductionExceptionHandlerResponse.CONTINUE, deadLetterQueueRecords); + public static Response resume(final List> deadLetterQueueRecords) { + return new Response(Result.RESUME, deadLetterQueueRecords); } /** - * Creates a {@code ProductionExceptionResponse} indicating that processing should continue. + * Creates a {@code Response} indicating that processing should continue. * - * @return a {@code ProductionExceptionResponse} with a {@link DeserializationExceptionHandler.DeserializationHandlerResponse#CONTINUE} status. + * @return a {@code Response} with a {@link ProductionExceptionHandler.Result#RESUME} status. */ - public static ProductionExceptionResponse continueProcessing() { - return continueProcessing(Collections.emptyList()); + public static Response resume() { + return resume(Collections.emptyList()); } /** - * Creates a {@code ProductionExceptionResponse} indicating that processing should retry. + * Creates a {@code Response} indicating that processing should retry. * - * @return a {@code ProductionExceptionResponse} with a {@link DeserializationExceptionHandler.DeserializationHandlerResponse#CONTINUE} status. + * @return a {@code Response} with a {@link ProductionExceptionHandler.Result#RETRY} status. */ - public static ProductionExceptionResponse retryProcessing() { - return new ProductionExceptionResponse(ProductionExceptionHandlerResponse.RETRY, Collections.emptyList()); + public static Response retry() { + return new Response(Result.RETRY, Collections.emptyList()); } /** - * Retrieves the production exception handler response. + * Retrieves the production exception handler result. * - * @return the {@link ProductionExceptionHandlerResponse} indicating whether processing should continue or fail. + * @return the {@link Result} indicating whether processing should continue, fail or retry. */ - public ProductionExceptionHandlerResponse response() { - return productionExceptionHandlerResponse; + public Result result() { + return result; } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index 694f0c6e008e3..aea1e272a59cf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -221,7 +221,7 @@ public void process(final Record record) { internalProcessorContext.recordContext().sourceRawValue() ); - final ProcessingExceptionHandler.ProcessingExceptionResponse processingExceptionResponse; + final ProcessingExceptionHandler.Response processingExceptionResponse; try { processingExceptionResponse = Objects.requireNonNull( processingExceptionHandler.handleError(errorHandlerContext, record, processingException), @@ -257,7 +257,7 @@ public void process(final Record record) { } } - if (processingExceptionResponse.response() == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { + if (processingExceptionResponse.result() == ProcessingExceptionHandler.Result.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 " + diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index b46d32401f940..118c27004933d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -343,7 +343,7 @@ private void handleException(final ProductionExceptionHandler.Serializati final ProducerRecord record = new ProducerRecord<>(topic, partition, timestamp, key, value, headers); - final ProductionExceptionHandler.ProductionExceptionResponse response; + final ProductionExceptionHandler.Response response; try { response = Objects.requireNonNull( productionExceptionHandler.handleSerializationError( @@ -386,7 +386,7 @@ private void handleException(final ProductionExceptionHandler.Serializati } } - if (maybeFailResponse(response.response()) == ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL) { + if (maybeFailResponse(response.result()) == ProductionExceptionHandler.Result.FAIL) { throw new StreamsException( String.format( "Unable to serialize record. ProducerRecord(topic=[%s], partition=[%d], timestamp=[%d]", @@ -483,7 +483,7 @@ private void recordSendError(final String topic, // TransactionAbortedException is only thrown after `abortTransaction()` was called, // so it's only a followup error, and Kafka Streams is already handling the original error } else { - final ProductionExceptionHandler.ProductionExceptionResponse response; + final ProductionExceptionHandler.Response response; try { response = Objects.requireNonNull( productionExceptionHandler.handleError( @@ -524,14 +524,14 @@ private void recordSendError(final String topic, } } - if (productionException instanceof RetriableException && response.response() == ProductionExceptionHandler.ProductionExceptionHandlerResponse.RETRY) { + if (productionException instanceof RetriableException && response.result() == ProductionExceptionHandler.Result.RETRY) { errorMessage += "\nThe broker is either slow or in bad state (like not having enough replicas) in responding the request, " + "or the connection to broker was interrupted sending the request or receiving the response. " + "\nConsider overwriting `max.block.ms` and /or " + "`delivery.timeout.ms` to a larger value to wait longer for such scenarios and avoid timeout errors"; sendException.set(new TaskCorruptedException(Collections.singleton(taskId))); } else { - if (maybeFailResponse(response.response()) == ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL) { + if (maybeFailResponse(response.result()) == ProductionExceptionHandler.Result.FAIL) { errorMessage += "\nException handler choose to FAIL the processing, no more records would be sent."; sendException.set(new StreamsException(errorMessage, productionException)); } else { @@ -544,12 +544,12 @@ private void recordSendError(final String topic, log.error(errorMessage, productionException); } - private ProductionExceptionHandler.ProductionExceptionHandlerResponse maybeFailResponse(final ProductionExceptionHandler.ProductionExceptionHandlerResponse response) { - if (response == ProductionExceptionHandler.ProductionExceptionHandlerResponse.RETRY) { + private ProductionExceptionHandler.Result maybeFailResponse(final ProductionExceptionHandler.Result result) { + if (result == ProductionExceptionHandler.Result.RETRY) { log.warn("ProductionExceptionHandler returned RETRY for a non-retriable exception. Will treat it as FAIL."); - return ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL; + return ProductionExceptionHandler.Result.FAIL; } else { - return response; + return result; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index b298934ffec4a..88e756d785a2d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; -import org.apache.kafka.streams.errors.DeserializationExceptionHandler.DeserializationHandlerResponse; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; import org.apache.kafka.streams.processor.api.ProcessorContext; @@ -52,7 +51,7 @@ public class RecordDeserializer { /** * @throws StreamsException if a deserialization error occurs and the deserialization callback returns - * {@link DeserializationHandlerResponse#FAIL FAIL} + * {@link DeserializationExceptionHandler.Result#FAIL FAIL} * or throws an exception itself */ ConsumerRecord deserialize(final ProcessorContext processorContext, @@ -102,7 +101,7 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa rawRecord.value() ); - final DeserializationExceptionHandler.DeserializationExceptionResponse response; + final DeserializationExceptionHandler.Response response; try { response = Objects.requireNonNull( deserializationExceptionHandler.handleError(errorHandlerContext, rawRecord, deserializationException), @@ -134,7 +133,7 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa } } - if (response.response() == DeserializationHandlerResponse.FAIL) { + if (response.result() == DeserializationExceptionHandler.Result.FAIL) { throw new StreamsException("Deserialization exception handler is set to fail upon" + " a deserialization error. If you would rather have the streaming pipeline" + " continue after a deserialization error, please set the " + diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 510acef221675..42b57e46aa4f3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -946,7 +946,7 @@ record = null; recordContext.sourceRawValue() ); - final ProcessingExceptionHandler.ProcessingExceptionResponse processingExceptionResponse; + final ProcessingExceptionHandler.Response processingExceptionResponse; try { processingExceptionResponse = Objects.requireNonNull( processingExceptionHandler.handleError(errorHandlerContext, null, processingException), @@ -977,7 +977,7 @@ record = null; } } - if (processingExceptionResponse.response() == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { + if (processingExceptionResponse.result() == ProcessingExceptionHandler.Result.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 " + diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index f13b9c46c449c..5261d3989df4f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -107,7 +107,7 @@ public void shouldThrowFailedProcessingExceptionWhenProcessingExceptionHandlerRe new ProcessorNode<>(NAME, new IgnoredInternalExceptionsProcessor(), Collections.emptySet()); final InternalProcessorContext internalProcessorContext = mockInternalProcessorContext(); - node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingExceptionResponse.failProcessing(), internalProcessorContext, false)); + node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.Response.fail(), internalProcessorContext, false)); final FailedProcessingException failedProcessingException = assertThrows(FailedProcessingException.class, () -> node.process(new Record<>(KEY, VALUE, TIMESTAMP))); @@ -124,7 +124,7 @@ public void shouldNotThrowFailedProcessingExceptionWhenProcessingExceptionHandle new ProcessorNode<>(NAME, new IgnoredInternalExceptionsProcessor(), Collections.emptySet()); final InternalProcessorContext internalProcessorContext = mockInternalProcessorContext(); - node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingExceptionResponse.continueProcessing(), internalProcessorContext, false)); + node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.Response.resume(), internalProcessorContext, false)); assertDoesNotThrow(() -> node.process(new Record<>(KEY, VALUE, TIMESTAMP))); } @@ -160,7 +160,7 @@ public void shouldThrowFailedProcessingExceptionWhenProcessingExceptionHandlerTh new ProcessorNode<>(NAME, new IgnoredInternalExceptionsProcessor(), Collections.emptySet()); final InternalProcessorContext internalProcessorContext = mockInternalProcessorContext(); - node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingExceptionResponse.continueProcessing(), internalProcessorContext, true)); + node.init(internalProcessorContext, new ProcessingExceptionHandlerMock(ProcessingExceptionHandler.Response.resume(), internalProcessorContext, true)); final FailedProcessingException failedProcessingException = assertThrows(FailedProcessingException.class, () -> node.process(new Record<>(KEY, VALUE, TIMESTAMP))); @@ -394,12 +394,12 @@ private InternalProcessorContext mockInternalProcessorContext() } public static class ProcessingExceptionHandlerMock implements ProcessingExceptionHandler { - private final ProcessingExceptionHandler.ProcessingExceptionResponse response; + private final Response response; private final InternalProcessorContext internalProcessorContext; private final boolean shouldThrowException; - public ProcessingExceptionHandlerMock(final ProcessingExceptionHandler.ProcessingExceptionResponse response, + public ProcessingExceptionHandlerMock(final Response response, final InternalProcessorContext internalProcessorContext, final boolean shouldThrowException) { this.response = response; @@ -408,7 +408,7 @@ public ProcessingExceptionHandlerMock(final ProcessingExceptionHandler.Processin } @Override - public ProcessingExceptionResponse handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { + public Response handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { assertEquals(internalProcessorContext.topic(), context.topic()); assertEquals(internalProcessorContext.partition(), context.partition()); assertEquals(internalProcessorContext.offset(), context.offset()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index 7c395836c8b64..69d21bae40b2e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -51,7 +51,6 @@ import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; import org.apache.kafka.streams.errors.ErrorHandlerContext; import org.apache.kafka.streams.errors.ProductionExceptionHandler; -import org.apache.kafka.streams.errors.ProductionExceptionHandler.ProductionExceptionHandlerResponse; import org.apache.kafka.streams.errors.ProductionExceptionHandler.SerializationExceptionOrigin; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; @@ -1202,7 +1201,7 @@ public void shouldThrowStreamsExceptionOnSubsequentSendIfFatalEvenWithContinueEx logContext, taskId, getExceptionalStreamsProducerOnSend(exception), - new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.continueProcessing())), + new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandler.Response.resume())), streamsMetrics, topology ); @@ -1229,7 +1228,7 @@ public void shouldThrowStreamsExceptionOnSubsequentFlushIfFatalEvenWithContinueE logContext, taskId, getExceptionalStreamsProducerOnSend(exception), - new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.continueProcessing())), + new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandler.Response.resume())), streamsMetrics, topology ); @@ -1253,7 +1252,7 @@ public void shouldThrowStreamsExceptionOnSubsequentCloseIfFatalEvenWithContinueE logContext, taskId, getExceptionalStreamsProducerOnSend(exception), - new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.continueProcessing())), + new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandler.Response.resume())), streamsMetrics, topology ); @@ -1277,7 +1276,7 @@ public void shouldNotThrowStreamsExceptionOnSubsequentCallIfASendFailsWithContin taskId, getExceptionalStreamsProducerOnSend(new RuntimeException("KABOOM!")), new ProductionExceptionHandlerMock( - Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.continueProcessing()), + Optional.of(ProductionExceptionHandler.Response.resume()), context, sinkNodeName, taskId @@ -1348,7 +1347,7 @@ public void shouldThrowStreamsExceptionOnUnknownTopicOrPartitionExceptionWhenExc taskId, getExceptionalStreamsProducerOnSend(exception), new ProductionExceptionHandlerMock( - Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.failProcessing()), + Optional.of(ProductionExceptionHandler.Response.fail()), context, sinkNodeName, taskId @@ -1378,7 +1377,7 @@ public void shouldNotThrowTaskCorruptedExceptionOnUnknownTopicOrPartitionExcepti taskId, getExceptionalStreamsProducerOnSend(exception), new ProductionExceptionHandlerMock( - Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.continueProcessing()), + Optional.of(ProductionExceptionHandler.Response.resume()), context, sinkNodeName, taskId @@ -1401,7 +1400,7 @@ public void shouldTreatRetryAsFailForNonRetriableException() { taskId, getExceptionalStreamsProducerOnSend(exception), new ProductionExceptionHandlerMock( - Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.retryProcessing()), + Optional.of(ProductionExceptionHandler.Response.retry()), context, sinkNodeName, taskId @@ -1536,7 +1535,7 @@ public void shouldThrowStreamsExceptionUsingDefaultExceptionHandler() { public void shouldDropRecordExceptionUsingAlwaysContinueExceptionHandler() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock( - Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.continueProcessing()), + Optional.of(ProductionExceptionHandler.Response.resume()), context, sinkNodeName, taskId, @@ -1565,7 +1564,7 @@ public void shouldDropRecordExceptionUsingAlwaysContinueExceptionHandler() { public void shouldThrowStreamsExceptionWhenValueSerializationFailedAndProductionExceptionHandlerRepliesWithFail() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock( - Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.failProcessing()), + Optional.of(ProductionExceptionHandler.Response.fail()), context, sinkNodeName, taskId, @@ -1586,7 +1585,7 @@ public void shouldThrowStreamsExceptionWhenValueSerializationFailedAndProduction public void shouldThrowStreamsExceptionWhenKeySerializationFailedAndProductionExceptionHandlerRepliesWithFail() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { final RecordCollector collector = newRecordCollector(new ProductionExceptionHandlerMock( - Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.failProcessing()), + Optional.of(ProductionExceptionHandler.Response.fail()), context, sinkNodeName, taskId, @@ -1796,7 +1795,7 @@ public void shouldThrowStreamsExceptionOnSubsequentFlushIfASendFailsAndProductio public void shouldNotCallProductionExceptionHandlerOnClassCastException() { try (final ErrorStringSerializer errorSerializer = new ErrorStringSerializer()) { final RecordCollector collector = newRecordCollector( - new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandler.ProductionExceptionResponse.continueProcessing())) + new ProductionExceptionHandlerMock(Optional.of(ProductionExceptionHandler.Response.resume())) ); collector.initialize(); @@ -2080,7 +2079,7 @@ public byte[] serialize(final String topic, final Headers headers, final String } public static class ProductionExceptionHandlerMock implements ProductionExceptionHandler { - private final Optional response; + private final Optional response; private boolean shouldThrowException; private InternalProcessorContext expectedContext; private String expectedProcessorNodeId; @@ -2097,11 +2096,11 @@ public ProductionExceptionHandlerMock() { this.expectedSerializationExceptionOrigin = null; } - public ProductionExceptionHandlerMock(final Optional response) { + public ProductionExceptionHandlerMock(final Optional response) { this.response = response; } - public ProductionExceptionHandlerMock(final Optional response, + public ProductionExceptionHandlerMock(final Optional response, final InternalProcessorContext context, final String processorNodeId, final TaskId taskId) { @@ -2121,7 +2120,7 @@ public ProductionExceptionHandlerMock(final boolean shouldThrowException, this.shouldThrowException = shouldThrowException; } - public ProductionExceptionHandlerMock(final Optional response, + public ProductionExceptionHandlerMock(final Optional response, final InternalProcessorContext context, final String processorNodeId, final TaskId taskId, @@ -2132,9 +2131,9 @@ public ProductionExceptionHandlerMock(final Optional record, - final Exception exception) { + public Response handleError(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception) { assertInputs(context, exception); if (shouldThrowException) { throw new RuntimeException("CRASH"); @@ -2144,10 +2143,10 @@ public ProductionExceptionResponse handleError(final ErrorHandlerContext context @SuppressWarnings("rawtypes") @Override - public ProductionExceptionResponse handleSerializationError(final ErrorHandlerContext context, - final ProducerRecord record, - final Exception exception, - final SerializationExceptionOrigin origin) { + public Response handleSerializationError(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception, + final SerializationExceptionOrigin origin) { assertInputs(context, exception); assertEquals(expectedSerializationExceptionOrigin, origin); if (shouldThrowException) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java index e924f770e995b..ca859bff5df94 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java @@ -115,7 +115,7 @@ public void shouldThrowStreamsExceptionWhenDeserializationFailsAndExceptionHandl "value" ), new DeserializationExceptionHandlerMock( - Optional.of(DeserializationExceptionHandler.DeserializationExceptionResponse.failProcessing()), + Optional.of(DeserializationExceptionHandler.Response.fail()), rawRecord, sourceNodeName, taskId @@ -154,7 +154,7 @@ public void shouldNotThrowStreamsExceptionWhenDeserializationFailsAndExceptionHa "value" ), new DeserializationExceptionHandlerMock( - Optional.of(DeserializationExceptionHandler.DeserializationExceptionResponse.continueProcessing()), + Optional.of(DeserializationExceptionHandler.Response.resume()), rawRecord, sourceNodeName, taskId @@ -341,12 +341,12 @@ public Object deserializeValue(final String topic, final Headers headers, final } public static class DeserializationExceptionHandlerMock implements DeserializationExceptionHandler { - private final Optional response; + private final Optional response; private final ConsumerRecord expectedRecord; private final String expectedProcessorNodeId; private final TaskId expectedTaskId; - public DeserializationExceptionHandlerMock(final Optional response, + public DeserializationExceptionHandlerMock(final Optional response, final ConsumerRecord record, final String processorNodeId, final TaskId taskId) { @@ -357,9 +357,9 @@ public DeserializationExceptionHandlerMock(final Optional record, - final Exception exception) { + public Response handleError(final ErrorHandlerContext context, + final ConsumerRecord record, + final Exception exception) { assertEquals(expectedRecord.topic(), context.topic()); assertEquals(expectedRecord.partition(), context.partition()); assertEquals(expectedRecord.offset(), context.offset()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 7f416f3409ca1..385719530b9d5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -3032,7 +3032,7 @@ public void punctuateShouldThrowFailedProcessingExceptionWhenProcessingException public static class CrashingProcessingExceptionHandler implements ProcessingExceptionHandler { @Override - public ProcessingExceptionResponse handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { + public Response handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { throw new RuntimeException("KABOOM from ProcessingExceptionHandlerMock!"); } @@ -3044,7 +3044,7 @@ public void configure(final Map configs) { public static class NullProcessingExceptionHandler implements ProcessingExceptionHandler { @Override - public ProcessingExceptionResponse handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { + public Response handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { return null; } From d083b6a3bdb30b08bbab6e11980122ad22d5ac97 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Thu, 19 Dec 2024 10:36:09 +0100 Subject: [PATCH 10/16] KAFKA-16505 --- .../integration/ProcessingExceptionHandlerIntegrationTest.java | 2 +- .../kafka/streams/errors/DeserializationExceptionHandler.java | 3 +-- .../kafka/streams/errors/ProcessingExceptionHandler.java | 3 +-- .../kafka/streams/errors/ProductionExceptionHandler.java | 3 +-- 4 files changed, 4 insertions(+), 7 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java index 9a83babde6214..88da45d855fb0 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java @@ -542,7 +542,7 @@ public void configure(final Map configs) { } public static class FailProcessingExceptionHandlerMockTest implements ProcessingExceptionHandler { - @Override + @Override public Response handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { assertProcessingExceptionHandlerInputs(context, record, exception); return Response.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 64e1255443268..2268bb5b29b2f 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 @@ -149,8 +149,7 @@ enum Result { * @return a {@link Result} enum value * @throws IllegalArgumentException if the provided value does not map to a valid {@link Result} */ - @Deprecated - public static DeserializationExceptionHandler.Result from(final DeserializationHandlerResponse value) { + private static DeserializationExceptionHandler.Result from(final DeserializationHandlerResponse value) { switch (value) { case FAIL: return Result.FAIL; 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 index ce6e1e31e6c04..0e071ec484234 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java @@ -116,8 +116,7 @@ enum Result { * @return a {@link ProcessingExceptionHandler.Result} enum value * @throws IllegalArgumentException if the provided value does not map to a valid {@link ProcessingExceptionHandler.Result} */ - @Deprecated - public static ProcessingExceptionHandler.Result from(final ProcessingHandlerResponse value) { + private static ProcessingExceptionHandler.Result from(final ProcessingHandlerResponse value) { switch (value) { case FAIL: return Result.FAIL; 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 0493e029bece0..bf4f32ce471ab 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 @@ -248,8 +248,7 @@ enum Result { * @return a {@link ProductionExceptionHandler.Result} enum value * @throws IllegalArgumentException if the provided value does not map to a valid {@link ProductionExceptionHandler.Result} */ - @Deprecated - public static ProductionExceptionHandler.Result from(final ProductionExceptionHandlerResponse value) { + private static ProductionExceptionHandler.Result from(final ProductionExceptionHandlerResponse value) { switch (value) { case FAIL: return Result.FAIL; From ebb44407d0fff936544904220a62f58be334359a Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Fri, 17 Jan 2025 16:08:06 +0100 Subject: [PATCH 11/16] KAFKA-16505 remove unused methods and add tests --- .../DefaultProductionExceptionHandler.java | 35 --------- .../DeserializationExceptionHandler.java | 4 +- .../streams/errors/ExceptionHandlerUtils.java | 24 +++--- .../LogAndContinueExceptionHandler.java | 42 ----------- ...AndContinueProcessingExceptionHandler.java | 18 ----- .../errors/LogAndFailExceptionHandler.java | 44 +---------- .../LogAndFailProcessingExceptionHandler.java | 19 +---- .../errors/ProductionExceptionHandler.java | 4 +- .../internals/RecordCollectorTest.java | 74 +++++++++++++++++++ 9 files changed, 93 insertions(+), 171 deletions(-) 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 04529291b141d..3f90faf9d4d05 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 @@ -32,30 +32,6 @@ public class DefaultProductionExceptionHandler implements ProductionExceptionHan private String deadLetterQueueTopic = null; - /** - * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ProducerRecord, Exception)} instead. - */ - @SuppressWarnings("deprecation") - @Deprecated - @Override - public ProductionExceptionHandlerResponse handle(final ProducerRecord record, - final Exception exception) { - return exception instanceof RetriableException ? - ProductionExceptionHandler.ProductionExceptionHandlerResponse.RETRY : - ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL; - } - - @SuppressWarnings("deprecation") - @Deprecated - @Override - public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, - final ProducerRecord record, - final Exception exception) { - return exception instanceof RetriableException ? - ProductionExceptionHandler.ProductionExceptionHandlerResponse.RETRY : - ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL; - } - @Override public Response handleError(final ErrorHandlerContext context, final ProducerRecord record, @@ -65,17 +41,6 @@ public Response handleError(final ErrorHandlerContext context, Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } - - @SuppressWarnings("deprecation") - @Deprecated - @Override - public ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context, - final ProducerRecord record, - final Exception exception, - final SerializationExceptionOrigin origin) { - return ProductionExceptionHandler.ProductionExceptionHandlerResponse.FAIL; - } - @Override public Response handleSerializationError(final ErrorHandlerContext context, final ProducerRecord record, 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 2268bb5b29b2f..b2978b200f6ac 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 @@ -164,7 +164,7 @@ private static DeserializationExceptionHandler.Result from(final Deserialization /** * Represents the result of handling a deserialization exception. *

- * The {@code Response} class encapsulates a {@link ProcessingExceptionHandler.Result}, + * The {@code Response} class encapsulates a {@link Result}, * indicating whether processing should continue or fail, along with an optional list of * {@link ProducerRecord} instances to be sent to a dead letter queue. *

@@ -220,7 +220,7 @@ public static Response resume(final List> deadLet /** * Creates a {@code Response} indicating that processing should continue. * - * @return a {@code Response} with a {@link DeserializationHandlerResponse#CONTINUE} status. + * @return a {@code Response} with a {@link DeserializationExceptionHandler.Result#RESUME} status. */ public static Response resume() { return resume(Collections.emptyList()); diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ExceptionHandlerUtils.java b/streams/src/main/java/org/apache/kafka/streams/errors/ExceptionHandlerUtils.java index c8a8cfdddc64d..41ddcd0a02f7a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ExceptionHandlerUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ExceptionHandlerUtils.java @@ -28,7 +28,7 @@ import java.util.List; /** - * {@code CommonExceptionHandler} Contains utilities method that could be used by all exception handlers + * {@code ExceptionHandlerUtils} Contains utilities method that could be used by all exception handlers */ class ExceptionHandlerUtils { static final String HEADER_ERRORS_EXCEPTION_NAME = "__streams.errors.exception"; @@ -45,6 +45,7 @@ static boolean shouldBuildDeadLetterQueueRecord(final String deadLetterQueueTopi /** * If required, return Dead Letter Queue records for the provided exception + * * @param key Serialized key for the records * @param value Serialized value for the records * @param context ErrorHandlerContext of the exception @@ -65,19 +66,20 @@ static List> maybeBuildDeadLetterQueueRecords(fin /** - * Build Dead Letter Queue records for the provided exception - * @param key Serialized key for the records - * @param value Serialized value for the records - * @param context ErrorHandlerContext of the exception - * @return A list of Dead Letter Queue records to produce + * Build dead letter queue record for the provided exception. + * + * @param key Serialized key for the record. + * @param value Serialized value for the record. + * @param context error handler context of the exception. + * @return A dead letter queue record to produce. */ static ProducerRecord buildDeadLetterQueueRecord(final String deadLetterQueueTopicName, - final byte[] key, - final byte[] value, - final ErrorHandlerContext context, - final Exception e) { + final byte[] key, + final byte[] value, + final ErrorHandlerContext context, + final Exception e) { if (deadLetterQueueTopicName == null) { - throw new InvalidConfigurationException(String.format("%s can not be null while building DeadLetterQueue record", StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); + throw new InvalidConfigurationException(String.format("%s cannot be null while building dead letter queue record", StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); } final ProducerRecord producerRecord = new ProducerRecord<>(deadLetterQueueTopicName, null, context.timestamp(), key, value); final StringWriter stackStraceStringWriter = new StringWriter(); 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 34e7615dbc58a..e79e47d6b014f 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,7 +18,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.processor.ProcessorContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,47 +34,6 @@ public class LogAndContinueExceptionHandler implements DeserializationExceptionH private static final Logger log = LoggerFactory.getLogger(LogAndContinueExceptionHandler.class); private String deadLetterQueueTopic = null; - /** - * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)} instead. - */ - @SuppressWarnings("deprecation") - @Deprecated - @Override - public DeserializationHandlerResponse handle(final ProcessorContext 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; - } - - @SuppressWarnings("deprecation") - @Deprecated - @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 Response handleError(final ErrorHandlerContext context, final ConsumerRecord record, diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java index 671bc89bcab4d..29e0f08595ba1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java @@ -34,24 +34,6 @@ public class LogAndContinueProcessingExceptionHandler implements ProcessingExcep private static final Logger log = LoggerFactory.getLogger(LogAndContinueProcessingExceptionHandler.class); private String deadLetterQueueTopic = null; - @Deprecated - @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 Response handleError(final ErrorHandlerContext context, final Record record, 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 5b3d5ca123bed..20f75b7dcf598 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 @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.processor.ProcessorContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,52 +34,11 @@ public class LogAndFailExceptionHandler implements DeserializationExceptionHandl private static final Logger log = LoggerFactory.getLogger(LogAndFailExceptionHandler.class); private String deadLetterQueueTopic = null; - /** - * @deprecated Since 3.9. Use {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)} instead. - */ - @SuppressWarnings("deprecation") - @Deprecated - @Override - public DeserializationHandlerResponse handle(final ProcessorContext 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; - } - - @SuppressWarnings("deprecation") - @Deprecated - @Override - 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 Response handleError(final ErrorHandlerContext context, final ConsumerRecord record, final Exception exception) { - log.warn( + log.error( "Exception caught during Deserialization, taskId: {}, topic: {}, partition: {}, offset: {}", context.taskId(), record.topic(), diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java index eb35021f8f86b..5f84b85843824 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java @@ -34,28 +34,11 @@ public class LogAndFailProcessingExceptionHandler implements ProcessingException private static final Logger log = LoggerFactory.getLogger(LogAndFailProcessingExceptionHandler.class); private String deadLetterQueueTopic = null; - @Deprecated - @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, - final Record record, final Exception exception) { - log.error( - "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 Response handleError(final ErrorHandlerContext context, final Record record, final Exception exception) { - log.warn( + log.error( "Exception caught during message processing, processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", context.processorNodeId(), context.taskId(), 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 bf4f32ce471ab..a17c99c699997 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 @@ -64,7 +64,7 @@ default ProductionExceptionHandlerResponse handle(final ProducerRecord record, final Exception exception) { - throw new UnsupportedOperationException(); + return handle(record, exception); } /** @@ -272,7 +272,7 @@ enum SerializationExceptionOrigin { /** * Represents the result of handling a production exception. *

- * The {@code Response} class encapsulates a {@link ProductionExceptionHandlerResponse}, + * The {@code Response} class encapsulates a {@link Result}, * indicating whether processing should continue or fail, along with an optional list of * {@link ProducerRecord} instances to be sent to a dead letter queue. *

diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index 69d21bae40b2e..08ef31064e1ed 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -2006,6 +2006,51 @@ public void configure(final Map configs) { } } + public void shouldCallOldImplementationExceptionHandler() { + final KafkaException exception = new KafkaException("KABOOM!"); + final StreamsProducer streamProducer = getExceptionalStreamsProducerOnSend(exception); + final OldProductionExceptionHandlerImplementation productionExceptionHandler = new OldProductionExceptionHandlerImplementation(); + + final RecordCollector collector = new RecordCollectorImpl( + logContext, + taskId, + streamProducer, + productionExceptionHandler, + streamsMetrics, + topology + ); + + collector.initialize(); + + collector.send(topic, "hello", "world", null, 0, null, stringSerializer, stringSerializer, sinkNodeName, context); + final Exception thrown = assertThrows(StreamsException.class, collector::flush); + + assertEquals(exception, thrown.getCause()); + } + + @Test + public void shouldCallOldImplementationWithRecordContextExceptionHandler() { + final KafkaException exception = new KafkaException("KABOOM!"); + final StreamsProducer streamProducer = getExceptionalStreamsProducerOnSend(exception); + final OldProductionExceptionHandlerWithRecordContextImplementation productionExceptionHandler = new OldProductionExceptionHandlerWithRecordContextImplementation(); + + final RecordCollector collector = new RecordCollectorImpl( + logContext, + taskId, + streamProducer, + productionExceptionHandler, + streamsMetrics, + topology + ); + + collector.initialize(); + + collector.send(topic, "hello", "world", null, 0, null, stringSerializer, stringSerializer, sinkNodeName, context); + final Exception thrown = assertThrows(StreamsException.class, collector::flush); + + assertEquals(exception, thrown.getCause()); + } + private RecordCollector newRecordCollector(final ProductionExceptionHandler productionExceptionHandler) { return new RecordCollectorImpl( logContext, @@ -2171,4 +2216,33 @@ private void assertInputs(final ErrorHandlerContext context, final Exception exc assertEquals("KABOOM!", exception.getMessage()); } } + + public class OldProductionExceptionHandlerImplementation implements ProductionExceptionHandler { + + @SuppressWarnings("deprecation") + @Override + public ProductionExceptionHandlerResponse handle(final ProducerRecord record, + final Exception exception) { + return ProductionExceptionHandlerResponse.FAIL; + } + + @Override + public void configure(final Map configs) { + } + } + + public class OldProductionExceptionHandlerWithRecordContextImplementation implements ProductionExceptionHandler { + + @SuppressWarnings("deprecation") + @Override + public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception) { + return ProductionExceptionHandlerResponse.FAIL; + } + + @Override + public void configure(final Map configs) { + } + } } From d1d41a158a5de92fef948b7c24a1548a7cdd7474 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Fri, 24 Jan 2025 15:53:32 +0100 Subject: [PATCH 12/16] KAFKA-16505 --- .../DefaultProductionExceptionHandler.java | 2 +- .../LogAndContinueExceptionHandler.java | 2 +- ...AndContinueProcessingExceptionHandler.java | 2 +- .../errors/LogAndFailExceptionHandler.java | 2 +- .../LogAndFailProcessingExceptionHandler.java | 2 +- .../ExceptionHandlerUtils.java | 23 +++--- .../errors/ExceptionHandlerUtilsTest.java | 1 + .../internals/ProcessorNodeTest.java | 62 ++++++++++++++++ .../internals/RecordCollectorTest.java | 71 ++++++++++++++++++- .../internals/RecordDeserializerTest.java | 63 ++++++++++++++++ 10 files changed, 212 insertions(+), 18 deletions(-) rename streams/src/main/java/org/apache/kafka/streams/errors/{ => internals}/ExceptionHandlerUtils.java (81%) 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 3f90faf9d4d05..0ec2087022191 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 @@ -22,7 +22,7 @@ import java.util.Map; -import static org.apache.kafka.streams.errors.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; +import static org.apache.kafka.streams.errors.internals.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; /** * {@code ProductionExceptionHandler} that always instructs streams to fail when an exception 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 e79e47d6b014f..b745ca9226c00 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 @@ -24,7 +24,7 @@ import java.util.Map; -import static org.apache.kafka.streams.errors.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; +import static org.apache.kafka.streams.errors.internals.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; /** * Deserialization handler that logs a deserialization exception and then diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java index 29e0f08595ba1..48116a8feb6a3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java @@ -24,7 +24,7 @@ import java.util.Map; -import static org.apache.kafka.streams.errors.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; +import static org.apache.kafka.streams.errors.internals.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; /** * Processing exception handler that logs a processing exception and then 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 20f75b7dcf598..2e0f688c6876d 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 @@ -24,7 +24,7 @@ import java.util.Map; -import static org.apache.kafka.streams.errors.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; +import static org.apache.kafka.streams.errors.internals.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; /** * Deserialization handler that logs a deserialization exception and then diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java index 5f84b85843824..7a70537b25b8f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java @@ -24,7 +24,7 @@ import java.util.Map; -import static org.apache.kafka.streams.errors.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; +import static org.apache.kafka.streams.errors.internals.ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords; /** * Processing exception handler that logs a processing exception and then diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ExceptionHandlerUtils.java b/streams/src/main/java/org/apache/kafka/streams/errors/internals/ExceptionHandlerUtils.java similarity index 81% rename from streams/src/main/java/org/apache/kafka/streams/errors/ExceptionHandlerUtils.java rename to streams/src/main/java/org/apache/kafka/streams/errors/internals/ExceptionHandlerUtils.java index 41ddcd0a02f7a..5764865c6d3e1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ExceptionHandlerUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/internals/ExceptionHandlerUtils.java @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.kafka.streams.errors; +package org.apache.kafka.streams.errors.internals; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.ErrorHandlerContext; import java.io.PrintWriter; import java.io.StringWriter; @@ -30,16 +31,16 @@ /** * {@code ExceptionHandlerUtils} Contains utilities method that could be used by all exception handlers */ -class ExceptionHandlerUtils { - static final String HEADER_ERRORS_EXCEPTION_NAME = "__streams.errors.exception"; - static final String HEADER_ERRORS_STACKTRACE_NAME = "__streams.errors.stacktrace"; - static final String HEADER_ERRORS_EXCEPTION_MESSAGE_NAME = "__streams.errors.message"; - static final String HEADER_ERRORS_TOPIC_NAME = "__streams.errors.topic"; - static final String HEADER_ERRORS_PARTITION_NAME = "__streams.errors.partition"; - static final String HEADER_ERRORS_OFFSET_NAME = "__streams.errors.offset"; +public class ExceptionHandlerUtils { + public static final String HEADER_ERRORS_EXCEPTION_NAME = "__streams.errors.exception"; + public static final String HEADER_ERRORS_STACKTRACE_NAME = "__streams.errors.stacktrace"; + public static final String HEADER_ERRORS_EXCEPTION_MESSAGE_NAME = "__streams.errors.message"; + public static final String HEADER_ERRORS_TOPIC_NAME = "__streams.errors.topic"; + public static final String HEADER_ERRORS_PARTITION_NAME = "__streams.errors.partition"; + public static final String HEADER_ERRORS_OFFSET_NAME = "__streams.errors.offset"; - static boolean shouldBuildDeadLetterQueueRecord(final String deadLetterQueueTopicName) { + public static boolean shouldBuildDeadLetterQueueRecord(final String deadLetterQueueTopicName) { return deadLetterQueueTopicName != null; } @@ -52,7 +53,7 @@ static boolean shouldBuildDeadLetterQueueRecord(final String deadLetterQueueTopi * @param exception Thrown exception * @return A list of Dead Letter Queue records to produce */ - static List> maybeBuildDeadLetterQueueRecords(final String deadLetterQueueTopicName, + public static List> maybeBuildDeadLetterQueueRecords(final String deadLetterQueueTopicName, final byte[] key, final byte[] value, final ErrorHandlerContext context, @@ -73,7 +74,7 @@ static List> maybeBuildDeadLetterQueueRecords(fin * @param context error handler context of the exception. * @return A dead letter queue record to produce. */ - static ProducerRecord buildDeadLetterQueueRecord(final String deadLetterQueueTopicName, + public static ProducerRecord buildDeadLetterQueueRecord(final String deadLetterQueueTopicName, final byte[] key, final byte[] value, final ErrorHandlerContext context, diff --git a/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java b/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java index e84a76427e61b..65ff62ce7e269 100644 --- a/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.errors.internals.DefaultErrorHandlerContext; +import org.apache.kafka.streams.errors.internals.ExceptionHandlerUtils; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.state.StateSerdes; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index 5261d3989df4f..8c180b8800b09 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.InvalidOffsetException; +import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.metrics.Metrics; @@ -56,10 +57,13 @@ import java.util.Collections; import java.util.HashSet; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Set; +import static org.apache.kafka.streams.errors.ProcessingExceptionHandler.Response; +import static org.apache.kafka.streams.errors.ProcessingExceptionHandler.Result; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.ROLLUP_VALUE; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -370,6 +374,64 @@ public void testTopologyLevelClassCastExceptionDirect() { assertTrue(se.getMessage().contains("pname")); } + @Test + void shouldFailWithDeadLetterQueueRecords() { + ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + List> records = Collections.singletonList(record); + + Response response = Response.fail(records); + + assertEquals(Result.FAIL, response.result()); + assertEquals(1, response.deadLetterQueueRecords().size()); + assertEquals(record, response.deadLetterQueueRecords().get(0)); + } + + @Test + void shouldFailWithoutDeadLetterQueueRecords() { + Response response = Response.fail(); + + assertEquals(Result.FAIL, response.result()); + assertTrue(response.deadLetterQueueRecords().isEmpty()); + } + + @Test + void shouldResumeWithDeadLetterQueueRecords() { + ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + List> records = Collections.singletonList(record); + + Response response = Response.resume(records); + + assertEquals(Result.RESUME, response.result()); + assertEquals(1, response.deadLetterQueueRecords().size()); + assertEquals(record, response.deadLetterQueueRecords().get(0)); + } + + @Test + void shouldResumeWithoutDeadLetterQueueRecords() { + Response response = Response.resume(); + + assertEquals(Result.RESUME, response.result()); + assertTrue(response.deadLetterQueueRecords().isEmpty()); + } + + + @Test + void shouldNotBeModifiable() { + ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + List> records = Collections.singletonList(record); + + Response response = Response.fail(records); + + assertThrows(UnsupportedOperationException.class, () -> response.deadLetterQueueRecords().add(record)); + } + + @Test + void shouldReturnsEmptyList() { + Response response = Response.fail(); + + assertTrue(response.deadLetterQueueRecords().isEmpty()); + } + @SuppressWarnings("unchecked") private InternalProcessorContext mockInternalProcessorContext() { final InternalProcessorContext internalProcessorContext = mock(InternalProcessorContext.class, withSettings().strictness(Strictness.LENIENT)); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index 08ef31064e1ed..a597e22075b38 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -93,6 +93,8 @@ import static org.apache.kafka.streams.internals.StreamsConfigUtils.ProcessingMode.EXACTLY_ONCE_V2; import static org.apache.kafka.streams.processor.internals.ClientUtils.producerRecordSizeInBytes; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.TOPIC_LEVEL_GROUP; +import static org.apache.kafka.streams.errors.ProductionExceptionHandler.Response; +import static org.apache.kafka.streams.errors.ProductionExceptionHandler.Result; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsEqual.equalTo; import static org.hamcrest.core.IsInstanceOf.instanceOf; @@ -2051,6 +2053,71 @@ public void shouldCallOldImplementationWithRecordContextExceptionHandler() { assertEquals(exception, thrown.getCause()); } + @Test + void shouldFailWithDeadLetterQueueRecords() { + ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + List> records = Collections.singletonList(record); + + ProductionExceptionHandler.Response response = ProductionExceptionHandler.Response.fail(records); + + assertEquals(Result.FAIL, response.result()); + assertEquals(1, response.deadLetterQueueRecords().size()); + assertEquals(record, response.deadLetterQueueRecords().get(0)); + } + + @Test + void shouldFailWithoutDeadLetterQueueRecords() { + ProductionExceptionHandler.Response response = ProductionExceptionHandler.Response.fail(); + + assertEquals(Result.FAIL, response.result()); + assertTrue(response.deadLetterQueueRecords().isEmpty()); + } + + @Test + void shouldResumeWithDeadLetterQueueRecords() { + ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + List> records = Collections.singletonList(record); + + Response response = Response.resume(records); + + assertEquals(Result.RESUME, response.result()); + assertEquals(1, response.deadLetterQueueRecords().size()); + assertEquals(record, response.deadLetterQueueRecords().get(0)); + } + + @Test + void shouldResumeWithoutDeadLetterQueueRecords() { + Response response = Response.resume(); + + assertEquals(Result.RESUME, response.result()); + assertTrue(response.deadLetterQueueRecords().isEmpty()); + } + + @Test + void shouldRetryWithoutDeadLetterQueueRecords() { + Response response = Response.retry(); + + assertEquals(Result.RETRY, response.result()); + assertTrue(response.deadLetterQueueRecords().isEmpty()); + } + + @Test + void shouldNotBeModifiable() { + ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + List> records = Collections.singletonList(record); + + Response response = Response.fail(records); + + assertThrows(UnsupportedOperationException.class, () -> response.deadLetterQueueRecords().add(record)); + } + + @Test + void shouldReturnsEmptyList() { + Response response = Response.fail(); + + assertTrue(response.deadLetterQueueRecords().isEmpty()); + } + private RecordCollector newRecordCollector(final ProductionExceptionHandler productionExceptionHandler) { return new RecordCollectorImpl( logContext, @@ -2217,7 +2284,7 @@ private void assertInputs(final ErrorHandlerContext context, final Exception exc } } - public class OldProductionExceptionHandlerImplementation implements ProductionExceptionHandler { + public static class OldProductionExceptionHandlerImplementation implements ProductionExceptionHandler { @SuppressWarnings("deprecation") @Override @@ -2231,7 +2298,7 @@ public void configure(final Map configs) { } } - public class OldProductionExceptionHandlerWithRecordContextImplementation implements ProductionExceptionHandler { + public static class OldProductionExceptionHandlerWithRecordContextImplementation implements ProductionExceptionHandler { @SuppressWarnings("deprecation") @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java index ca859bff5df94..d592ce755a87f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeader; @@ -42,14 +43,18 @@ import java.nio.charset.StandardCharsets; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Optional; +import static org.apache.kafka.streams.errors.DeserializationExceptionHandler.Response; +import static org.apache.kafka.streams.errors.DeserializationExceptionHandler.Result; import static org.apache.kafka.streams.StreamsConfig.DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; public class RecordDeserializerTest { private final String sourceNodeName = "source-node"; @@ -305,6 +310,64 @@ public void shouldBuildDeadLetterQueueRecordsInLogAndContinueDeserializationExce } } + @Test + void shouldFailWithDeadLetterQueueRecords() { + ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + List> records = Collections.singletonList(record); + + Response response = Response.fail(records); + + assertEquals(Result.FAIL, response.result()); + assertEquals(1, response.deadLetterQueueRecords().size()); + assertEquals(record, response.deadLetterQueueRecords().get(0)); + } + + @Test + void shouldFailWithoutDeadLetterQueueRecords() { + Response response = DeserializationExceptionHandler.Response.fail(); + + assertEquals(Result.FAIL, response.result()); + assertTrue(response.deadLetterQueueRecords().isEmpty()); + } + + @Test + void shouldResumeWithDeadLetterQueueRecords() { + ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + List> records = Collections.singletonList(record); + + Response response = Response.resume(records); + + assertEquals(Result.RESUME, response.result()); + assertEquals(1, response.deadLetterQueueRecords().size()); + assertEquals(record, response.deadLetterQueueRecords().get(0)); + } + + @Test + void shouldResumeWithoutDeadLetterQueueRecords() { + Response response = Response.resume(); + + assertEquals(Result.RESUME, response.result()); + assertTrue(response.deadLetterQueueRecords().isEmpty()); + } + + + @Test + void shouldNotBeModifiable() { + ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + List> records = Collections.singletonList(record); + + Response response = Response.fail(records); + + assertThrows(UnsupportedOperationException.class, () -> response.deadLetterQueueRecords().add(record)); + } + + @Test + void shouldReturnsEmptyList() { + Response response = Response.fail(); + + assertTrue(response.deadLetterQueueRecords().isEmpty()); + } + static class TheSourceNode extends SourceNode { private final boolean keyThrowsException; private final boolean valueThrowsException; From 899b878440827b0b326f1cb25887bed1aabcd84e Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Mon, 24 Mar 2025 14:41:03 +0100 Subject: [PATCH 13/16] KAFKA-16505: fix unit tests --- .../apache/kafka/streams/StreamsConfig.java | 2 +- .../DefaultProductionExceptionHandler.java | 1 + .../internals/ProcessorNodeTest.java | 24 +++++++------- .../internals/RecordCollectorTest.java | 32 +++++++++---------- .../internals/RecordDeserializerTest.java | 26 +++++++-------- 5 files changed, 43 insertions(+), 42 deletions(-) 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 018dd71972a4f..a25fa0b19c3c5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -620,7 +620,7 @@ public class StreamsConfig extends AbstractConfig { "used. Otherwise, the classic group protocol will be used."; public static final String ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG = "errors.dead.letter.queue.topic.name"; - public static final String ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_DOC = "If not null, the default exception handler will build and send a Dead Letter Queue record in the provided topic name if an error occurs.\n" + + private static final String ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_DOC = "If not null, the default exception handler will build and send a Dead Letter Queue record in the provided topic name if an error occurs.\n" + "If a custom deserialization/production or processing exception handler is set, this parameter is ignored for this handler."; /** {@code log.summary.interval.ms} */ 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 0ec2087022191..4c99b08ba7e0e 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 @@ -41,6 +41,7 @@ public Response handleError(final ErrorHandlerContext context, Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); } + @SuppressWarnings("rawtypes") @Override public Response handleSerializationError(final ErrorHandlerContext context, final ProducerRecord record, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index 8c180b8800b09..5270d00978ed6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -376,10 +376,10 @@ public void testTopologyLevelClassCastExceptionDirect() { @Test void shouldFailWithDeadLetterQueueRecords() { - ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); - List> records = Collections.singletonList(record); + final ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + final List> records = Collections.singletonList(record); - Response response = Response.fail(records); + final Response response = Response.fail(records); assertEquals(Result.FAIL, response.result()); assertEquals(1, response.deadLetterQueueRecords().size()); @@ -388,7 +388,7 @@ void shouldFailWithDeadLetterQueueRecords() { @Test void shouldFailWithoutDeadLetterQueueRecords() { - Response response = Response.fail(); + final Response response = Response.fail(); assertEquals(Result.FAIL, response.result()); assertTrue(response.deadLetterQueueRecords().isEmpty()); @@ -396,10 +396,10 @@ void shouldFailWithoutDeadLetterQueueRecords() { @Test void shouldResumeWithDeadLetterQueueRecords() { - ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); - List> records = Collections.singletonList(record); + final ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + final List> records = Collections.singletonList(record); - Response response = Response.resume(records); + final Response response = Response.resume(records); assertEquals(Result.RESUME, response.result()); assertEquals(1, response.deadLetterQueueRecords().size()); @@ -408,7 +408,7 @@ void shouldResumeWithDeadLetterQueueRecords() { @Test void shouldResumeWithoutDeadLetterQueueRecords() { - Response response = Response.resume(); + final Response response = Response.resume(); assertEquals(Result.RESUME, response.result()); assertTrue(response.deadLetterQueueRecords().isEmpty()); @@ -417,17 +417,17 @@ void shouldResumeWithoutDeadLetterQueueRecords() { @Test void shouldNotBeModifiable() { - ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); - List> records = Collections.singletonList(record); + final ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + final List> records = Collections.singletonList(record); - Response response = Response.fail(records); + final Response response = Response.fail(records); assertThrows(UnsupportedOperationException.class, () -> response.deadLetterQueueRecords().add(record)); } @Test void shouldReturnsEmptyList() { - Response response = Response.fail(); + final Response response = Response.fail(); assertTrue(response.deadLetterQueueRecords().isEmpty()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index a597e22075b38..cc49f71fea07b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -89,12 +89,12 @@ import static java.util.Collections.singletonMap; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.streams.errors.ProductionExceptionHandler.Response; +import static org.apache.kafka.streams.errors.ProductionExceptionHandler.Result; import static org.apache.kafka.streams.internals.StreamsConfigUtils.ProcessingMode.AT_LEAST_ONCE; import static org.apache.kafka.streams.internals.StreamsConfigUtils.ProcessingMode.EXACTLY_ONCE_V2; import static org.apache.kafka.streams.processor.internals.ClientUtils.producerRecordSizeInBytes; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.TOPIC_LEVEL_GROUP; -import static org.apache.kafka.streams.errors.ProductionExceptionHandler.Response; -import static org.apache.kafka.streams.errors.ProductionExceptionHandler.Result; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsEqual.equalTo; import static org.hamcrest.core.IsInstanceOf.instanceOf; @@ -1848,7 +1848,7 @@ public void shouldBuildDeadLetterQueueRecordsInDefaultExceptionHandlerDuringDese collector.initialize(); assertThat(mockProducer.history().isEmpty(), equalTo(true)); - final StreamsException error = assertThrows( + assertThrows( StreamsException.class, () -> collector.send(topic, "hello", "world", null, 0, null, errorSerializer, stringSerializer, sinkNodeName, context) @@ -2055,10 +2055,10 @@ public void shouldCallOldImplementationWithRecordContextExceptionHandler() { @Test void shouldFailWithDeadLetterQueueRecords() { - ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); - List> records = Collections.singletonList(record); + final ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + final List> records = Collections.singletonList(record); - ProductionExceptionHandler.Response response = ProductionExceptionHandler.Response.fail(records); + final ProductionExceptionHandler.Response response = ProductionExceptionHandler.Response.fail(records); assertEquals(Result.FAIL, response.result()); assertEquals(1, response.deadLetterQueueRecords().size()); @@ -2067,7 +2067,7 @@ void shouldFailWithDeadLetterQueueRecords() { @Test void shouldFailWithoutDeadLetterQueueRecords() { - ProductionExceptionHandler.Response response = ProductionExceptionHandler.Response.fail(); + final ProductionExceptionHandler.Response response = ProductionExceptionHandler.Response.fail(); assertEquals(Result.FAIL, response.result()); assertTrue(response.deadLetterQueueRecords().isEmpty()); @@ -2075,10 +2075,10 @@ void shouldFailWithoutDeadLetterQueueRecords() { @Test void shouldResumeWithDeadLetterQueueRecords() { - ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); - List> records = Collections.singletonList(record); + final ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + final List> records = Collections.singletonList(record); - Response response = Response.resume(records); + final Response response = Response.resume(records); assertEquals(Result.RESUME, response.result()); assertEquals(1, response.deadLetterQueueRecords().size()); @@ -2087,7 +2087,7 @@ void shouldResumeWithDeadLetterQueueRecords() { @Test void shouldResumeWithoutDeadLetterQueueRecords() { - Response response = Response.resume(); + final Response response = Response.resume(); assertEquals(Result.RESUME, response.result()); assertTrue(response.deadLetterQueueRecords().isEmpty()); @@ -2095,7 +2095,7 @@ void shouldResumeWithoutDeadLetterQueueRecords() { @Test void shouldRetryWithoutDeadLetterQueueRecords() { - Response response = Response.retry(); + final Response response = Response.retry(); assertEquals(Result.RETRY, response.result()); assertTrue(response.deadLetterQueueRecords().isEmpty()); @@ -2103,17 +2103,17 @@ void shouldRetryWithoutDeadLetterQueueRecords() { @Test void shouldNotBeModifiable() { - ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); - List> records = Collections.singletonList(record); + final ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + final List> records = Collections.singletonList(record); - Response response = Response.fail(records); + final Response response = Response.fail(records); assertThrows(UnsupportedOperationException.class, () -> response.deadLetterQueueRecords().add(record)); } @Test void shouldReturnsEmptyList() { - Response response = Response.fail(); + final Response response = Response.fail(); assertTrue(response.deadLetterQueueRecords().isEmpty()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java index d592ce755a87f..49a5984758ad6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java @@ -47,9 +47,9 @@ import java.util.Map; import java.util.Optional; +import static org.apache.kafka.streams.StreamsConfig.DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG; import static org.apache.kafka.streams.errors.DeserializationExceptionHandler.Response; import static org.apache.kafka.streams.errors.DeserializationExceptionHandler.Result; -import static org.apache.kafka.streams.StreamsConfig.DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNull; @@ -312,10 +312,10 @@ public void shouldBuildDeadLetterQueueRecordsInLogAndContinueDeserializationExce @Test void shouldFailWithDeadLetterQueueRecords() { - ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); - List> records = Collections.singletonList(record); + final ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + final List> records = Collections.singletonList(record); - Response response = Response.fail(records); + final Response response = Response.fail(records); assertEquals(Result.FAIL, response.result()); assertEquals(1, response.deadLetterQueueRecords().size()); @@ -324,7 +324,7 @@ void shouldFailWithDeadLetterQueueRecords() { @Test void shouldFailWithoutDeadLetterQueueRecords() { - Response response = DeserializationExceptionHandler.Response.fail(); + final Response response = DeserializationExceptionHandler.Response.fail(); assertEquals(Result.FAIL, response.result()); assertTrue(response.deadLetterQueueRecords().isEmpty()); @@ -332,10 +332,10 @@ void shouldFailWithoutDeadLetterQueueRecords() { @Test void shouldResumeWithDeadLetterQueueRecords() { - ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); - List> records = Collections.singletonList(record); + final ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + final List> records = Collections.singletonList(record); - Response response = Response.resume(records); + final Response response = Response.resume(records); assertEquals(Result.RESUME, response.result()); assertEquals(1, response.deadLetterQueueRecords().size()); @@ -344,7 +344,7 @@ void shouldResumeWithDeadLetterQueueRecords() { @Test void shouldResumeWithoutDeadLetterQueueRecords() { - Response response = Response.resume(); + final Response response = Response.resume(); assertEquals(Result.RESUME, response.result()); assertTrue(response.deadLetterQueueRecords().isEmpty()); @@ -353,17 +353,17 @@ void shouldResumeWithoutDeadLetterQueueRecords() { @Test void shouldNotBeModifiable() { - ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); - List> records = Collections.singletonList(record); + final ProducerRecord record = new ProducerRecord<>("topic", new byte[]{}, new byte[]{}); + final List> records = Collections.singletonList(record); - Response response = Response.fail(records); + final Response response = Response.fail(records); assertThrows(UnsupportedOperationException.class, () -> response.deadLetterQueueRecords().add(record)); } @Test void shouldReturnsEmptyList() { - Response response = Response.fail(); + final Response response = Response.fail(); assertTrue(response.deadLetterQueueRecords().isEmpty()); } From 288f2869b69a8f9c17d875450e64aed05ebca0cb Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Thu, 5 Jun 2025 13:59:30 +0200 Subject: [PATCH 14/16] KFK-16505 fix after rebase --- .../kafka/streams/errors/ExceptionHandlerUtilsTest.java | 4 +++- .../apache/kafka/streams/state/KeyValueStoreTestDriver.java | 4 ++-- .../test/java/org/apache/kafka/test/MockRecordCollector.java | 4 ++-- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java b/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java index 65ff62ce7e269..ba0ecebc1531e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java @@ -101,6 +101,8 @@ private static DefaultErrorHandlerContext getErrorHandlerContext(final InternalP internalProcessorContext.headers(), internalProcessorContext.currentNode().name(), internalProcessorContext.taskId(), - internalProcessorContext.timestamp()); + internalProcessorContext.timestamp(), + internalProcessorContext.recordContext().sourceRawKey(), + internalProcessorContext.recordContext().sourceRawValue()); } } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java index c1959b7de7f71..78c6dedcbf45c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java @@ -234,7 +234,7 @@ public void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context) { + final InternalProcessorContext context) { // for byte arrays we need to wrap it for comparison final byte[] keyBytes = keySerializer.serialize(topic, headers, key); @@ -255,7 +255,7 @@ public void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context, + final InternalProcessorContext context, final StreamPartitioner partitioner) { throw new UnsupportedOperationException(); } diff --git a/streams/src/test/java/org/apache/kafka/test/MockRecordCollector.java b/streams/src/test/java/org/apache/kafka/test/MockRecordCollector.java index 643d5d88d32c8..ed6898b371386 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockRecordCollector.java +++ b/streams/src/test/java/org/apache/kafka/test/MockRecordCollector.java @@ -49,7 +49,7 @@ public void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context) { + final InternalProcessorContext context) { collected.add(new ProducerRecord<>( topic, partition, @@ -69,7 +69,7 @@ public void send(final String topic, final Serializer keySerializer, final Serializer valueSerializer, final String processorNodeId, - final InternalProcessorContext context, + final InternalProcessorContext context, final StreamPartitioner partitioner) { collected.add(new ProducerRecord<>( topic, From b55a6a5464b068eb55e66c80a5e4be2ee044d9c5 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Thu, 5 Jun 2025 15:10:58 +0200 Subject: [PATCH 15/16] KFK-16505 complete rawKey and rawValue --- .../errors/DefaultProductionExceptionHandler.java | 4 ++-- .../streams/errors/LogAndContinueExceptionHandler.java | 2 +- .../LogAndContinueProcessingExceptionHandler.java | 2 +- .../streams/errors/LogAndFailExceptionHandler.java | 2 +- .../errors/LogAndFailProcessingExceptionHandler.java | 2 +- .../streams/errors/ExceptionHandlerUtilsTest.java | 10 ++++++++-- .../streams/processor/internals/ProcessorNodeTest.java | 4 ++++ .../processor/internals/RecordDeserializerTest.java | 4 ++++ 8 files changed, 22 insertions(+), 8 deletions(-) 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 4c99b08ba7e0e..e796df783d678 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 @@ -38,7 +38,7 @@ public Response handleError(final ErrorHandlerContext context, final Exception exception) { return exception instanceof RetriableException ? Response.retry() : - Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, context.sourceRawKey(), context.sourceRawKey(), context, exception)); } @SuppressWarnings("rawtypes") @@ -47,7 +47,7 @@ public Response handleSerializationError(final ErrorHandlerContext context, final ProducerRecord record, final Exception exception, final SerializationExceptionOrigin origin) { - return Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + return Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, context.sourceRawKey(), context.sourceRawValue(), context, exception)); } 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 b745ca9226c00..63972b0840d44 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 @@ -47,7 +47,7 @@ public Response handleError(final ErrorHandlerContext context, exception ); - return Response.resume(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + return Response.resume(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, context.sourceRawKey(), context.sourceRawValue(), context, exception)); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java index 48116a8feb6a3..17de09e5f0cd2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueProcessingExceptionHandler.java @@ -47,7 +47,7 @@ public Response handleError(final ErrorHandlerContext context, context.offset(), exception ); - return Response.resume(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + return Response.resume(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, context.sourceRawKey(), context.sourceRawValue(), context, exception)); } @Override 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 2e0f688c6876d..6fc129b4d78bf 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 @@ -47,7 +47,7 @@ public Response handleError(final ErrorHandlerContext context, exception ); - return Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + return Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, context.sourceRawKey(), context.sourceRawValue(), context, exception)); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java index 7a70537b25b8f..5372d9ad0b679 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailProcessingExceptionHandler.java @@ -48,7 +48,7 @@ public Response handleError(final ErrorHandlerContext context, exception ); - return Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, null, null, context, exception)); + return Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, context.sourceRawKey(), context.sourceRawValue(), context, exception)); } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java b/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java index ba0ecebc1531e..084045ac45e54 100644 --- a/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java @@ -50,6 +50,8 @@ public void checkDealLetterQueueRecords() { final StringSerializer stringSerializer = new StringSerializer(); final StringDeserializer stringDeserializer = new StringDeserializer(); final MockRecordCollector collector = new MockRecordCollector(); + final String key = "key"; + final String value = "value"; final InternalProcessorContext internalProcessorContext = new InternalMockProcessorContext<>( new StateSerdes<>("sink", Serdes.ByteArray(), Serdes.ByteArray()), collector @@ -60,12 +62,14 @@ public void checkDealLetterQueueRecords() { 3, "source", new RecordHeaders(Collections.singletonList( - new RecordHeader("sourceHeader", stringSerializer.serialize(null, "hello world")))) + new RecordHeader("sourceHeader", stringSerializer.serialize(null, "hello world")))), + key.getBytes(), + value.getBytes() )); final ErrorHandlerContext errorHandlerContext = getErrorHandlerContext(internalProcessorContext); final NullPointerException exception = new NullPointerException("Oopsie!"); - final Iterable> dlqRecords = ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords("dlq", null, null, errorHandlerContext, exception); + final Iterable> dlqRecords = ExceptionHandlerUtils.maybeBuildDeadLetterQueueRecords("dlq", errorHandlerContext.sourceRawKey(), errorHandlerContext.sourceRawValue(), errorHandlerContext, exception); final Iterator> iterator = dlqRecords.iterator(); assertTrue(iterator.hasNext()); @@ -76,6 +80,8 @@ public void checkDealLetterQueueRecords() { assertEquals("dlq", dlqRecord.topic()); assertEquals(errorHandlerContext.timestamp(), dlqRecord.timestamp()); assertEquals(1, dlqRecord.timestamp()); + assertEquals(key, new String(dlqRecord.key())); + assertEquals(value, new String(dlqRecord.value())); assertEquals(exception.toString(), stringDeserializer.deserialize(null, headers.lastHeader(ExceptionHandlerUtils.HEADER_ERRORS_EXCEPTION_NAME).value())); assertEquals(exception.getMessage(), stringDeserializer.deserialize(null, headers.lastHeader(ExceptionHandlerUtils.HEADER_ERRORS_EXCEPTION_MESSAGE_NAME).value())); assertEquals("source", stringDeserializer.deserialize(null, headers.lastHeader(ExceptionHandlerUtils.HEADER_ERRORS_TOPIC_NAME).value())); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index 5270d00978ed6..6b7c06580c02e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -197,6 +197,8 @@ public void shouldBuildDeadLetterQueueRecordsInDefaultProcessingExceptionHandler assertEquals(1, collector.collected().size()); assertEquals("dlq", collector.collected().get(0).topic()); + assertEquals("sourceKey", new String((byte[]) collector.collected().get(0).key())); + assertEquals("sourceValue", new String((byte[]) collector.collected().get(0).value())); } @Test @@ -220,6 +222,8 @@ public void shouldBuildDeadLetterQueueRecordsInLogAndContinueProcessingException assertEquals(1, collector.collected().size()); assertEquals("dlq", collector.collected().get(0).topic()); + assertEquals("sourceKey", new String((byte[]) collector.collected().get(0).key())); + assertEquals("sourceValue", new String((byte[]) collector.collected().get(0).value())); } private static class ExceptionalProcessor implements Processor { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java index 49a5984758ad6..db726d78dcceb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java @@ -269,6 +269,8 @@ public void shouldBuildDeadLetterQueueRecordsInDefaultDeserializationException() assertEquals(1, collector.collected().size()); assertEquals("dlq", collector.collected().get(0).topic()); + assertEquals("hello", new String((byte[]) collector.collected().get(0).key())); + assertEquals("world", new String((byte[]) collector.collected().get(0).value())); } } @@ -307,6 +309,8 @@ public void shouldBuildDeadLetterQueueRecordsInLogAndContinueDeserializationExce assertEquals(1, collector.collected().size()); assertEquals("dlq", collector.collected().get(0).topic()); + assertEquals("hello", new String((byte[]) collector.collected().get(0).key())); + assertEquals("world", new String((byte[]) collector.collected().get(0).value())); } } From f1b8286f831170447c7cb7a2756c3ad269a6bede Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Wed, 16 Jul 2025 14:48:41 +0200 Subject: [PATCH 16/16] KFK-16505 --- .../ProcessingExceptionHandlerIntegrationTest.java | 2 +- .../java/org/apache/kafka/streams/StreamsConfig.java | 3 ++- .../errors/DefaultProductionExceptionHandler.java | 2 +- .../errors/DeserializationExceptionHandler.java | 6 +++--- .../streams/errors/ProcessingExceptionHandler.java | 6 +++--- .../streams/errors/ProductionExceptionHandler.java | 6 +++--- .../errors/internals/ExceptionHandlerUtils.java | 6 +++--- .../streams/processor/internals/ProcessorNode.java | 10 +++++----- .../processor/internals/RecordCollectorImpl.java | 4 ++-- .../streams/errors/ExceptionHandlerUtilsTest.java | 2 +- .../processor/internals/RecordCollectorTest.java | 4 ++-- 11 files changed, 26 insertions(+), 25 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java index 88da45d855fb0..4406f292205f8 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java @@ -357,7 +357,7 @@ public void shouldStopProcessingWhenProcessingExceptionHandlerReturnsNull() { final StreamsException e = assertThrows(StreamsException.class, () -> inputTopic.pipeInput(eventError.key, eventError.value, Instant.EPOCH)); assertEquals("Fatal user code error in processing error callback", e.getMessage()); assertInstanceOf(NullPointerException.class, e.getCause()); - assertEquals("Invalid ProcessingExceptionResponse response.", e.getCause().getMessage()); + assertEquals("Invalid ProcessingExceptionHandler response.", e.getCause().getMessage()); assertFalse(isExecuted.get()); } } 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 a25fa0b19c3c5..6364e7e912612 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -620,7 +620,8 @@ public class StreamsConfig extends AbstractConfig { "used. Otherwise, the classic group protocol will be used."; public static final String ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG = "errors.dead.letter.queue.topic.name"; - private static final String ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_DOC = "If not null, the default exception handler will build and send a Dead Letter Queue record in the provided topic name if an error occurs.\n" + + + private static final String ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_DOC = "If not null, the default exception handler will build and send a Dead Letter Queue record to the topic with the provided name if an error occurs.\n" + "If a custom deserialization/production or processing exception handler is set, this parameter is ignored for this handler."; /** {@code log.summary.interval.ms} */ 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 e796df783d678..3e9eb2fba86e0 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 @@ -38,7 +38,7 @@ public Response handleError(final ErrorHandlerContext context, final Exception exception) { return exception instanceof RetriableException ? Response.retry() : - Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, context.sourceRawKey(), context.sourceRawKey(), context, exception)); + Response.fail(maybeBuildDeadLetterQueueRecords(deadLetterQueueTopic, context.sourceRawKey(), context.sourceRawValue(), context, exception)); } @SuppressWarnings("rawtypes") 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 b2978b200f6ac..8c3667c20f498 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 @@ -171,16 +171,16 @@ private static DeserializationExceptionHandler.Result from(final Deserialization */ class Response { - private Result result; + private final Result result; - private List> deadLetterQueueRecords; + private final List> deadLetterQueueRecords; /** * Constructs a new {@code DeserializationExceptionResponse} object. * * @param result the result indicating whether processing should continue or fail; * must not be {@code null}. - * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. + * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. */ private Response(final Result result, final List> deadLetterQueueRecords) { 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 index 0e071ec484234..f4c32764877ec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java @@ -138,16 +138,16 @@ private static ProcessingExceptionHandler.Result from(final ProcessingHandlerRes */ class Response { - private Result result; + private final Result result; - private List> deadLetterQueueRecords; + private final List> deadLetterQueueRecords; /** * Constructs a new {@code ProcessingExceptionResponse} object. * * @param result the result indicating whether processing should continue or fail; * must not be {@code null}. - * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. + * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. */ private Response(final Result result, final List> deadLetterQueueRecords) { 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 a17c99c699997..717866a9bed60 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 @@ -279,16 +279,16 @@ enum SerializationExceptionOrigin { */ class Response { - private Result result; + private final Result result; - private List> deadLetterQueueRecords; + private final List> deadLetterQueueRecords; /** * Constructs a new {@code Response} object. * * @param result the result indicating whether processing should continue or fail; * must not be {@code null}. - * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. + * @param deadLetterQueueRecords the list of records to be sent to the dead letter queue; may be {@code null}. */ private Response(final Result result, final List> deadLetterQueueRecords) { diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/internals/ExceptionHandlerUtils.java b/streams/src/main/java/org/apache/kafka/streams/errors/internals/ExceptionHandlerUtils.java index 5764865c6d3e1..d3fd221cea8a5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/internals/ExceptionHandlerUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/internals/ExceptionHandlerUtils.java @@ -83,14 +83,14 @@ public static ProducerRecord buildDeadLetterQueueRecord(final St throw new InvalidConfigurationException(String.format("%s cannot be null while building dead letter queue record", StreamsConfig.ERRORS_DEAD_LETTER_QUEUE_TOPIC_NAME_CONFIG)); } final ProducerRecord producerRecord = new ProducerRecord<>(deadLetterQueueTopicName, null, context.timestamp(), key, value); - final StringWriter stackStraceStringWriter = new StringWriter(); - final PrintWriter stackTracePrintWriter = new PrintWriter(stackStraceStringWriter); + final StringWriter stackTraceStringWriter = new StringWriter(); + final PrintWriter stackTracePrintWriter = new PrintWriter(stackTraceStringWriter); e.printStackTrace(stackTracePrintWriter); try (final StringSerializer stringSerializer = new StringSerializer()) { producerRecord.headers().add(HEADER_ERRORS_EXCEPTION_NAME, stringSerializer.serialize(null, e.toString())); producerRecord.headers().add(HEADER_ERRORS_EXCEPTION_MESSAGE_NAME, stringSerializer.serialize(null, e.getMessage())); - producerRecord.headers().add(HEADER_ERRORS_STACKTRACE_NAME, stringSerializer.serialize(null, stackStraceStringWriter.toString())); + producerRecord.headers().add(HEADER_ERRORS_STACKTRACE_NAME, stringSerializer.serialize(null, stackTraceStringWriter.toString())); producerRecord.headers().add(HEADER_ERRORS_TOPIC_NAME, stringSerializer.serialize(null, context.topic())); producerRecord.headers().add(HEADER_ERRORS_PARTITION_NAME, stringSerializer.serialize(null, String.valueOf(context.partition()))); producerRecord.headers().add(HEADER_ERRORS_OFFSET_NAME, stringSerializer.serialize(null, String.valueOf(context.offset()))); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index aea1e272a59cf..bbf82ff9033ba 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -221,11 +221,11 @@ public void process(final Record record) { internalProcessorContext.recordContext().sourceRawValue() ); - final ProcessingExceptionHandler.Response processingExceptionResponse; + final ProcessingExceptionHandler.Response response; try { - processingExceptionResponse = Objects.requireNonNull( + response = Objects.requireNonNull( processingExceptionHandler.handleError(errorHandlerContext, record, processingException), - "Invalid ProcessingExceptionResponse response." + "Invalid ProcessingExceptionHandler response." ); } catch (final Exception fatalUserException) { // while Java distinguishes checked vs unchecked exceptions, other languages @@ -243,7 +243,7 @@ public void process(final Record record) { ); } - final List> deadLetterQueueRecords = processingExceptionResponse.deadLetterQueueRecords(); + final List> deadLetterQueueRecords = response.deadLetterQueueRecords(); if (!deadLetterQueueRecords.isEmpty()) { final RecordCollector collector = ((RecordCollector.Supplier) internalProcessorContext).recordCollector(); for (final ProducerRecord deadLetterQueueRecord : deadLetterQueueRecords) { @@ -257,7 +257,7 @@ public void process(final Record record) { } } - if (processingExceptionResponse.result() == ProcessingExceptionHandler.Result.FAIL) { + if (response.result() == ProcessingExceptionHandler.Result.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 " + diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index 118c27004933d..c4178733ef319 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -352,7 +352,7 @@ private void handleException(final ProductionExceptionHandler.Serializati serializationException, origin ), - "Invalid ProductionExceptionResponse response." + "Invalid ProductionExceptionHandler response." ); } catch (final Exception fatalUserException) { // while Java distinguishes checked vs unchecked exceptions, other languages @@ -491,7 +491,7 @@ private void recordSendError(final String topic, serializedRecord, productionException ), - "Invalid ProductionExceptionResponse response." + "Invalid ProductionExceptionHandler response." ); } catch (final Exception fatalUserException) { // while Java distinguishes checked vs unchecked exceptions, other languages diff --git a/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java b/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java index 084045ac45e54..915f3a3f650bb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/errors/ExceptionHandlerUtilsTest.java @@ -46,7 +46,7 @@ @ExtendWith(MockitoExtension.class) public class ExceptionHandlerUtilsTest { @Test - public void checkDealLetterQueueRecords() { + public void checkDeadLetterQueueRecords() { final StringSerializer stringSerializer = new StringSerializer(); final StringDeserializer stringDeserializer = new StringDeserializer(); final MockRecordCollector collector = new MockRecordCollector(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index cc49f71fea07b..64564278a18b1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -1715,7 +1715,7 @@ public void shouldThrowStreamsExceptionWhenSerializationFailedAndProductionExcep assertEquals("Fatal user code error in production error callback", exception.getMessage()); assertInstanceOf(NullPointerException.class, exception.getCause()); - assertEquals("Invalid ProductionExceptionResponse response.", exception.getCause().getMessage()); + assertEquals("Invalid ProductionExceptionHandler response.", exception.getCause().getMessage()); } } @@ -1764,7 +1764,7 @@ public void shouldThrowStreamsExceptionOnSubsequentFlushIfASendFailsAndProductio final StreamsException thrown = assertThrows(StreamsException.class, collector::flush); assertEquals("Fatal user code error in production error callback", thrown.getMessage()); assertInstanceOf(NullPointerException.class, thrown.getCause()); - assertEquals("Invalid ProductionExceptionResponse response.", thrown.getCause().getMessage()); + assertEquals("Invalid ProductionExceptionHandler response.", thrown.getCause().getMessage()); } @Test