From d0f6cc768f31f92c60185a408fa5ce4f9d9b7b1a Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Wed, 15 May 2024 10:15:28 +0200 Subject: [PATCH 01/28] KAFKA-16448: add new processor for error handling --- .../processor/ErrorHandlerContext.java | 122 ++++++++++++++++++ 1 file changed, 122 insertions(+) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java new file mode 100644 index 0000000000000..b6c222770953c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java @@ -0,0 +1,122 @@ +package org.apache.kafka.streams.processor; + +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; + +/** + * ErrorHandlerContext interface + */ +public interface ErrorHandlerContext { + + /** + * Return the topic name of the current input record; could be {@code null} if it is not + * available. + * + *

For example, if this method is invoked within a {@link Punctuator#punctuate(long) + * punctuation callback}, or while processing a record that was forwarded by a punctuation + * callback, the record won't have an associated topic. + * Another example is + * {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...)} + * (and siblings), that do not always guarantee to provide a valid topic name, as they might be + * executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL. + * + * @return the topic name + */ + String topic(); + + /** + * Return the partition id of the current input record; could be {@code -1} if it is not + * available. + * + *

For example, if this method is invoked within a {@link Punctuator#punctuate(long) + * punctuation callback}, or while processing a record that was forwarded by a punctuation + * callback, the record won't have an associated partition id. + * Another example is + * {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...)} + * (and siblings), that do not always guarantee to provide a valid partition id, as they might be + * executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL. + * + * @return the partition id + */ + int partition(); + + /** + * Return the offset of the current input record; could be {@code -1} if it is not + * available. + * + *

For example, if this method is invoked within a {@link Punctuator#punctuate(long) + * punctuation callback}, or while processing a record that was forwarded by a punctuation + * callback, the record won't have an associated offset. + * Another example is + * {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...)} + * (and siblings), that do not always guarantee to provide a valid offset, as they might be + * executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL. + * + * @return the offset + */ + long offset(); + + /** + * Return the headers of the current source record; could be an empty header if it is not + * available. + * + *

For example, if this method is invoked within a {@link Punctuator#punctuate(long) + * punctuation callback}, or while processing a record that was forwarded by a punctuation + * callback, the record might not have any associated headers. + * Another example is + * {@link org.apache.kafka.streams.kstream.KTable#transformValues(ValueTransformerWithKeySupplier, String...)} + * (and siblings), that do not always guarantee to provide valid headers, as they might be + * executed "out-of-band" due to some internal optimizations applied by the Kafka Streams DSL. + * + * @return the headers + */ + Headers headers(); + + /** + * Return the non-deserialized byte[] of the input message key if the context has been triggered by a message. + * + *

If this method is invoked within a {@link Punctuator#punctuate(long) + * punctuation callback}, or while processing a record that was forwarded by a punctuation + * callback, it will return null. + * + *

If this method is invoked in a sub-topology due to a repartition, the returned key would be one sent + * to the repartition topic. + * + *

Always returns null if this method is invoked within a + * {@link ProductionExceptionHandler.handle(ErrorHandlerContext, ProducerRecord, Exception, SerializationExceptionOrigin)} + * + * @return the raw byte of the key of the source message + */ + byte[] sourceRawKey(); + + /** + * Return the non-deserialized byte[] of the input message value if the context has been triggered by a message. + * + *

If this method is invoked within a {@link Punctuator#punctuate(long) + * punctuation callback}, or while processing a record that was forwarded by a punctuation + * callback, it will return null. + * + *

If this method is invoked in a sub-topology due to a repartition, the returned value would be one sent + * to the repartition topic. + * + *

Always returns null if this method is invoked within a + * {@link ProductionExceptionHandler.handle(ErrorHandlerContext, ProducerRecord, Exception, SerializationExceptionOrigin)} + * + * @return the raw byte of the value of the source message + */ + byte[] sourceRawValue(); + + /** + * Return the current processor node id. + * + * @return the processor node id + */ + String processorNodeId(); + + /** + * Return the task id. + * + * @return the task id + */ + TaskId taskId(); +} From bac5ed1ac4a24d1e48bdc64c7b4c18add5a7f23c Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Wed, 15 May 2024 10:37:25 +0200 Subject: [PATCH 02/28] KAFKA-16448: enum to get the origin of a SerializationException --- .../streams/errors/SerializationExceptionOrigin.java | 9 +++++++++ 1 file changed, 9 insertions(+) create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/SerializationExceptionOrigin.java diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/SerializationExceptionOrigin.java b/streams/src/main/java/org/apache/kafka/streams/errors/SerializationExceptionOrigin.java new file mode 100644 index 0000000000000..b90c3a3d078c9 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/SerializationExceptionOrigin.java @@ -0,0 +1,9 @@ +package org.apache.kafka.streams.errors; + +/** + * Indicates whether a {@link org.apache.kafka.common.errors.SerializationException} comes from the key or the value + */ +public enum SerializationExceptionOrigin { + KEY, + VALUE +} From c97cf4cd62de5830eff8f3dd537c1e15c5408d6a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Wed, 15 May 2024 11:04:33 +0200 Subject: [PATCH 03/28] KAFKA-16448: Add ProcessingExceptionHandler and update ProductionExceptionHandler --- .../errors/ProcessingExceptionHandler.java | 36 +++++++++++++ .../errors/ProductionExceptionHandler.java | 50 ++++++++++++++++++- 2 files changed, 84 insertions(+), 2 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java new file mode 100644 index 0000000000000..9d2712a2534f0 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java @@ -0,0 +1,36 @@ +package org.apache.kafka.streams.errors; + +import org.apache.kafka.common.Configurable; +import org.apache.kafka.streams.processor.ErrorHandlerContext; + +/** + * An interface that allows user code to inspect a record that has failed processing + */ +public interface ProcessingExceptionHandler extends Configurable { + /** + * Inspect a record and the exception received + * + * @param context processing context metadata + * @param record record where the exception occurred + * @param exception the actual exception + */ + ProcessingHandlerResponse handle(ErrorHandlerContext context, Record record, Exception exception); + + enum ProcessingHandlerResponse { + /* continue with processing */ + CONTINUE(1, "CONTINUE"), + /* fail the processing and stop */ + FAIL(2, "FAIL"); + + /** the permanent and immutable name of processing exception response */ + public final String name; + + /** the permanent and immutable id of processing exception response */ + public final int id; + + ProcessingHandlerResponse(final int id, final String name) { + this.id = id; + this.name = name; + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java index 6ae0170bfc906..30c9999d5f097 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Configurable; +import org.apache.kafka.streams.processor.ErrorHandlerContext; /** * Interface that specifies how an exception when attempting to produce a result to @@ -30,9 +31,28 @@ public interface ProductionExceptionHandler extends Configurable { * * @param record The record that failed to produce * @param exception The exception that occurred during production + * @deprecated Use {@link #handle(ErrorHandlerContext, ProducerRecord, Exception)} instead */ - ProductionExceptionHandlerResponse handle(final ProducerRecord record, - final Exception exception); + @Deprecated + default ProductionExceptionHandlerResponse handle(final ProducerRecord record, + final Exception exception) { + throw new UnsupportedOperationException(); + } + + /** + * Inspect a record that we attempted to produce, and the exception that resulted + * from attempting to produce it and determine whether or not to continue processing. + * + * @param context The error handler context metadata + * @param record The record that failed to produce + * @param exception The exception that occurred during production + */ + @SuppressWarnings("deprecation") + default ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception) { + return handle(record, exception); + } /** * Handles serialization exception and determine if the process should continue. The default implementation is to @@ -40,12 +60,31 @@ ProductionExceptionHandlerResponse handle(final ProducerRecord r * * @param record the record that failed to serialize * @param exception the exception that occurred during serialization + * @deprecated Use {@link #handle(ErrorHandlerContext, ProducerRecord, Exception)} instead */ + @Deprecated default ProductionExceptionHandlerResponse handleSerializationException(final ProducerRecord record, final Exception exception) { return ProductionExceptionHandlerResponse.FAIL; } + /** + * Handles serialization exception and determine if the process should continue. The default implementation is to + * fail the process. + * + * @param context the error handler context metadata + * @param record the record that failed to serialize + * @param exception the exception that occurred during serialization + * @param origin the origin of the serialization exception + */ + @SuppressWarnings("deprecation") + default ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception, + final SerializationExceptionOrigin origin) { + return handleSerializationException(record, exception); + } + enum ProductionExceptionHandlerResponse { /* continue processing */ CONTINUE(0, "CONTINUE"), @@ -68,4 +107,11 @@ enum ProductionExceptionHandlerResponse { this.name = name; } } + + enum SerializationExceptionOrigin { + /* serialization exception occurred during serialization of the key */ + KEY, + /* serialization exception occurred during serialization of the value */ + VALUE + } } From 962e2e204d153dbe4d2fba117230a484410ac6a6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Wed, 15 May 2024 11:21:09 +0200 Subject: [PATCH 04/28] KAFKA-16448: Add ProcessingExceptionHandler impls --- .../errors/ProcessingExceptionHandler.java | 3 +- ...cessingLogAndContinueExceptionHandler.java | 31 +++++++++++++++++++ .../ProcessingLogAndFailExceptionHandler.java | 31 +++++++++++++++++++ .../errors/ProductionExceptionHandler.java | 3 +- 4 files changed, 66 insertions(+), 2 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java 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 9d2712a2534f0..c24aa6c3bb19a 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 @@ -2,6 +2,7 @@ import org.apache.kafka.common.Configurable; import org.apache.kafka.streams.processor.ErrorHandlerContext; +import org.apache.kafka.streams.processor.api.Record; /** * An interface that allows user code to inspect a record that has failed processing @@ -14,7 +15,7 @@ public interface ProcessingExceptionHandler extends Configurable { * @param record record where the exception occurred * @param exception the actual exception */ - ProcessingHandlerResponse handle(ErrorHandlerContext context, Record record, Exception exception); + ProcessingHandlerResponse handle(ErrorHandlerContext context, Record record, Exception exception); enum ProcessingHandlerResponse { /* continue with processing */ diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java new file mode 100644 index 0000000000000..c934c5b09437a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java @@ -0,0 +1,31 @@ +package org.apache.kafka.streams.errors; + +import org.apache.kafka.streams.processor.ErrorHandlerContext; +import org.apache.kafka.streams.processor.api.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +/** + * Processing exception handler that logs a processing exception and then + * signals the processing pipeline to continue processing more records. + */ +public class ProcessingLogAndContinueExceptionHandler implements ProcessingExceptionHandler { + private static final Logger log = LoggerFactory.getLogger(ProcessingLogAndContinueExceptionHandler.class); + + @Override + public ProcessingHandlerResponse handle(ErrorHandlerContext context, Record record, Exception exception) { + log.warn("Exception caught during message processing, " + + "processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", + context.processorNodeId(), context.taskId(), context.topic(), context.partition(), context.offset(), + exception); + + return ProcessingHandlerResponse.CONTINUE; + } + + @Override + public void configure(Map configs) { + // ignore + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java new file mode 100644 index 0000000000000..9319b554f5edd --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java @@ -0,0 +1,31 @@ +package org.apache.kafka.streams.errors; + +import org.apache.kafka.streams.processor.ErrorHandlerContext; +import org.apache.kafka.streams.processor.api.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +/** + * Processing exception handler that logs a processing exception and then + * signals the processing pipeline to stop processing more records and fail. + */ +public class ProcessingLogAndFailExceptionHandler implements ProcessingExceptionHandler { + private static final Logger log = LoggerFactory.getLogger(ProcessingLogAndFailExceptionHandler.class); + + @Override + public ProcessingHandlerResponse handle(ErrorHandlerContext context, Record record, Exception exception) { + log.warn("Exception caught during message processing, " + + "processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", + context.processorNodeId(), context.taskId(), context.topic(), context.partition(), context.offset(), + exception); + + return ProcessingHandlerResponse.FAIL; + } + + @Override + public void configure(Map configs) { + // ignore + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java index 30c9999d5f097..915a6b13bb323 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,7 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Configurable; import org.apache.kafka.streams.processor.ErrorHandlerContext; +import sun.reflect.generics.reflectiveObjects.NotImplementedException; /** * Interface that specifies how an exception when attempting to produce a result to @@ -36,7 +37,7 @@ public interface ProductionExceptionHandler extends Configurable { @Deprecated default ProductionExceptionHandlerResponse handle(final ProducerRecord record, final Exception exception) { - throw new UnsupportedOperationException(); + throw new NotImplementedException(); } /** From 00e2bf38e36d0d99c0b7f7c13b38852a3ee96b74 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Wed, 15 May 2024 13:40:57 +0200 Subject: [PATCH 05/28] KAFKA-16448: Add error handler context impl --- .../DefaultProductionExceptionHandler.java | 4 +- .../processor/ErrorHandlerContextImpl.java | 72 +++++++++++++++++++ 2 files changed, 75 insertions(+), 1 deletion(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContextImpl.java diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/DefaultProductionExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/DefaultProductionExceptionHandler.java index 4fdb1a3fc0a95..a976346141277 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/DefaultProductionExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/DefaultProductionExceptionHandler.java @@ -18,6 +18,7 @@ import java.util.Map; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.streams.processor.ErrorHandlerContext; /** * {@code ProductionExceptionHandler} that always instructs streams to fail when an exception @@ -25,7 +26,8 @@ */ public class DefaultProductionExceptionHandler implements ProductionExceptionHandler { @Override - public ProductionExceptionHandlerResponse handle(final ProducerRecord record, + public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, + final ProducerRecord record, final Exception exception) { return ProductionExceptionHandlerResponse.FAIL; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContextImpl.java new file mode 100644 index 0000000000000..2f729daae87a5 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContextImpl.java @@ -0,0 +1,72 @@ +package org.apache.kafka.streams.processor; + +import org.apache.kafka.common.header.Headers; + +public class ErrorHandlerContextImpl implements ErrorHandlerContext { + private final String topic; + private final int partition; + private final long offset; + private final Headers headers; + private final byte[] sourceRawKey; + private final byte[] sourceRawValue; + private final String processorNodeId; + private final TaskId taskId; + + public ErrorHandlerContextImpl(String topic, + int partition, + long offset, + Headers headers, + byte[] sourceRawKey, + byte[] sourceRawValue, + String processorNodeId, + TaskId taskId) { + this.topic = topic; + this.partition = partition; + this.offset = offset; + this.headers = headers; + this.sourceRawKey = sourceRawKey; + this.sourceRawValue = sourceRawValue; + this.processorNodeId = processorNodeId; + this.taskId = taskId; + } + + @Override + public String topic() { + return topic; + } + + @Override + public int partition() { + return partition; + } + + @Override + public long offset() { + return offset; + } + + @Override + public Headers headers() { + return headers; + } + + @Override + public byte[] sourceRawKey() { + return sourceRawKey; + } + + @Override + public byte[] sourceRawValue() { + return sourceRawValue; + } + + @Override + public String processorNodeId() { + return processorNodeId; + } + + @Override + public TaskId taskId() { + return taskId; + } +} From d99b48fc79abe8d2c4c615d50d3dd5b50b9d43f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Wed, 15 May 2024 21:59:22 +0200 Subject: [PATCH 06/28] KAFKA-16448: Catch processing exceptions at Stream Task level --- .../apache/kafka/streams/StreamsConfig.java | 927 ++++++++++-------- .../apache/kafka/streams/TopologyConfig.java | 38 +- .../errors/ProcessingExceptionHandler.java | 24 +- ...cessingLogAndContinueExceptionHandler.java | 26 +- .../ProcessingLogAndFailExceptionHandler.java | 26 +- .../errors/ProductionExceptionHandler.java | 23 +- .../errors/SerializationExceptionOrigin.java | 9 - .../processor/ErrorHandlerContext.java | 16 + .../processor/ErrorHandlerContextImpl.java | 72 -- .../internals/ErrorHandlerContextImpl.java | 90 ++ .../internals/RecordCollectorImpl.java | 4 +- .../processor/internals/StreamTask.java | 34 +- ...aysContinueProductionExceptionHandler.java | 10 +- 13 files changed, 780 insertions(+), 519 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/SerializationExceptionOrigin.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContextImpl.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.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 38313467851fc..32f81b243b118 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.streams; -import java.util.Arrays; -import java.util.HashSet; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AdminClientConfig; @@ -39,6 +37,8 @@ import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.LogAndFailExceptionHandler; +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; +import org.apache.kafka.streams.errors.ProcessingLogAndFailExceptionHandler; import org.apache.kafka.streams.errors.ProductionExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.internals.StreamsConfigUtils; @@ -54,8 +54,10 @@ import java.io.File; import java.time.Duration; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -94,7 +96,7 @@ * * StreamsConfig streamsConfig = new StreamsConfig(streamsProperties); * } - * + *

* This instance can also be used to pass in custom configurations to different modules (e.g. passing a special config in your customized serde class). * The consumer/producer/admin prefix can also be used to distinguish these custom config values passed to different clients with the same config name. * * Example: @@ -112,28 +114,28 @@ * // and "my.custom.config2" -> "boom" from the config map * StreamsConfig streamsConfig = new StreamsConfig(streamsProperties); * } - * - * + *

+ *

* When increasing {@link ProducerConfig#MAX_BLOCK_MS_CONFIG} to be more resilient to non-available brokers you should also * increase {@link ConsumerConfig#MAX_POLL_INTERVAL_MS_CONFIG} using the following guidance: *

  *     max.poll.interval.ms > max.block.ms
  * 
- * - * + *

+ *

* Kafka Streams requires at least the following properties to be set: *

- * + *

* By default, Kafka Streams does not allow users to overwrite the following properties (Streams setting shown in parentheses): *

- * + *

* If {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} is set to {@link #EXACTLY_ONCE_V2 "exactly_once_v2"}, * {@link #EXACTLY_ONCE "exactly_once"} (deprecated), or {@link #EXACTLY_ONCE_BETA "exactly_once_beta"} (deprecated), Kafka Streams does not * allow users to overwrite the following properties (Streams setting shown in parentheses): @@ -240,13 +242,16 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String CLIENT_TAG_PREFIX = "client.tag."; - /** {@code topology.optimization} */ + /** + * {@code topology.optimization} + */ public static final String TOPOLOGY_OPTIMIZATION_CONFIG = "topology.optimization"; + private static final String CONFIG_ERROR_MSG = "Acceptable values are:" - + " \"+NO_OPTIMIZATION+\", \"+OPTIMIZE+\", " - + "or a comma separated list of specific optimizations: " - + "(\"+REUSE_KTABLE_SOURCE_TOPICS+\", \"+MERGE_REPARTITION_TOPICS+\" + " - + "\"SINGLE_STORE_SELF_JOIN+\")."; + + " \"+NO_OPTIMIZATION+\", \"+OPTIMIZE+\", " + + "or a comma separated list of specific optimizations: " + + "(\"+REUSE_KTABLE_SOURCE_TOPICS+\", \"+MERGE_REPARTITION_TOPICS+\" + " + + "\"SINGLE_STORE_SELF_JOIN+\")."; private static final String TOPOLOGY_OPTIMIZATION_DOC = "A configuration telling Kafka " + "Streams if it should optimize the topology and what optimizations to apply. " + CONFIG_ERROR_MSG @@ -469,61 +474,84 @@ public class StreamsConfig extends AbstractConfig { */ public static final String METRICS_LATEST = "latest"; - /** {@code acceptable.recovery.lag} */ + /** + * {@code acceptable.recovery.lag} + */ public static final String ACCEPTABLE_RECOVERY_LAG_CONFIG = "acceptable.recovery.lag"; private static final String ACCEPTABLE_RECOVERY_LAG_DOC = "The maximum acceptable lag (number of offsets to catch up) for a client to be considered caught-up enough" + - " to receive an active task assignment. Upon assignment, it will still restore the rest of the changelog" + - " before processing. To avoid a pause in processing during rebalances, this config" + - " should correspond to a recovery time of well under a minute for a given workload. Must be at least 0."; + " to receive an active task assignment. Upon assignment, it will still restore the rest of the changelog" + + " before processing. To avoid a pause in processing during rebalances, this config" + + " should correspond to a recovery time of well under a minute for a given workload. Must be at least 0."; - /** {@code application.id} */ + /** + * {@code application.id} + */ @SuppressWarnings("WeakerAccess") public static final String APPLICATION_ID_CONFIG = "application.id"; private static final String APPLICATION_ID_DOC = "An identifier for the stream processing application. Must be unique within the Kafka cluster. It is used as 1) the default client-id prefix, 2) the group-id for membership management, 3) the changelog topic prefix."; - /**{@code application.server} */ + /** + * {@code application.server} + */ @SuppressWarnings("WeakerAccess") public static final String APPLICATION_SERVER_CONFIG = "application.server"; private static final String APPLICATION_SERVER_DOC = "A host:port pair pointing to a user-defined endpoint that can be used for state store discovery and interactive queries on this KafkaStreams instance."; - /** {@code bootstrap.servers} */ + /** + * {@code bootstrap.servers} + */ @SuppressWarnings("WeakerAccess") public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; - /** {@code buffered.records.per.partition} */ + /** + * {@code buffered.records.per.partition} + */ @SuppressWarnings("WeakerAccess") public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition"; public static final String BUFFERED_RECORDS_PER_PARTITION_DOC = "Maximum number of records to buffer per partition."; - /** {@code built.in.metrics.version} */ + /** + * {@code built.in.metrics.version} + */ public static final String BUILT_IN_METRICS_VERSION_CONFIG = "built.in.metrics.version"; private static final String BUILT_IN_METRICS_VERSION_DOC = "Version of the built-in metrics to use."; - /** {@code cache.max.bytes.buffering} - * @deprecated since 3.4.0 Use {@link #STATESTORE_CACHE_MAX_BYTES_CONFIG "statestore.cache.max.bytes"} instead. */ + /** + * {@code cache.max.bytes.buffering} + * + * @deprecated since 3.4.0 Use {@link #STATESTORE_CACHE_MAX_BYTES_CONFIG "statestore.cache.max.bytes"} instead. + */ @SuppressWarnings("WeakerAccess") @Deprecated public static final String CACHE_MAX_BYTES_BUFFERING_CONFIG = "cache.max.bytes.buffering"; public static final String CACHE_MAX_BYTES_BUFFERING_DOC = "Maximum number of memory bytes to be used for buffering across all threads"; - /** {@code statestore.cache.max.bytes} */ + /** + * {@code statestore.cache.max.bytes} + */ @SuppressWarnings("WeakerAccess") public static final String STATESTORE_CACHE_MAX_BYTES_CONFIG = "statestore.cache.max.bytes"; public static final String STATESTORE_CACHE_MAX_BYTES_DOC = "Maximum number of memory bytes to be used for statestore cache across all threads"; - /** {@code client.id} */ + /** + * {@code client.id} + */ @SuppressWarnings("WeakerAccess") public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; private static final String CLIENT_ID_DOC = "An ID prefix string used for the client IDs of internal (main, restore, and global) consumers , producers, and admin clients" + " with pattern <client.id>-[Global]StreamThread[-<threadSequenceNumber>]-<consumer|producer|restore-consumer|global-consumer>."; - /** {@code enable.metrics.push} */ + /** + * {@code enable.metrics.push} + */ @SuppressWarnings("WeakerAccess") - public static final String ENABLE_METRICS_PUSH_CONFIG = CommonClientConfigs.ENABLE_METRICS_PUSH_CONFIG; - public static final String ENABLE_METRICS_PUSH_DOC = "Whether to enable pushing of internal client metrics for (main, restore, and global) consumers, producers, and admin clients." + + public static final String ENABLE_METRICS_PUSH_CONFIG = CommonClientConfigs.ENABLE_METRICS_PUSH_CONFIG; + public static final String ENABLE_METRICS_PUSH_DOC = "Whether to enable pushing of internal client metrics for (main, restore, and global) consumers, producers, and admin clients." + " The cluster must have a client metrics subscription which corresponds to a client."; - /** {@code commit.interval.ms} */ + /** + * {@code commit.interval.ms} + */ @SuppressWarnings("WeakerAccess") public static final String COMMIT_INTERVAL_MS_CONFIG = "commit.interval.ms"; private static final String COMMIT_INTERVAL_MS_DOC = "The frequency in milliseconds with which to commit processing progress." + @@ -532,28 +560,45 @@ public class StreamsConfig extends AbstractConfig { " (Note, if processing.guarantee is set to " + EXACTLY_ONCE_V2 + ", " + EXACTLY_ONCE + ",the default value is " + EOS_DEFAULT_COMMIT_INTERVAL_MS + "," + " otherwise the default value is " + DEFAULT_COMMIT_INTERVAL_MS + "."; - /** {@code repartition.purge.interval.ms} */ + /** + * {@code repartition.purge.interval.ms} + */ @SuppressWarnings("WeakerAccess") public static final String REPARTITION_PURGE_INTERVAL_MS_CONFIG = "repartition.purge.interval.ms"; private static final String REPARTITION_PURGE_INTERVAL_MS_DOC = "The frequency in milliseconds with which to delete fully consumed records from repartition topics." + - " Purging will occur after at least this value since the last purge, but may be delayed until later." + - " (Note, unlike commit.interval.ms, the default for this value remains unchanged when processing.guarantee is set to " + EXACTLY_ONCE_V2 + ")."; + " Purging will occur after at least this value since the last purge, but may be delayed until later." + + " (Note, unlike commit.interval.ms, the default for this value remains unchanged when processing.guarantee is set to " + EXACTLY_ONCE_V2 + ")."; - /** {@code connections.max.idle.ms} */ + /** + * {@code connections.max.idle.ms} + */ @SuppressWarnings("WeakerAccess") public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; - /** {@code default.deserialization.exception.handler} */ + /** + * {@code default.deserialization.exception.handler} + */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG = "default.deserialization.exception.handler"; public static final String DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the org.apache.kafka.streams.errors.DeserializationExceptionHandler interface."; - /** {@code default.production.exception.handler} */ + /** + * {@code default.production.exception.handler} + */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG = "default.production.exception.handler"; private static final String DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the org.apache.kafka.streams.errors.ProductionExceptionHandler interface."; - /** {@code default.dsl.store} */ + /** + * {@code default.deserialization.exception.handler} + */ + @SuppressWarnings("WeakerAccess") + public static final String PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG = "processing.exception.handler"; + public static final String PROCESSING_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the org.apache.kafka.streams.errors.ProcessingExceptionHandler interface."; + + /** + * {@code default.dsl.store} + */ @Deprecated @SuppressWarnings("WeakerAccess") public static final String DEFAULT_DSL_STORE_CONFIG = "default.dsl.store"; @@ -567,21 +612,29 @@ public class StreamsConfig extends AbstractConfig { @Deprecated public static final String DEFAULT_DSL_STORE = ROCKS_DB; - /** {@code dsl.store.suppliers.class } */ + /** + * {@code dsl.store.suppliers.class } + */ public static final String DSL_STORE_SUPPLIERS_CLASS_CONFIG = "dsl.store.suppliers.class"; static final String DSL_STORE_SUPPLIERS_CLASS_DOC = "Defines which store implementations to plug in to DSL operators. Must implement the org.apache.kafka.streams.state.DslStoreSuppliers interface."; static final Class DSL_STORE_SUPPLIERS_CLASS_DEFAULT = BuiltInDslStoreSuppliers.RocksDBDslStoreSuppliers.class; - /** {@code default.windowed.key.serde.inner - * @deprecated since 3.0.0 Use {@link #WINDOWED_INNER_CLASS_SERDE "windowed.inner.class.serde"} instead.} */ + /** + * {@code default.windowed.key.serde.inner + * + * @deprecated since 3.0.0 Use {@link #WINDOWED_INNER_CLASS_SERDE "windowed.inner.class.serde"} instead.} + */ @SuppressWarnings("WeakerAccess") @Deprecated public static final String DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS = "default.windowed.key.serde.inner"; private static final String DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS_DOC = "Default serializer / deserializer for the inner class of a windowed key. Must implement the " + "org.apache.kafka.common.serialization.Serde interface."; - /** {@code default.windowed.value.serde.inner - * @deprecated since 3.0.0 Use {@link #WINDOWED_INNER_CLASS_SERDE "windowed.inner.class.serde"} instead.} */ + /** + * {@code default.windowed.value.serde.inner + * + * @deprecated since 3.0.0 Use {@link #WINDOWED_INNER_CLASS_SERDE "windowed.inner.class.serde"} instead.} + */ @SuppressWarnings("WeakerAccess") @Deprecated public static final String DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS = "default.windowed.value.serde.inner"; @@ -592,27 +645,35 @@ public class StreamsConfig extends AbstractConfig { private static final String WINDOWED_INNER_CLASS_SERDE_DOC = " Default serializer / deserializer for the inner class of a windowed record. Must implement the " + "org.apache.kafka.common.serialization.Serde interface. Note that setting this config in KafkaStreams application would result " + "in an error as it is meant to be used only from Plain consumer client."; - - /** {@code default key.serde} */ + + /** + * {@code default key.serde} + */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_KEY_SERDE_CLASS_CONFIG = "default.key.serde"; private static final String DEFAULT_KEY_SERDE_CLASS_DOC = "Default serializer / deserializer class for key that implements the org.apache.kafka.common.serialization.Serde interface. " - + "Note when windowed serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" - + DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS + "' as well"; + + "Note when windowed serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + + DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS + "' as well"; - /** {@code default value.serde} */ + /** + * {@code default value.serde} + */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_VALUE_SERDE_CLASS_CONFIG = "default.value.serde"; private static final String DEFAULT_VALUE_SERDE_CLASS_DOC = "Default serializer / deserializer class for value that implements the org.apache.kafka.common.serialization.Serde interface. " - + "Note when windowed serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" - + DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS + "' as well"; + + "Note when windowed serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + + DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS + "' as well"; - /** {@code default.timestamp.extractor} */ + /** + * {@code default.timestamp.extractor} + */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG = "default.timestamp.extractor"; public static final String DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC = "Default timestamp extractor class that implements the org.apache.kafka.streams.processor.TimestampExtractor interface."; - /** {@code max.task.idle.ms} */ + /** + * {@code max.task.idle.ms} + */ public static final String MAX_TASK_IDLE_MS_CONFIG = "max.task.idle.ms"; public static final String MAX_TASK_IDLE_MS_DOC = "This config controls whether joins and merges" + " may produce out-of-order results." @@ -627,62 +688,87 @@ public class StreamsConfig extends AbstractConfig { + " Set to -1 to disable idling entirely and process any locally available data," + " even though doing so may produce out-of-order processing."; - /** {@code max.warmup.replicas} */ + /** + * {@code max.warmup.replicas} + */ public static final String MAX_WARMUP_REPLICAS_CONFIG = "max.warmup.replicas"; private static final String MAX_WARMUP_REPLICAS_DOC = "The maximum number of warmup replicas (extra standbys beyond the configured num.standbys) that can be assigned at once for the purpose of keeping " + - " the task available on one instance while it is warming up on another instance it has been reassigned to. Used to throttle how much extra broker " + - " traffic and cluster state can be used for high availability. Must be at least 1." + - "Note that one warmup replica corresponds to one Stream Task. Furthermore, note that each warmup replica can only be promoted to an active task " + - "during a rebalance (normally during a so-called probing rebalance, which occur at a frequency specified by the `probing.rebalance.interval.ms` config). This means " + - "that the maximum rate at which active tasks can be migrated from one Kafka Streams Instance to another instance can be determined by " + - "(`max.warmup.replicas` / `probing.rebalance.interval.ms`)."; + " the task available on one instance while it is warming up on another instance it has been reassigned to. Used to throttle how much extra broker " + + " traffic and cluster state can be used for high availability. Must be at least 1." + + "Note that one warmup replica corresponds to one Stream Task. Furthermore, note that each warmup replica can only be promoted to an active task " + + "during a rebalance (normally during a so-called probing rebalance, which occur at a frequency specified by the `probing.rebalance.interval.ms` config). This means " + + "that the maximum rate at which active tasks can be migrated from one Kafka Streams Instance to another instance can be determined by " + + "(`max.warmup.replicas` / `probing.rebalance.interval.ms`)."; - /** {@code metadata.max.age.ms} */ + /** + * {@code metadata.max.age.ms} + */ @SuppressWarnings("WeakerAccess") public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; - /** {@code metrics.num.samples} */ + /** + * {@code metrics.num.samples} + */ @SuppressWarnings("WeakerAccess") public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG; - /** {@code metrics.record.level} */ + /** + * {@code metrics.record.level} + */ @SuppressWarnings("WeakerAccess") public static final String METRICS_RECORDING_LEVEL_CONFIG = CommonClientConfigs.METRICS_RECORDING_LEVEL_CONFIG; - /** {@code metric.reporters} */ + /** + * {@code metric.reporters} + */ @SuppressWarnings("WeakerAccess") public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG; - /** {@code metrics.sample.window.ms} */ + /** + * {@code metrics.sample.window.ms} + */ @SuppressWarnings("WeakerAccess") public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG; - /** {@code auto.include.jmx.reporter - * @deprecated and will removed in 4.0.0 Use {@link JMX_REPORTER "jmx.reporter"} instead.} */ + /** + * {@code auto.include.jmx.reporter + * + * @deprecated and will removed in 4.0.0 Use {@link JMX_REPORTER "jmx.reporter"} instead.} + */ @Deprecated public static final String AUTO_INCLUDE_JMX_REPORTER_CONFIG = CommonClientConfigs.AUTO_INCLUDE_JMX_REPORTER_CONFIG; - /** {@code num.standby.replicas} */ + /** + * {@code num.standby.replicas} + */ @SuppressWarnings("WeakerAccess") public static final String NUM_STANDBY_REPLICAS_CONFIG = "num.standby.replicas"; private static final String NUM_STANDBY_REPLICAS_DOC = "The number of standby replicas for each task."; - /** {@code num.stream.threads} */ + /** + * {@code num.stream.threads} + */ @SuppressWarnings("WeakerAccess") public static final String NUM_STREAM_THREADS_CONFIG = "num.stream.threads"; private static final String NUM_STREAM_THREADS_DOC = "The number of threads to execute stream processing."; - /** {@code poll.ms} */ + /** + * {@code poll.ms} + */ @SuppressWarnings("WeakerAccess") public static final String POLL_MS_CONFIG = "poll.ms"; private static final String POLL_MS_DOC = "The amount of time in milliseconds to block waiting for input."; - /** {@code probing.rebalance.interval.ms} */ + /** + * {@code probing.rebalance.interval.ms} + */ public static final String PROBING_REBALANCE_INTERVAL_MS_CONFIG = "probing.rebalance.interval.ms"; private static final String PROBING_REBALANCE_INTERVAL_MS_DOC = "The maximum time in milliseconds to wait before triggering a rebalance to probe for warmup replicas that have finished warming up and are ready to become active." + " Probing rebalances will continue to be triggered until the assignment is balanced. Must be at least 1 minute."; - /** {@code processing.guarantee} */ + /** + * {@code processing.guarantee} + */ @SuppressWarnings("WeakerAccess") public static final String PROCESSING_GUARANTEE_CONFIG = "processing.guarantee"; private static final String PROCESSING_GUARANTEE_DOC = "The processing guarantee that should be used. " + @@ -694,32 +780,44 @@ public class StreamsConfig extends AbstractConfig { "recommended setting for production; for development you can change this, by adjusting broker setting " + "transaction.state.log.replication.factor and transaction.state.log.min.isr."; - /** {@code receive.buffer.bytes} */ + /** + * {@code receive.buffer.bytes} + */ @SuppressWarnings("WeakerAccess") public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG; - /** {@code rack.aware.assignment.tags} */ + /** + * {@code rack.aware.assignment.tags} + */ @SuppressWarnings("WeakerAccess") public static final String RACK_AWARE_ASSIGNMENT_TAGS_CONFIG = "rack.aware.assignment.tags"; private static final String RACK_AWARE_ASSIGNMENT_TAGS_DOC = "List of client tag keys used to distribute standby replicas across Kafka Streams instances." + - " When configured, Kafka Streams will make a best-effort to distribute" + - " the standby tasks over each client tag dimension."; + " When configured, Kafka Streams will make a best-effort to distribute" + + " the standby tasks over each client tag dimension."; - /** {@code reconnect.backoff.ms} */ + /** + * {@code reconnect.backoff.ms} + */ @SuppressWarnings("WeakerAccess") public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; - /** {@code reconnect.backoff.max} */ + /** + * {@code reconnect.backoff.max} + */ @SuppressWarnings("WeakerAccess") public static final String RECONNECT_BACKOFF_MAX_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG; - /** {@code replication.factor} */ + /** + * {@code replication.factor} + */ @SuppressWarnings("WeakerAccess") public static final String REPLICATION_FACTOR_CONFIG = "replication.factor"; private static final String REPLICATION_FACTOR_DOC = "The replication factor for change log topics and repartition topics created by the stream processing application." + " The default of -1 (meaning: use broker default replication factor) requires broker version 2.4 or newer"; - /** {@code request.timeout.ms} */ + /** + * {@code request.timeout.ms} + */ @SuppressWarnings("WeakerAccess") public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG; @@ -734,45 +832,63 @@ public class StreamsConfig extends AbstractConfig { @Deprecated public static final String RETRIES_CONFIG = CommonClientConfigs.RETRIES_CONFIG; - /** {@code retry.backoff.ms} */ + /** + * {@code retry.backoff.ms} + */ @SuppressWarnings("WeakerAccess") public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG; - /** {@code rocksdb.config.setter} */ + /** + * {@code rocksdb.config.setter} + */ @SuppressWarnings("WeakerAccess") public static final String ROCKSDB_CONFIG_SETTER_CLASS_CONFIG = "rocksdb.config.setter"; private static final String ROCKSDB_CONFIG_SETTER_CLASS_DOC = "A Rocks DB config setter class or class name that implements the org.apache.kafka.streams.state.RocksDBConfigSetter interface"; - /** {@code security.protocol} */ + /** + * {@code security.protocol} + */ @SuppressWarnings("WeakerAccess") public static final String SECURITY_PROTOCOL_CONFIG = CommonClientConfigs.SECURITY_PROTOCOL_CONFIG; - /** {@code send.buffer.bytes} */ + /** + * {@code send.buffer.bytes} + */ @SuppressWarnings("WeakerAccess") public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG; - /** {@code state.cleanup.delay} */ + /** + * {@code state.cleanup.delay} + */ @SuppressWarnings("WeakerAccess") public static final String STATE_CLEANUP_DELAY_MS_CONFIG = "state.cleanup.delay.ms"; private static final String STATE_CLEANUP_DELAY_MS_DOC = "The amount of time in milliseconds to wait before deleting state when a partition has migrated. Only state directories that have not been modified for at least state.cleanup.delay.ms will be removed"; - /** {@code state.dir} */ + /** + * {@code state.dir} + */ @SuppressWarnings("WeakerAccess") public static final String STATE_DIR_CONFIG = "state.dir"; private static final String STATE_DIR_DOC = "Directory location for state store. This path must be unique for each streams instance sharing the same underlying filesystem. Note that if not configured, then the default location will be different in each environment as it is computed using System.getProperty(\"java.io.tmpdir\")"; - /** {@code task.timeout.ms} */ + /** + * {@code task.timeout.ms} + */ public static final String TASK_TIMEOUT_MS_CONFIG = "task.timeout.ms"; public static final String TASK_TIMEOUT_MS_DOC = "The maximum amount of time in milliseconds a task might stall due to internal errors and retries until an error is raised. " + "For a timeout of 0ms, a task would raise an error for the first internal error. " + "For any timeout larger than 0ms, a task will retry at least once before an error is raised."; - /** {@code window.size.ms} */ + /** + * {@code window.size.ms} + */ public static final String WINDOW_SIZE_MS_CONFIG = "window.size.ms"; private static final String WINDOW_SIZE_MS_DOC = "Sets window size for the deserializer in order to calculate window end times."; - /** {@code upgrade.from} */ + /** + * {@code upgrade.from} + */ @SuppressWarnings("WeakerAccess") public static final String UPGRADE_FROM_CONFIG = "upgrade.from"; private static final String UPGRADE_FROM_DOC = "Allows upgrading in a backward compatible way. " + @@ -787,12 +903,16 @@ public class StreamsConfig extends AbstractConfig { UPGRADE_FROM_32 + "\", \"" + UPGRADE_FROM_33 + "\", \"" + UPGRADE_FROM_34 + "\", \"" + UPGRADE_FROM_35 + "\", \"" + UPGRADE_FROM_36 + "\", \"" + UPGRADE_FROM_37 + "(for upgrading from the corresponding old version)."; - /** {@code windowstore.changelog.additional.retention.ms} */ + /** + * {@code windowstore.changelog.additional.retention.ms} + */ @SuppressWarnings("WeakerAccess") public static final String WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG = "windowstore.changelog.additional.retention.ms"; private static final String WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC = "Added to a windows maintainMs to ensure data is not deleted from the log prematurely. Allows for clock drift. Default is 1 day"; - /** {@code default.client.supplier} */ + /** + * {@code default.client.supplier} + */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_CLIENT_SUPPLIER_CONFIG = "default.client.supplier"; public static final String DEFAULT_CLIENT_SUPPLIER_DOC = "Client supplier class that implements the org.apache.kafka.streams.KafkaClientSupplier interface."; @@ -801,7 +921,9 @@ public class StreamsConfig extends AbstractConfig { public static final String RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC = "min_traffic"; public static final String RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY = "balance_subtopology"; - /** {@code } rack.aware.assignment.strategy */ + /** + * {@code } rack.aware.assignment.strategy + */ @SuppressWarnings("WeakerAccess") public static final String RACK_AWARE_ASSIGNMENT_STRATEGY_CONFIG = "rack.aware.assignment.strategy"; public static final String RACK_AWARE_ASSIGNMENT_STRATEGY_DOC = "The strategy we use for rack aware assignment. Rack aware assignment will take client.rack and racks of TopicPartition into account when assigning" @@ -823,6 +945,7 @@ public class StreamsConfig extends AbstractConfig { /** * {@code topology.optimization} + * * @deprecated since 2.7; use {@link #TOPOLOGY_OPTIMIZATION_CONFIG} instead */ @Deprecated @@ -846,349 +969,355 @@ public class StreamsConfig extends AbstractConfig { // HIGH .define(APPLICATION_ID_CONFIG, // required with no default value - Type.STRING, - Importance.HIGH, - APPLICATION_ID_DOC) + Type.STRING, + Importance.HIGH, + APPLICATION_ID_DOC) .define(BOOTSTRAP_SERVERS_CONFIG, // required with no default value - Type.LIST, - Importance.HIGH, - CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) + Type.LIST, + Importance.HIGH, + CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(NUM_STANDBY_REPLICAS_CONFIG, - Type.INT, - 0, - Importance.HIGH, - NUM_STANDBY_REPLICAS_DOC) + Type.INT, + 0, + Importance.HIGH, + NUM_STANDBY_REPLICAS_DOC) .define(STATE_DIR_CONFIG, - Type.STRING, - System.getProperty("java.io.tmpdir") + File.separator + "kafka-streams", - Importance.HIGH, - STATE_DIR_DOC, - "${java.io.tmpdir}") + Type.STRING, + System.getProperty("java.io.tmpdir") + File.separator + "kafka-streams", + Importance.HIGH, + STATE_DIR_DOC, + "${java.io.tmpdir}") // MEDIUM .define(ACCEPTABLE_RECOVERY_LAG_CONFIG, - Type.LONG, - 10_000L, - atLeast(0), - Importance.MEDIUM, - ACCEPTABLE_RECOVERY_LAG_DOC) + Type.LONG, + 10_000L, + atLeast(0), + Importance.MEDIUM, + ACCEPTABLE_RECOVERY_LAG_DOC) .define(CACHE_MAX_BYTES_BUFFERING_CONFIG, - Type.LONG, - 10 * 1024 * 1024L, - atLeast(0), - Importance.MEDIUM, - CACHE_MAX_BYTES_BUFFERING_DOC) + Type.LONG, + 10 * 1024 * 1024L, + atLeast(0), + Importance.MEDIUM, + CACHE_MAX_BYTES_BUFFERING_DOC) .define(STATESTORE_CACHE_MAX_BYTES_CONFIG, - Type.LONG, - 10 * 1024 * 1024L, - atLeast(0), - Importance.MEDIUM, - STATESTORE_CACHE_MAX_BYTES_DOC) + Type.LONG, + 10 * 1024 * 1024L, + atLeast(0), + Importance.MEDIUM, + STATESTORE_CACHE_MAX_BYTES_DOC) .define(CLIENT_ID_CONFIG, - Type.STRING, - "", - Importance.MEDIUM, - CLIENT_ID_DOC, - "<application.id>-<random-UUID>") + Type.STRING, + "", + Importance.MEDIUM, + CLIENT_ID_DOC, + "<application.id>-<random-UUID>") .define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, - Type.CLASS, - LogAndFailExceptionHandler.class.getName(), - Importance.MEDIUM, - DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC) + Type.CLASS, + LogAndFailExceptionHandler.class.getName(), + Importance.MEDIUM, + DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC) .define(DEFAULT_KEY_SERDE_CLASS_CONFIG, - Type.CLASS, - null, - Importance.MEDIUM, - DEFAULT_KEY_SERDE_CLASS_DOC) + Type.CLASS, + null, + Importance.MEDIUM, + DEFAULT_KEY_SERDE_CLASS_DOC) .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC) + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC) .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC) + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC) .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC) + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC) .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC) + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC) .define(DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, - Type.CLASS, - DefaultProductionExceptionHandler.class.getName(), - Importance.MEDIUM, - DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC) + Type.CLASS, + DefaultProductionExceptionHandler.class.getName(), + Importance.MEDIUM, + DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC) + .define(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, + Type.CLASS, + ProcessingLogAndFailExceptionHandler.class.getName(), + Importance.MEDIUM, + PROCESSING_EXCEPTION_HANDLER_CLASS_DOC) .define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, - Type.CLASS, - FailOnInvalidTimestamp.class.getName(), - Importance.MEDIUM, - DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC) + Type.CLASS, + FailOnInvalidTimestamp.class.getName(), + Importance.MEDIUM, + DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC) .define(DEFAULT_VALUE_SERDE_CLASS_CONFIG, - Type.CLASS, - null, - Importance.MEDIUM, - DEFAULT_VALUE_SERDE_CLASS_DOC) + Type.CLASS, + null, + Importance.MEDIUM, + DEFAULT_VALUE_SERDE_CLASS_DOC) .define(MAX_TASK_IDLE_MS_CONFIG, - Type.LONG, - 0L, - Importance.MEDIUM, - MAX_TASK_IDLE_MS_DOC) + Type.LONG, + 0L, + Importance.MEDIUM, + MAX_TASK_IDLE_MS_DOC) .define(MAX_WARMUP_REPLICAS_CONFIG, - Type.INT, - 2, - atLeast(1), - Importance.MEDIUM, - MAX_WARMUP_REPLICAS_DOC) + Type.INT, + 2, + atLeast(1), + Importance.MEDIUM, + MAX_WARMUP_REPLICAS_DOC) .define(NUM_STREAM_THREADS_CONFIG, - Type.INT, - 1, - Importance.MEDIUM, - NUM_STREAM_THREADS_DOC) + Type.INT, + 1, + Importance.MEDIUM, + NUM_STREAM_THREADS_DOC) .define(PROCESSING_GUARANTEE_CONFIG, - Type.STRING, - AT_LEAST_ONCE, - in(AT_LEAST_ONCE, EXACTLY_ONCE, EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2), - Importance.MEDIUM, - PROCESSING_GUARANTEE_DOC) + Type.STRING, + AT_LEAST_ONCE, + in(AT_LEAST_ONCE, EXACTLY_ONCE, EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2), + Importance.MEDIUM, + PROCESSING_GUARANTEE_DOC) .define(RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG, - Type.INT, - null, - Importance.MEDIUM, - RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_DOC) + Type.INT, + null, + Importance.MEDIUM, + RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_DOC) .define(RACK_AWARE_ASSIGNMENT_STRATEGY_CONFIG, - Type.STRING, - RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, - in(RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC, RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY), - Importance.MEDIUM, - RACK_AWARE_ASSIGNMENT_STRATEGY_DOC) + Type.STRING, + RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, + in(RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC, RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY), + Importance.MEDIUM, + RACK_AWARE_ASSIGNMENT_STRATEGY_DOC) .define(RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, - Type.LIST, - Collections.emptyList(), - atMostOfSize(MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE), - Importance.MEDIUM, - RACK_AWARE_ASSIGNMENT_TAGS_DOC) + Type.LIST, + Collections.emptyList(), + atMostOfSize(MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE), + Importance.MEDIUM, + RACK_AWARE_ASSIGNMENT_TAGS_DOC) .define(RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG, - Type.INT, - null, - Importance.MEDIUM, - RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_DOC) + Type.INT, + null, + Importance.MEDIUM, + RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_DOC) .define(REPLICATION_FACTOR_CONFIG, - Type.INT, - -1, - Importance.MEDIUM, - REPLICATION_FACTOR_DOC) + Type.INT, + -1, + Importance.MEDIUM, + REPLICATION_FACTOR_DOC) .define(SECURITY_PROTOCOL_CONFIG, - Type.STRING, - CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL, - ConfigDef.CaseInsensitiveValidString.in(Utils.enumOptions(SecurityProtocol.class)), - Importance.MEDIUM, - CommonClientConfigs.SECURITY_PROTOCOL_DOC) + Type.STRING, + CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL, + ConfigDef.CaseInsensitiveValidString.in(Utils.enumOptions(SecurityProtocol.class)), + Importance.MEDIUM, + CommonClientConfigs.SECURITY_PROTOCOL_DOC) .define(TASK_TIMEOUT_MS_CONFIG, - Type.LONG, - Duration.ofMinutes(5L).toMillis(), - atLeast(0L), - Importance.MEDIUM, - TASK_TIMEOUT_MS_DOC) + Type.LONG, + Duration.ofMinutes(5L).toMillis(), + atLeast(0L), + Importance.MEDIUM, + TASK_TIMEOUT_MS_DOC) .define(TOPOLOGY_OPTIMIZATION_CONFIG, - Type.STRING, - NO_OPTIMIZATION, - (name, value) -> verifyTopologyOptimizationConfigs((String) value), - Importance.MEDIUM, - TOPOLOGY_OPTIMIZATION_DOC) + Type.STRING, + NO_OPTIMIZATION, + (name, value) -> verifyTopologyOptimizationConfigs((String) value), + Importance.MEDIUM, + TOPOLOGY_OPTIMIZATION_DOC) // LOW .define(APPLICATION_SERVER_CONFIG, - Type.STRING, - "", - Importance.LOW, - APPLICATION_SERVER_DOC) + Type.STRING, + "", + Importance.LOW, + APPLICATION_SERVER_DOC) .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, - Type.INT, - 1000, - Importance.LOW, - BUFFERED_RECORDS_PER_PARTITION_DOC) + Type.INT, + 1000, + Importance.LOW, + BUFFERED_RECORDS_PER_PARTITION_DOC) .define(BUILT_IN_METRICS_VERSION_CONFIG, - Type.STRING, - METRICS_LATEST, - in( - METRICS_LATEST - ), - Importance.LOW, - BUILT_IN_METRICS_VERSION_DOC) + Type.STRING, + METRICS_LATEST, + in( + METRICS_LATEST + ), + Importance.LOW, + BUILT_IN_METRICS_VERSION_DOC) .define(COMMIT_INTERVAL_MS_CONFIG, - Type.LONG, - DEFAULT_COMMIT_INTERVAL_MS, - atLeast(0), - Importance.LOW, - COMMIT_INTERVAL_MS_DOC) + Type.LONG, + DEFAULT_COMMIT_INTERVAL_MS, + atLeast(0), + Importance.LOW, + COMMIT_INTERVAL_MS_DOC) .define(ENABLE_METRICS_PUSH_CONFIG, - Type.BOOLEAN, - true, - Importance.LOW, - ENABLE_METRICS_PUSH_DOC) + Type.BOOLEAN, + true, + Importance.LOW, + ENABLE_METRICS_PUSH_DOC) .define(REPARTITION_PURGE_INTERVAL_MS_CONFIG, - Type.LONG, - DEFAULT_COMMIT_INTERVAL_MS, - atLeast(0), - Importance.LOW, - REPARTITION_PURGE_INTERVAL_MS_DOC) + Type.LONG, + DEFAULT_COMMIT_INTERVAL_MS, + atLeast(0), + Importance.LOW, + REPARTITION_PURGE_INTERVAL_MS_DOC) .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, - Type.LONG, - 9 * 60 * 1000L, - Importance.LOW, - CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC) + Type.LONG, + 9 * 60 * 1000L, + Importance.LOW, + CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC) .define(DEFAULT_DSL_STORE_CONFIG, - Type.STRING, - DEFAULT_DSL_STORE, - in(ROCKS_DB, IN_MEMORY), - Importance.LOW, - DEFAULT_DSL_STORE_DOC) + Type.STRING, + DEFAULT_DSL_STORE, + in(ROCKS_DB, IN_MEMORY), + Importance.LOW, + DEFAULT_DSL_STORE_DOC) .define(DSL_STORE_SUPPLIERS_CLASS_CONFIG, - Type.CLASS, - DSL_STORE_SUPPLIERS_CLASS_DEFAULT, - Importance.LOW, - DSL_STORE_SUPPLIERS_CLASS_DOC) + Type.CLASS, + DSL_STORE_SUPPLIERS_CLASS_DEFAULT, + Importance.LOW, + DSL_STORE_SUPPLIERS_CLASS_DOC) .define(DEFAULT_CLIENT_SUPPLIER_CONFIG, - Type.CLASS, - DefaultKafkaClientSupplier.class.getName(), - Importance.LOW, - DEFAULT_CLIENT_SUPPLIER_DOC) + Type.CLASS, + DefaultKafkaClientSupplier.class.getName(), + Importance.LOW, + DEFAULT_CLIENT_SUPPLIER_DOC) .define(METADATA_MAX_AGE_CONFIG, - Type.LONG, - 5 * 60 * 1000L, - atLeast(0), - Importance.LOW, - CommonClientConfigs.METADATA_MAX_AGE_DOC) + Type.LONG, + 5 * 60 * 1000L, + atLeast(0), + Importance.LOW, + CommonClientConfigs.METADATA_MAX_AGE_DOC) .define(METRICS_NUM_SAMPLES_CONFIG, - Type.INT, - 2, - atLeast(1), - Importance.LOW, - CommonClientConfigs.METRICS_NUM_SAMPLES_DOC) + Type.INT, + 2, + atLeast(1), + Importance.LOW, + CommonClientConfigs.METRICS_NUM_SAMPLES_DOC) .define(METRIC_REPORTER_CLASSES_CONFIG, - Type.LIST, - "", - Importance.LOW, - CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) + Type.LIST, + "", + Importance.LOW, + CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(METRICS_RECORDING_LEVEL_CONFIG, - Type.STRING, - Sensor.RecordingLevel.INFO.toString(), - in(Sensor.RecordingLevel.INFO.toString(), Sensor.RecordingLevel.DEBUG.toString(), RecordingLevel.TRACE.toString()), - Importance.LOW, - CommonClientConfigs.METRICS_RECORDING_LEVEL_DOC) + Type.STRING, + Sensor.RecordingLevel.INFO.toString(), + in(Sensor.RecordingLevel.INFO.toString(), Sensor.RecordingLevel.DEBUG.toString(), RecordingLevel.TRACE.toString()), + Importance.LOW, + CommonClientConfigs.METRICS_RECORDING_LEVEL_DOC) .define(METRICS_SAMPLE_WINDOW_MS_CONFIG, - Type.LONG, - 30000L, - atLeast(0), - Importance.LOW, - CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC) + Type.LONG, + 30000L, + atLeast(0), + Importance.LOW, + CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC) .define(AUTO_INCLUDE_JMX_REPORTER_CONFIG, - Type.BOOLEAN, - true, - Importance.LOW, - CommonClientConfigs.AUTO_INCLUDE_JMX_REPORTER_DOC) + Type.BOOLEAN, + true, + Importance.LOW, + CommonClientConfigs.AUTO_INCLUDE_JMX_REPORTER_DOC) .define(POLL_MS_CONFIG, - Type.LONG, - 100L, - Importance.LOW, - POLL_MS_DOC) + Type.LONG, + 100L, + Importance.LOW, + POLL_MS_DOC) .define(PROBING_REBALANCE_INTERVAL_MS_CONFIG, - Type.LONG, - 10 * 60 * 1000L, - atLeast(60 * 1000L), - Importance.LOW, - PROBING_REBALANCE_INTERVAL_MS_DOC) + Type.LONG, + 10 * 60 * 1000L, + atLeast(60 * 1000L), + Importance.LOW, + PROBING_REBALANCE_INTERVAL_MS_DOC) .define(RECEIVE_BUFFER_CONFIG, - Type.INT, - 32 * 1024, - atLeast(CommonClientConfigs.RECEIVE_BUFFER_LOWER_BOUND), - Importance.LOW, - CommonClientConfigs.RECEIVE_BUFFER_DOC) + Type.INT, + 32 * 1024, + atLeast(CommonClientConfigs.RECEIVE_BUFFER_LOWER_BOUND), + Importance.LOW, + CommonClientConfigs.RECEIVE_BUFFER_DOC) .define(RECONNECT_BACKOFF_MS_CONFIG, - Type.LONG, - 50L, - atLeast(0L), - Importance.LOW, - CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC) + Type.LONG, + 50L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC) .define(RECONNECT_BACKOFF_MAX_MS_CONFIG, - Type.LONG, - 1000L, - atLeast(0L), - Importance.LOW, - CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_DOC) + Type.LONG, + 1000L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_DOC) .define(RETRIES_CONFIG, - Type.INT, - 0, - between(0, Integer.MAX_VALUE), - Importance.LOW, - CommonClientConfigs.RETRIES_DOC) + Type.INT, + 0, + between(0, Integer.MAX_VALUE), + Importance.LOW, + CommonClientConfigs.RETRIES_DOC) .define(RETRY_BACKOFF_MS_CONFIG, - Type.LONG, - 100L, - atLeast(0L), - Importance.LOW, - CommonClientConfigs.RETRY_BACKOFF_MS_DOC) + Type.LONG, + 100L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RETRY_BACKOFF_MS_DOC) .define(REQUEST_TIMEOUT_MS_CONFIG, - Type.INT, - 40 * 1000, - atLeast(0), - Importance.LOW, - CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC) + Type.INT, + 40 * 1000, + atLeast(0), + Importance.LOW, + CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC) .define(ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, - Type.CLASS, - null, - Importance.LOW, - ROCKSDB_CONFIG_SETTER_CLASS_DOC) + Type.CLASS, + null, + Importance.LOW, + ROCKSDB_CONFIG_SETTER_CLASS_DOC) .define(SEND_BUFFER_CONFIG, - Type.INT, - 128 * 1024, - atLeast(CommonClientConfigs.SEND_BUFFER_LOWER_BOUND), - Importance.LOW, - CommonClientConfigs.SEND_BUFFER_DOC) + Type.INT, + 128 * 1024, + atLeast(CommonClientConfigs.SEND_BUFFER_LOWER_BOUND), + Importance.LOW, + CommonClientConfigs.SEND_BUFFER_DOC) .define(STATE_CLEANUP_DELAY_MS_CONFIG, - Type.LONG, - 10 * 60 * 1000L, - Importance.LOW, - STATE_CLEANUP_DELAY_MS_DOC) + Type.LONG, + 10 * 60 * 1000L, + Importance.LOW, + STATE_CLEANUP_DELAY_MS_DOC) .define(UPGRADE_FROM_CONFIG, - Type.STRING, - null, - in(Stream.concat( - Stream.of((String) null), - Arrays.stream(UpgradeFromValues.values()).map(UpgradeFromValues::toString) - ).toArray(String[]::new) - ), - Importance.LOW, - UPGRADE_FROM_DOC) + Type.STRING, + null, + in(Stream.concat( + Stream.of((String) null), + Arrays.stream(UpgradeFromValues.values()).map(UpgradeFromValues::toString) + ).toArray(String[]::new) + ), + Importance.LOW, + UPGRADE_FROM_DOC) .define(WINDOWED_INNER_CLASS_SERDE, Type.STRING, null, Importance.LOW, WINDOWED_INNER_CLASS_SERDE_DOC) .define(WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, - Type.LONG, - 24 * 60 * 60 * 1000L, - Importance.LOW, - WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC) + Type.LONG, + 24 * 60 * 60 * 1000L, + Importance.LOW, + WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC) .define(WINDOW_SIZE_MS_CONFIG, - Type.LONG, - null, - Importance.LOW, - WINDOW_SIZE_MS_DOC); + Type.LONG, + null, + Importance.LOW, + WINDOW_SIZE_MS_DOC); } // this is the list of configs for underlying clients // that streams prefer different default values private static final Map PRODUCER_DEFAULT_OVERRIDES; + static { final Map tempProducerDefaultOverrides = new HashMap<>(); tempProducerDefaultOverrides.put(ProducerConfig.LINGER_MS_CONFIG, "100"); @@ -1196,6 +1325,7 @@ public class StreamsConfig extends AbstractConfig { } private static final Map PRODUCER_EOS_OVERRIDES; + static { final Map tempProducerDefaultOverrides = new HashMap<>(PRODUCER_DEFAULT_OVERRIDES); tempProducerDefaultOverrides.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, Integer.MAX_VALUE); @@ -1207,6 +1337,7 @@ public class StreamsConfig extends AbstractConfig { } private static final Map CONSUMER_DEFAULT_OVERRIDES; + static { final Map tempConsumerDefaultOverrides = new HashMap<>(); tempConsumerDefaultOverrides.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "1000"); @@ -1217,6 +1348,7 @@ public class StreamsConfig extends AbstractConfig { } private static final Map CONSUMER_EOS_OVERRIDES; + static { final Map tempConsumerDefaultOverrides = new HashMap<>(CONSUMER_DEFAULT_OVERRIDES); tempConsumerDefaultOverrides.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, READ_COMMITTED.toString()); @@ -1253,7 +1385,7 @@ public static class InternalConfig { public static boolean getStateUpdaterEnabled(final Map configs) { return InternalConfig.getBoolean(configs, InternalConfig.STATE_UPDATER_ENABLED, false); } - + // Private API to enable processing threads (i.e. polling is decoupled from processing) public static final String PROCESSING_THREADS_ENABLED = "__processing.threads.enabled__"; @@ -1418,12 +1550,12 @@ protected StreamsConfig(final Map props, final String processingModeConfig = getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); if (processingModeConfig.equals(EXACTLY_ONCE)) { log.warn("Configuration parameter `{}` is deprecated and will be removed in the 4.0.0 release. " + - "Please use `{}` instead. Note that this requires broker version 2.5+ so you should prepare " - + "to upgrade your brokers if necessary.", EXACTLY_ONCE, EXACTLY_ONCE_V2); + "Please use `{}` instead. Note that this requires broker version 2.5+ so you should prepare " + + "to upgrade your brokers if necessary.", EXACTLY_ONCE, EXACTLY_ONCE_V2); } if (processingModeConfig.equals(EXACTLY_ONCE_BETA)) { log.warn("Configuration parameter `{}` is deprecated and will be removed in the 4.0.0 release. " + - "Please use `{}` instead.", EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2); + "Please use `{}` instead.", EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2); } if (props.containsKey(RETRIES_CONFIG)) { @@ -1440,18 +1572,18 @@ private void verifyEOSTransactionTimeoutCompatibility() { final long commitInterval = getLong(COMMIT_INTERVAL_MS_CONFIG); final String transactionTimeoutConfigKey = producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG); final int transactionTimeout = - originals().containsKey(transactionTimeoutConfigKey) ? - (int) Objects.requireNonNull( - parseType(transactionTimeoutConfigKey, originals().get(transactionTimeoutConfigKey), Type.INT), - "Could not parse config `" + COMMIT_INTERVAL_MS_CONFIG + "` because it's set to `null`") : - DEFAULT_TRANSACTION_TIMEOUT; + originals().containsKey(transactionTimeoutConfigKey) ? + (int) Objects.requireNonNull( + parseType(transactionTimeoutConfigKey, originals().get(transactionTimeoutConfigKey), Type.INT), + "Could not parse config `" + COMMIT_INTERVAL_MS_CONFIG + "` because it's set to `null`") : + DEFAULT_TRANSACTION_TIMEOUT; if (transactionTimeout < commitInterval) { throw new IllegalArgumentException(String.format("Transaction timeout %d was set lower than " + - "streams commit interval %d. This will cause ongoing transaction always timeout due to inactivity " + - "caused by long commit interval. Consider reconfiguring commit interval to match " + - "transaction timeout by tuning 'commit.interval.ms' config, or increase the transaction timeout to match " + - "commit interval by tuning `producer.transaction.timeout.ms` config.", + "streams commit interval %d. This will cause ongoing transaction always timeout due to inactivity " + + "caused by long commit interval. Consider reconfiguring commit interval to match " + + "transaction timeout by tuning 'commit.interval.ms' config, or increase the transaction timeout to match " + + "commit interval by tuning `producer.transaction.timeout.ms` config.", transactionTimeout, commitInterval)); } } @@ -1463,7 +1595,7 @@ protected Map postProcessParsedConfig(final Map if (StreamsConfigUtils.eosEnabled(this) && !originals().containsKey(COMMIT_INTERVAL_MS_CONFIG)) { log.debug("Using {} default value of {} as exactly once is enabled.", - COMMIT_INTERVAL_MS_CONFIG, EOS_DEFAULT_COMMIT_INTERVAL_MS); + COMMIT_INTERVAL_MS_CONFIG, EOS_DEFAULT_COMMIT_INTERVAL_MS); configUpdates.put(COMMIT_INTERVAL_MS_CONFIG, EOS_DEFAULT_COMMIT_INTERVAL_MS); } @@ -1478,28 +1610,28 @@ private void validateRackAwarenessConfiguration() { if (clientTags.size() > MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE) { throw new ConfigException("At most " + MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE + " client tags " + - "can be specified using " + CLIENT_TAG_PREFIX + " prefix."); + "can be specified using " + CLIENT_TAG_PREFIX + " prefix."); } for (final String rackAwareAssignmentTag : rackAwareAssignmentTags) { if (!clientTags.containsKey(rackAwareAssignmentTag)) { throw new ConfigException(RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, - rackAwareAssignmentTags, - "Contains invalid value [" + rackAwareAssignmentTag + "] " + - "which doesn't have corresponding tag set via [" + CLIENT_TAG_PREFIX + "] prefix."); + rackAwareAssignmentTags, + "Contains invalid value [" + rackAwareAssignmentTag + "] " + + "which doesn't have corresponding tag set via [" + CLIENT_TAG_PREFIX + "] prefix."); } } clientTags.forEach((tagKey, tagValue) -> { if (tagKey.length() > MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH) { throw new ConfigException(CLIENT_TAG_PREFIX, - tagKey, - "Tag key exceeds maximum length of " + MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH + "."); + tagKey, + "Tag key exceeds maximum length of " + MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH + "."); } if (tagValue.length() > MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH) { throw new ConfigException(CLIENT_TAG_PREFIX, - tagValue, - "Tag value exceeds maximum length of " + MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH + "."); + tagValue, + "Tag value exceeds maximum length of " + MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH + "."); } }); } @@ -1534,25 +1666,25 @@ private void checkIfUnexpectedUserSpecifiedConsumerConfig(final Map getMainConsumerConfigs(final String groupId, final St if (segmentSize < batchSize) { throw new IllegalArgumentException(String.format("Specified topic segment size %d is is smaller than the configured producer batch size %d, this will cause produced batch not able to be appended to the topic", - segmentSize, - batchSize)); + segmentSize, + batchSize)); } } @@ -1764,6 +1896,7 @@ public Map getProducerConfigs(final String clientId) { /** * Get the configs for the {@link Admin admin client}. + * * @param clientId clientId * @return Map of the admin client configuration. */ @@ -1832,7 +1965,7 @@ public static Set verifyTopologyOptimizationConfigs(final String config) throw new ConfigException("\"" + config + "\" is not a valid optimization config. " + CONFIG_ERROR_MSG); } } - for (final String conf: configs) { + for (final String conf : configs) { if (!TOPOLOGY_OPTIMIZATION_CONFIGS.contains(conf)) { throw new ConfigException("Unrecognized config. " + CONFIG_ERROR_MSG); } @@ -1849,6 +1982,7 @@ public static Set verifyTopologyOptimizationConfigs(final String config) /** * Return configured KafkaClientSupplier + * * @return Configured KafkaClientSupplier */ public KafkaClientSupplier getKafkaClientSupplier() { @@ -1865,7 +1999,7 @@ public KafkaClientSupplier getKafkaClientSupplier() { @SuppressWarnings("WeakerAccess") public Serde defaultKeySerde() { final Object keySerdeConfigSetting = get(DEFAULT_KEY_SERDE_CLASS_CONFIG); - if (keySerdeConfigSetting == null) { + if (keySerdeConfigSetting == null) { throw new ConfigException("Please specify a key serde or set one through StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG"); } try { @@ -1915,6 +2049,11 @@ public ProductionExceptionHandler defaultProductionExceptionHandler() { return getConfiguredInstance(DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, ProductionExceptionHandler.class); } + @SuppressWarnings("WeakerAccess") + public ProcessingExceptionHandler processingExceptionHandler() { + return getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class); + } + /** * Override any client properties in the original configs with overrides * @@ -1927,7 +2066,7 @@ private Map clientProps(final Set configNames, // iterate all client config names, filter out non-client configs from the original // property map and use the overridden values when they are not specified by users final Map parsed = new HashMap<>(); - for (final String configName: configNames) { + for (final String configName : configNames) { if (originals.containsKey(configName)) { parsed.put(configName, originals.get(configName)); } diff --git a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java index 2d4157eba7885..c3a9dcb2d50c7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java @@ -22,15 +22,16 @@ import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.internals.StreamsConfigUtils; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.internals.MaterializedInternal; import org.apache.kafka.streams.processor.TimestampExtractor; - import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper; import org.apache.kafka.streams.state.DslStoreSuppliers; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import java.util.Optional; import java.util.Properties; import java.util.function.Supplier; @@ -40,23 +41,24 @@ import static org.apache.kafka.streams.StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_DOC; import static org.apache.kafka.streams.StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG; import static org.apache.kafka.streams.StreamsConfig.CACHE_MAX_BYTES_BUFFERING_DOC; -import static org.apache.kafka.streams.StreamsConfig.DSL_STORE_SUPPLIERS_CLASS_CONFIG; -import static org.apache.kafka.streams.StreamsConfig.DSL_STORE_SUPPLIERS_CLASS_DEFAULT; -import static org.apache.kafka.streams.StreamsConfig.DSL_STORE_SUPPLIERS_CLASS_DOC; -import static org.apache.kafka.streams.StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG; -import static org.apache.kafka.streams.StreamsConfig.STATESTORE_CACHE_MAX_BYTES_DOC; import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG; import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC; +import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_CONFIG; +import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_DOC; import static org.apache.kafka.streams.StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG; import static org.apache.kafka.streams.StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC; +import static org.apache.kafka.streams.StreamsConfig.DSL_STORE_SUPPLIERS_CLASS_CONFIG; +import static org.apache.kafka.streams.StreamsConfig.DSL_STORE_SUPPLIERS_CLASS_DEFAULT; +import static org.apache.kafka.streams.StreamsConfig.DSL_STORE_SUPPLIERS_CLASS_DOC; +import static org.apache.kafka.streams.StreamsConfig.IN_MEMORY; import static org.apache.kafka.streams.StreamsConfig.MAX_TASK_IDLE_MS_CONFIG; import static org.apache.kafka.streams.StreamsConfig.MAX_TASK_IDLE_MS_DOC; +import static org.apache.kafka.streams.StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG; +import static org.apache.kafka.streams.StreamsConfig.ROCKS_DB; +import static org.apache.kafka.streams.StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG; +import static org.apache.kafka.streams.StreamsConfig.STATESTORE_CACHE_MAX_BYTES_DOC; import static org.apache.kafka.streams.StreamsConfig.TASK_TIMEOUT_MS_CONFIG; import static org.apache.kafka.streams.StreamsConfig.TASK_TIMEOUT_MS_DOC; -import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_CONFIG; -import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_DOC; -import static org.apache.kafka.streams.StreamsConfig.ROCKS_DB; -import static org.apache.kafka.streams.StreamsConfig.IN_MEMORY; import static org.apache.kafka.streams.internals.StreamsConfigUtils.getTotalCacheSize; /** @@ -135,6 +137,7 @@ public class TopologyConfig extends AbstractConfig { public final Class dslStoreSuppliers; public final Supplier timestampExtractorSupplier; public final Supplier deserializationExceptionHandlerSupplier; + public final Supplier processingExceptionHandlerSupplier; public TopologyConfig(final StreamsConfig globalAppConfigs) { this(null, globalAppConfigs, new Properties()); @@ -225,6 +228,13 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo deserializationExceptionHandlerSupplier = () -> globalAppConfigs.getConfiguredInstance(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, DeserializationExceptionHandler.class); } + if (isTopologyOverride(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, topologyOverrides)) { + processingExceptionHandlerSupplier = () -> getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class); + log.info("Topology {} is overriding {} to {}", topologyName, PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, getClass(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG)); + } else { + processingExceptionHandlerSupplier = () -> globalAppConfigs.getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class); + } + if (isTopologyOverride(DEFAULT_DSL_STORE_CONFIG, topologyOverrides)) { storeType = getString(DEFAULT_DSL_STORE_CONFIG); log.info("Topology {} is overriding {} to {}", topologyName, DEFAULT_DSL_STORE_CONFIG, storeType); @@ -280,7 +290,8 @@ public TaskConfig getTaskConfig() { maxBufferedSize, timestampExtractorSupplier.get(), deserializationExceptionHandlerSupplier.get(), - eosEnabled + eosEnabled, + processingExceptionHandlerSupplier.get() ); } @@ -291,19 +302,22 @@ public static class TaskConfig { public final TimestampExtractor timestampExtractor; public final DeserializationExceptionHandler deserializationExceptionHandler; public final boolean eosEnabled; + public final ProcessingExceptionHandler processingExceptionHandler; private TaskConfig(final long maxTaskIdleMs, final long taskTimeoutMs, final int maxBufferedSize, final TimestampExtractor timestampExtractor, final DeserializationExceptionHandler deserializationExceptionHandler, - final boolean eosEnabled) { + final boolean eosEnabled, + final ProcessingExceptionHandler processingExceptionHandler) { this.maxTaskIdleMs = maxTaskIdleMs; this.taskTimeoutMs = taskTimeoutMs; this.maxBufferedSize = maxBufferedSize; this.timestampExtractor = timestampExtractor; this.deserializationExceptionHandler = deserializationExceptionHandler; this.eosEnabled = eosEnabled; + this.processingExceptionHandler = processingExceptionHandler; } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingExceptionHandler.java index c24aa6c3bb19a..acf4c58fbc481 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 @@ -1,3 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.kafka.streams.errors; import org.apache.kafka.common.Configurable; @@ -23,10 +39,14 @@ enum ProcessingHandlerResponse { /* fail the processing and stop */ FAIL(2, "FAIL"); - /** the permanent and immutable name of processing exception response */ + /** + * the permanent and immutable name of processing exception response + */ public final String name; - /** the permanent and immutable id of processing exception response */ + /** + * the permanent and immutable id of processing exception response + */ public final int id; ProcessingHandlerResponse(final int id, final String name) { diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java index c934c5b09437a..93d6a330c4bab 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndContinueExceptionHandler.java @@ -1,3 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.kafka.streams.errors; import org.apache.kafka.streams.processor.ErrorHandlerContext; @@ -15,17 +31,17 @@ public class ProcessingLogAndContinueExceptionHandler implements ProcessingExcep private static final Logger log = LoggerFactory.getLogger(ProcessingLogAndContinueExceptionHandler.class); @Override - public ProcessingHandlerResponse handle(ErrorHandlerContext context, Record record, 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(), context.taskId(), context.topic(), context.partition(), context.offset(), - exception); + "processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", + context.processorNodeId(), context.taskId(), context.topic(), context.partition(), context.offset(), + exception); return ProcessingHandlerResponse.CONTINUE; } @Override - public void configure(Map configs) { + public void configure(final Map configs) { // ignore } } diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java index 9319b554f5edd..facb300a9692b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProcessingLogAndFailExceptionHandler.java @@ -1,3 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.kafka.streams.errors; import org.apache.kafka.streams.processor.ErrorHandlerContext; @@ -15,17 +31,17 @@ public class ProcessingLogAndFailExceptionHandler implements ProcessingException private static final Logger log = LoggerFactory.getLogger(ProcessingLogAndFailExceptionHandler.class); @Override - public ProcessingHandlerResponse handle(ErrorHandlerContext context, Record record, 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(), context.taskId(), context.topic(), context.partition(), context.offset(), - exception); + "processor node: {}, taskId: {}, source topic: {}, source partition: {}, source offset: {}", + context.processorNodeId(), context.taskId(), context.topic(), context.partition(), context.offset(), + exception); return ProcessingHandlerResponse.FAIL; } @Override - public void configure(Map configs) { + public void configure(final Map configs) { // ignore } } diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java index 915a6b13bb323..c1b05ccc2755d 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,7 +19,6 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Configurable; import org.apache.kafka.streams.processor.ErrorHandlerContext; -import sun.reflect.generics.reflectiveObjects.NotImplementedException; /** * Interface that specifies how an exception when attempting to produce a result to @@ -30,22 +29,22 @@ public interface ProductionExceptionHandler extends Configurable { * Inspect a record that we attempted to produce, and the exception that resulted * from attempting to produce it and determine whether or not to continue processing. * - * @param record The record that failed to produce + * @param record The record that failed to produce * @param exception The exception that occurred during production * @deprecated Use {@link #handle(ErrorHandlerContext, ProducerRecord, Exception)} instead */ @Deprecated default ProductionExceptionHandlerResponse handle(final ProducerRecord record, - final Exception exception) { - throw new NotImplementedException(); + final Exception exception) { + throw new UnsupportedOperationException(); } /** * Inspect a record that we attempted to produce, and the exception that resulted * from attempting to produce it and determine whether or not to continue processing. * - * @param context The error handler context metadata - * @param record The record that failed to produce + * @param context The error handler context metadata + * @param record The record that failed to produce * @param exception The exception that occurred during production */ @SuppressWarnings("deprecation") @@ -59,8 +58,8 @@ default ProductionExceptionHandlerResponse handle(final ErrorHandlerContext cont * Handles serialization exception and determine if the process should continue. The default implementation is to * fail the process. * - * @param record the record that failed to serialize - * @param exception the exception that occurred during serialization + * @param record the record that failed to serialize + * @param exception the exception that occurred during serialization * @deprecated Use {@link #handle(ErrorHandlerContext, ProducerRecord, Exception)} instead */ @Deprecated @@ -73,10 +72,10 @@ default ProductionExceptionHandlerResponse handleSerializationException(final Pr * 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 + * @param context the error handler context metadata + * @param record the record that failed to serialize + * @param exception the exception that occurred during serialization + * @param origin the origin of the serialization exception */ @SuppressWarnings("deprecation") default ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context, diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/SerializationExceptionOrigin.java b/streams/src/main/java/org/apache/kafka/streams/errors/SerializationExceptionOrigin.java deleted file mode 100644 index b90c3a3d078c9..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/errors/SerializationExceptionOrigin.java +++ /dev/null @@ -1,9 +0,0 @@ -package org.apache.kafka.streams.errors; - -/** - * Indicates whether a {@link org.apache.kafka.common.errors.SerializationException} comes from the key or the value - */ -public enum SerializationExceptionOrigin { - KEY, - VALUE -} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java index b6c222770953c..8fa60c0c62d26 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java @@ -1,3 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.kafka.streams.processor; import org.apache.kafka.common.header.Headers; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContextImpl.java deleted file mode 100644 index 2f729daae87a5..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContextImpl.java +++ /dev/null @@ -1,72 +0,0 @@ -package org.apache.kafka.streams.processor; - -import org.apache.kafka.common.header.Headers; - -public class ErrorHandlerContextImpl implements ErrorHandlerContext { - private final String topic; - private final int partition; - private final long offset; - private final Headers headers; - private final byte[] sourceRawKey; - private final byte[] sourceRawValue; - private final String processorNodeId; - private final TaskId taskId; - - public ErrorHandlerContextImpl(String topic, - int partition, - long offset, - Headers headers, - byte[] sourceRawKey, - byte[] sourceRawValue, - String processorNodeId, - TaskId taskId) { - this.topic = topic; - this.partition = partition; - this.offset = offset; - this.headers = headers; - this.sourceRawKey = sourceRawKey; - this.sourceRawValue = sourceRawValue; - this.processorNodeId = processorNodeId; - this.taskId = taskId; - } - - @Override - public String topic() { - return topic; - } - - @Override - public int partition() { - return partition; - } - - @Override - public long offset() { - return offset; - } - - @Override - public Headers headers() { - return headers; - } - - @Override - public byte[] sourceRawKey() { - return sourceRawKey; - } - - @Override - public byte[] sourceRawValue() { - return sourceRawValue; - } - - @Override - public String processorNodeId() { - return processorNodeId; - } - - @Override - public TaskId taskId() { - return taskId; - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java new file mode 100644 index 0000000000000..515d900d7c2fd --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.streams.processor.ErrorHandlerContext; +import org.apache.kafka.streams.processor.TaskId; + +public class ErrorHandlerContextImpl implements ErrorHandlerContext { + private final String topic; + private final int partition; + private final long offset; + private final Headers headers; + private final byte[] sourceRawKey; + private final byte[] sourceRawValue; + private final String processorNodeId; + private final TaskId taskId; + + public ErrorHandlerContextImpl(final String topic, + final int partition, + final long offset, + final Headers headers, + final byte[] sourceRawKey, + final byte[] sourceRawValue, + final String processorNodeId, + final TaskId taskId) { + this.topic = topic; + this.partition = partition; + this.offset = offset; + this.headers = headers; + this.sourceRawKey = sourceRawKey; + this.sourceRawValue = sourceRawValue; + this.processorNodeId = processorNodeId; + this.taskId = taskId; + } + + @Override + public String topic() { + return topic; + } + + @Override + public int partition() { + return partition; + } + + @Override + public long offset() { + return offset; + } + + @Override + public Headers headers() { + return headers; + } + + @Override + public byte[] sourceRawKey() { + return sourceRawKey; + } + + @Override + public byte[] sourceRawValue() { + return sourceRawValue; + } + + @Override + public String processorNodeId() { + return processorNodeId; + } + + @Override + public TaskId taskId() { + return taskId; + } +} 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 8abf5e897623f..755d9a9582490 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 @@ -219,7 +219,7 @@ public void send(final String topic, log.debug(String.format("Error serializing record to topic %s", topic), exception); try { - response = productionExceptionHandler.handleSerializationException(record, exception); + response = productionExceptionHandler.handleSerializationException(null, record, exception, null); } catch (final Exception e) { log.error("Fatal when handling serialization exception", e); recordSendError(topic, e, null); @@ -309,7 +309,7 @@ private void recordSendError(final String topic, final Exception exception, fina "`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 (productionExceptionHandler.handle(serializedRecord, exception) == ProductionExceptionHandlerResponse.FAIL) { + if (productionExceptionHandler.handle(null, serializedRecord, exception) == ProductionExceptionHandlerResponse.FAIL) { errorMessage += "\nException handler choose to FAIL the processing, no more records would be sent."; sendException.set(new StreamsException(errorMessage, exception)); } else { 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 163e2c0997e6d..f551a2093b2b7 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 @@ -27,13 +27,16 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.TopologyConfig.TaskConfig; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.LockException; +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.processor.Cancellable; +import org.apache.kafka.streams.processor.ErrorHandlerContext; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.TaskId; @@ -44,7 +47,6 @@ import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; -import org.apache.kafka.streams.TopologyConfig.TaskConfig; import org.apache.kafka.streams.state.internals.ThreadCache; import java.io.IOException; @@ -61,8 +63,9 @@ import java.util.stream.Collectors; import static java.util.Collections.singleton; -import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeRecordSensor; +import static org.apache.kafka.streams.StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; +import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeRecordSensor; /** * A StreamTask is associated with a {@link AbstractPartitionGroup}, and is assigned to a StreamThread for processing. @@ -111,6 +114,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator, private boolean commitRequested = false; private boolean hasPendingTxCommit = false; private Optional timeCurrentIdlingStarted; + private ProcessingExceptionHandler processingExceptionHandler; @SuppressWarnings({"rawtypes", "this-escape", "checkstyle:ParameterNumber"}) public StreamTask(final TaskId id, @@ -217,6 +221,7 @@ public StreamTask(final TaskId id, highWatermark.put(topicPartition, -1L); } timeCurrentIdlingStarted = Optional.empty(); + this.processingExceptionHandler = config.processingExceptionHandler; } // create queues for each assigned partition and associate them @@ -844,7 +849,30 @@ private void doProcess(final long wallClockTime) { processorContext.timestamp(), processorContext.headers() ); - maybeMeasureLatency(() -> currNode.process(toProcess), time, processLatencySensor); + maybeMeasureLatency(() -> { + try { + currNode.process(toProcess); + } catch (final Exception e) { + final ErrorHandlerContext errorHandlerContext = new ErrorHandlerContextImpl(processorContext.topic(), + processorContext.partition(), + processorContext.offset(), + processorContext.headers(), + null, + null, + currNode.name(), + processorContext.taskId()); + final ProcessingExceptionHandler.ProcessingHandlerResponse response = this.processingExceptionHandler + .handle(errorHandlerContext, toProcess, e); + + if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { + throw new StreamsException("Processing exception handler is set to fail upon" + + " a processing error. If you would rather have the streaming pipeline" + + " continue after a deserialization error, please set the " + + PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.", + e); + } + } + }, time, processLatencySensor); log.trace("Completed processing one record [{}]", record); } diff --git a/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java b/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java index be1e98e4a7154..61fea5f27bb40 100644 --- a/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java +++ b/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.errors; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.streams.processor.ErrorHandlerContext; import java.util.Map; @@ -26,14 +27,17 @@ */ public class AlwaysContinueProductionExceptionHandler implements ProductionExceptionHandler { @Override - public ProductionExceptionHandlerResponse handle(final ProducerRecord record, + public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, + final ProducerRecord record, final Exception exception) { return ProductionExceptionHandlerResponse.CONTINUE; } @Override - public ProductionExceptionHandlerResponse handleSerializationException(final ProducerRecord record, - final Exception exception) { + public ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception, + final SerializationExceptionOrigin origin) { return ProductionExceptionHandlerResponse.CONTINUE; } From 5ffca71fd6d93af521d32fc3a9d32824f0554a53 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Wed, 15 May 2024 23:32:23 +0200 Subject: [PATCH 07/28] KAFKA-16448: Move processing exception catch to processor context impl to catch errors on child processors --- .../internals/ProcessorContextImpl.java | 22 +++++++++++++- .../processor/internals/ProcessorNode.java | 1 - .../processor/internals/StreamTask.java | 30 +------------------ 3 files changed, 22 insertions(+), 31 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index b484d26f0fe87..fd6b8cc2fa6e8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -22,8 +22,10 @@ import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.Cancellable; +import org.apache.kafka.streams.processor.ErrorHandlerContext; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.StateStore; @@ -44,6 +46,7 @@ import java.util.Map; import static org.apache.kafka.streams.StreamsConfig.InternalConfig.IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED; +import static org.apache.kafka.streams.StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG; import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; import static org.apache.kafka.streams.internals.ApiUtils.validateMillisecondDuration; import static org.apache.kafka.streams.processor.internals.AbstractReadOnlyDecorator.getReadOnlyStore; @@ -288,7 +291,24 @@ private void forwardInternal(final ProcessorNode child, final Record record) { setCurrentNode(child); - child.process(record); + try { + child.process(record); + } catch (final Exception e) { + final ErrorHandlerContext errorHandlerContext = new ErrorHandlerContextImpl(topic(), + partition(), offset(), headers(), null, null, child.name(), taskId()); + final ProcessingExceptionHandler.ProcessingHandlerResponse response = streamTask.config + .processingExceptionHandler + .handle(errorHandlerContext, record, e); + + if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { + throw new StreamsException("Processing exception handler is set to fail upon" + + " a processing error. If you would rather have the streaming pipeline" + + " continue after a deserialization error, please set the " + + PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.", + e); + } + } + if (child.isTerminalNode()) { streamTask.maybeRecordE2ELatency(record.timestamp(), currentSystemTimeMs(), child.name()); 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 fbd976cb11274..1c2ea7f9ef7b5 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 @@ -143,7 +143,6 @@ protected void throwIfClosed() { } } - public void process(final Record record) { throwIfClosed(); 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 f551a2093b2b7..f6ff7f35ad7d5 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 @@ -30,13 +30,11 @@ import org.apache.kafka.streams.TopologyConfig.TaskConfig; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.LockException; -import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.processor.Cancellable; -import org.apache.kafka.streams.processor.ErrorHandlerContext; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.TaskId; @@ -63,7 +61,6 @@ import java.util.stream.Collectors; import static java.util.Collections.singleton; -import static org.apache.kafka.streams.StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeRecordSensor; @@ -114,7 +111,6 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator, private boolean commitRequested = false; private boolean hasPendingTxCommit = false; private Optional timeCurrentIdlingStarted; - private ProcessingExceptionHandler processingExceptionHandler; @SuppressWarnings({"rawtypes", "this-escape", "checkstyle:ParameterNumber"}) public StreamTask(final TaskId id, @@ -221,7 +217,6 @@ public StreamTask(final TaskId id, highWatermark.put(topicPartition, -1L); } timeCurrentIdlingStarted = Optional.empty(); - this.processingExceptionHandler = config.processingExceptionHandler; } // create queues for each assigned partition and associate them @@ -849,30 +844,7 @@ private void doProcess(final long wallClockTime) { processorContext.timestamp(), processorContext.headers() ); - maybeMeasureLatency(() -> { - try { - currNode.process(toProcess); - } catch (final Exception e) { - final ErrorHandlerContext errorHandlerContext = new ErrorHandlerContextImpl(processorContext.topic(), - processorContext.partition(), - processorContext.offset(), - processorContext.headers(), - null, - null, - currNode.name(), - processorContext.taskId()); - final ProcessingExceptionHandler.ProcessingHandlerResponse response = this.processingExceptionHandler - .handle(errorHandlerContext, toProcess, e); - - if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { - throw new StreamsException("Processing exception handler is set to fail upon" + - " a processing error. If you would rather have the streaming pipeline" + - " continue after a deserialization error, please set the " + - PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.", - e); - } - } - }, time, processLatencySensor); + maybeMeasureLatency(() -> currNode.process(toProcess), time, processLatencySensor); log.trace("Completed processing one record [{}]", record); } From a5ce52b43471f915a433acf1132c2e2008408677 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Thu, 16 May 2024 11:03:31 +0200 Subject: [PATCH 08/28] KAFKA-16448: add error handling for deserialisation --- gradle.properties | 2 +- .../DeserializationExceptionHandler.java | 27 ++- .../LogAndContinueExceptionHandler.java | 15 ++ .../errors/LogAndFailExceptionHandler.java | 14 ++ .../errors/ProductionExceptionHandler.java | 3 +- .../errors/SerializationExceptionOrigin.java | 9 - .../processor/ErrorHandlerContext.java | 2 + .../ErrorHandlerContextImpl.java | 31 ++-- .../processor/internals/ProcessorNode.java | 2 - .../internals/RecordCollectorImpl.java | 31 +++- .../internals/RecordDeserializer.java | 17 +- .../RestrictiveProcessorContext.java | 154 ++++++++++++++++++ ...aysContinueProductionExceptionHandler.java | 2 + .../internals/RecordDeserializerTest.java | 72 +++++++- 14 files changed, 343 insertions(+), 38 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/errors/SerializationExceptionOrigin.java rename streams/src/main/java/org/apache/kafka/streams/processor/{ => internals}/ErrorHandlerContextImpl.java (64%) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/RestrictiveProcessorContext.java diff --git a/gradle.properties b/gradle.properties index 4880248cac320..aa2a3d5f25ada 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,7 +23,7 @@ group=org.apache.kafka # - streams/quickstart/pom.xml # - streams/quickstart/java/src/main/resources/archetype-resources/pom.xml # - streams/quickstart/java/pom.xml -version=3.8.0-SNAPSHOT +version=3.8.0-SNAPSHOT-KIP-1033 scalaVersion=2.13.12 # Adding swaggerVersion in gradle.properties to have a single version in place for swagger # New version of Swagger 2.2.14 requires minimum JDK 11. diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java index 95ccfeced8e43..93f9ad5ff37b6 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 @@ -19,7 +19,9 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.Configurable; +import org.apache.kafka.streams.processor.ErrorHandlerContext; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.internals.ErrorHandlerContextImpl; /** * Interface that specifies how an exception from source node deserialization @@ -37,11 +39,28 @@ public interface DeserializationExceptionHandler extends Configurable { * @param context processor context * @param record record that failed deserialization * @param exception the actual exception + * @deprecated Please {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception) */ - @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. - DeserializationHandlerResponse handle(final ProcessorContext context, - final ConsumerRecord record, - final Exception exception); + @Deprecated + default DeserializationHandlerResponse handle(final ProcessorContext context, + final ConsumerRecord record, + final Exception exception) { + + throw new UnsupportedOperationException(); + } + /** + * Inspect a record and the exception received. + * + * @param context error handler context + * @param record record that failed deserialization + * @param exception the actual exception + */ + default DeserializationHandlerResponse handle(final ErrorHandlerContext context, + final ConsumerRecord record, + final Exception exception) { + + return handle(((ErrorHandlerContextImpl) context).convertToProcessorContext(), record, exception); + } /** * Enumeration that describes the response from the exception handler. diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java index 4f9a0964405a2..0494b44500210 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streams.processor.ErrorHandlerContext; import org.apache.kafka.streams.processor.ProcessorContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,6 +33,7 @@ public class LogAndContinueExceptionHandler implements DeserializationExceptionH private static final Logger log = LoggerFactory.getLogger(LogAndContinueExceptionHandler.class); @Override + @Deprecated public DeserializationHandlerResponse handle(final ProcessorContext context, final ConsumerRecord record, final Exception exception) { @@ -44,6 +46,19 @@ public DeserializationHandlerResponse handle(final ProcessorContext context, return DeserializationHandlerResponse.CONTINUE; } + @Override + public DeserializationHandlerResponse handle(final ErrorHandlerContext context, + final ConsumerRecord record, + final Exception exception) { + + log.warn("Exception caught during Deserialization, " + + "taskId: {}, topic: {}, partition: {}, offset: {}", + context.taskId(), record.topic(), record.partition(), record.offset(), + exception); + + return DeserializationHandlerResponse.CONTINUE; + } + @Override public void configure(final Map configs) { // ignore diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java index 61d210649ba9a..5fc197a10b03e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.errors; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streams.processor.ErrorHandlerContext; import org.apache.kafka.streams.processor.ProcessorContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,6 +33,7 @@ public class LogAndFailExceptionHandler implements DeserializationExceptionHandl private static final Logger log = LoggerFactory.getLogger(LogAndFailExceptionHandler.class); @Override + @Deprecated public DeserializationHandlerResponse handle(final ProcessorContext context, final ConsumerRecord record, final Exception exception) { @@ -44,6 +46,18 @@ public DeserializationHandlerResponse handle(final ProcessorContext context, return DeserializationHandlerResponse.FAIL; } + public DeserializationHandlerResponse handle(final ErrorHandlerContext context, + final ConsumerRecord record, + final Exception exception) { + + log.error("Exception caught during Deserialization, " + + "taskId: {}, topic: {}, partition: {}, offset: {}", + context.taskId(), record.topic(), record.partition(), record.offset(), + exception); + + return DeserializationHandlerResponse.FAIL; + } + @Override public void configure(final Map configs) { // ignore diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java index 915a6b13bb323..30c9999d5f097 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,7 +19,6 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Configurable; import org.apache.kafka.streams.processor.ErrorHandlerContext; -import sun.reflect.generics.reflectiveObjects.NotImplementedException; /** * Interface that specifies how an exception when attempting to produce a result to @@ -37,7 +36,7 @@ public interface ProductionExceptionHandler extends Configurable { @Deprecated default ProductionExceptionHandlerResponse handle(final ProducerRecord record, final Exception exception) { - throw new NotImplementedException(); + throw new UnsupportedOperationException(); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/SerializationExceptionOrigin.java b/streams/src/main/java/org/apache/kafka/streams/errors/SerializationExceptionOrigin.java deleted file mode 100644 index b90c3a3d078c9..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/errors/SerializationExceptionOrigin.java +++ /dev/null @@ -1,9 +0,0 @@ -package org.apache.kafka.streams.errors; - -/** - * Indicates whether a {@link org.apache.kafka.common.errors.SerializationException} comes from the key or the value - */ -public enum SerializationExceptionOrigin { - KEY, - VALUE -} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java index b6c222770953c..9da864e34f0bd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java @@ -2,6 +2,8 @@ import org.apache.kafka.common.header.Headers; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; /** * ErrorHandlerContext interface diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java similarity index 64% rename from streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContextImpl.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java index 2f729daae87a5..c35c56766c65c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java @@ -1,8 +1,13 @@ -package org.apache.kafka.streams.processor; +package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.header.Headers; +import org.apache.kafka.streams.processor.ErrorHandlerContext; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.TaskId; public class ErrorHandlerContextImpl implements ErrorHandlerContext { + + private InternalProcessorContext processorContext; private final String topic; private final int partition; private final long offset; @@ -12,7 +17,8 @@ public class ErrorHandlerContextImpl implements ErrorHandlerContext { private final String processorNodeId; private final TaskId taskId; - public ErrorHandlerContextImpl(String topic, + public ErrorHandlerContextImpl(InternalProcessorContext processorContext, + String topic, int partition, long offset, Headers headers, @@ -20,6 +26,7 @@ public ErrorHandlerContextImpl(String topic, byte[] sourceRawValue, String processorNodeId, TaskId taskId) { + this.processorContext = processorContext; this.topic = topic; this.partition = partition; this.offset = offset; @@ -32,41 +39,45 @@ public ErrorHandlerContextImpl(String topic, @Override public String topic() { - return topic; + return this.topic; } @Override public int partition() { - return partition; + return this.partition; } @Override public long offset() { - return offset; + return this.offset; } @Override public Headers headers() { - return headers; + return this.headers; } @Override public byte[] sourceRawKey() { - return sourceRawKey; + return this.sourceRawKey; } @Override public byte[] sourceRawValue() { - return sourceRawValue; + return this.sourceRawValue; } @Override public String processorNodeId() { - return processorNodeId; + return this.processorNodeId; } @Override public TaskId taskId() { - return taskId; + return this.taskId; + } + + public ProcessorContext convertToProcessorContext() { + return new RestrictiveProcessorContext(this.processorContext); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index fbd976cb11274..342fb6b0a1c1a 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 @@ -53,7 +53,6 @@ public ProcessorNode(final String name) { public ProcessorNode(final String name, final Processor processor, final Set stateStores) { - this.name = name; this.processor = processor; this.fixedKeyProcessor = null; @@ -65,7 +64,6 @@ public ProcessorNode(final String name, public ProcessorNode(final String name, final FixedKeyProcessor processor, final Set stateStores) { - this.name = name; this.processor = null; this.fixedKeyProcessor = processor; 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 8abf5e897623f..9bdd0ba48ed3b 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 @@ -219,10 +219,20 @@ public void send(final String topic, log.debug(String.format("Error serializing record to topic %s", topic), exception); try { - response = productionExceptionHandler.handleSerializationException(record, exception); + ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( + context, + record.topic(), + record.partition(), + context.offset(), + record.headers(), + null, + null, + processorNodeId, + taskId); + response = productionExceptionHandler.handleSerializationException(errorHandlerContext, record, exception, ProductionExceptionHandler.SerializationExceptionOrigin.VALUE); } catch (final Exception e) { log.error("Fatal when handling serialization exception", e); - recordSendError(topic, e, null); + recordSendError(topic, e, null, context, processorNodeId); return; } @@ -281,7 +291,7 @@ public void send(final String topic, topicProducedSensor.record(bytesProduced, context.currentSystemTimeMs()); } } else { - recordSendError(topic, exception, serializedRecord); + recordSendError(topic, exception, serializedRecord, context, processorNodeId); // 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); @@ -289,7 +299,8 @@ public void send(final String topic, }); } - private void recordSendError(final String topic, final Exception exception, final ProducerRecord serializedRecord) { + private void recordSendError(final String topic, final Exception exception, final ProducerRecord serializedRecord, + final InternalProcessorContext context, final String processorNodeId) { String errorMessage = String.format(SEND_EXCEPTION_MESSAGE, topic, taskId, exception.toString()); if (isFatalException(exception)) { @@ -309,7 +320,17 @@ private void recordSendError(final String topic, final Exception exception, fina "`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 (productionExceptionHandler.handle(serializedRecord, exception) == ProductionExceptionHandlerResponse.FAIL) { + ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( + context, + serializedRecord.topic(), + serializedRecord.partition(), + context.offset(), + serializedRecord.headers(), + null, + null, + processorNodeId, + taskId); + if (productionExceptionHandler.handle(errorHandlerContext, serializedRecord, exception) == ProductionExceptionHandlerResponse.FAIL) { errorMessage += "\nException handler choose to FAIL the processing, no more records would be sent."; sendException.set(new StreamsException(errorMessage, exception)); } else { 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 9c56a138acecd..65640470ef936 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 @@ -81,10 +81,19 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa final Sensor droppedRecordsSensor) { final DeserializationExceptionHandler.DeserializationHandlerResponse response; try { - response = deserializationExceptionHandler.handle( - (InternalProcessorContext) processorContext, - rawRecord, - deserializationException); + + ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( + (InternalProcessorContext) processorContext, + rawRecord.topic(), + rawRecord.partition(), + rawRecord.offset(), + rawRecord.headers(), + rawRecord.key(), + rawRecord.value(), + null, + processorContext.taskId()); + response = deserializationExceptionHandler.handle(errorHandlerContext, rawRecord, deserializationException); + } catch (final Exception fatalUserException) { log.error( "Deserialization error callback failed after deserialization error for record {}", diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RestrictiveProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RestrictiveProcessorContext.java new file mode 100644 index 0000000000000..a590f8d6c0b8d --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RestrictiveProcessorContext.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.processor.*; + +import java.io.File; +import java.time.Duration; +import java.util.Map; +import java.util.Objects; + +/** + * {@code ProcessorContext} implementation that will throw on any call . + */ +public final class RestrictiveProcessorContext implements ProcessorContext { + private final ProcessorContext delegate; + + private static final String EXPLANATION = "ProcessorContext#forward() is not supported from this context, " + + "as the framework must ensure the key is not changed (#forward allows changing the key on " + + "messages which are sent). Try another function, which doesn't allow the key to be changed " + + "(for example - #transformValues)."; + + public RestrictiveProcessorContext(final ProcessorContext delegate) { + this.delegate = Objects.requireNonNull(delegate, "delegate"); + } + + @Override + public String topic() { + return delegate.topic(); + } + + @Override + public int partition() { + return delegate.partition(); + } + + @Override + public long offset() { + return delegate.offset(); + } + + @Override + public Headers headers() { + return delegate.headers(); + } + + @Override + public TaskId taskId() { + return delegate.taskId(); + } + + @Override + public String applicationId() { + return delegate.applicationId(); + } + + @Override + public Serde keySerde() { + return delegate.keySerde(); + } + + @Override + public Serde valueSerde() { + return delegate.valueSerde(); + } + + @Override + public File stateDir() { + return delegate.stateDir(); + } + + @Override + public StreamsMetrics metrics() { + return delegate.metrics(); + } + + @Override + public long currentSystemTimeMs() { + return delegate.currentSystemTimeMs(); + } + + @Override + public long currentStreamTimeMs() { + return delegate.currentStreamTimeMs(); + } + + @Override + public long timestamp() { + return delegate.timestamp(); + } + + @Override + public Map appConfigs() { + return delegate.appConfigs(); + } + + @Override + public Map appConfigsWithPrefix(final String prefix) { + return delegate.appConfigsWithPrefix(prefix); + } + + @Override + public void register(final StateStore store, + final StateRestoreCallback stateRestoreCallback) { + throw new StreamsException(EXPLANATION); + } + + @Override + public S getStateStore(final String name) { + throw new StreamsException(EXPLANATION); + } + + @Override + public Cancellable schedule(final Duration interval, + final PunctuationType type, + final Punctuator callback) throws IllegalArgumentException { + throw new StreamsException(EXPLANATION); + } + + @Override + public void forward(final K key, final V value) { + throw new StreamsException(EXPLANATION); + } + + @Override + public void forward(final K key, final V value, final To to) { + throw new StreamsException(EXPLANATION); + } + + @Override + public void commit() { + throw new StreamsException(EXPLANATION); + } + + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java b/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java index be1e98e4a7154..c247043a3d8cd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java +++ b/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java @@ -26,12 +26,14 @@ */ public class AlwaysContinueProductionExceptionHandler implements ProductionExceptionHandler { @Override + @Deprecated public ProductionExceptionHandlerResponse handle(final ProducerRecord record, final Exception exception) { return ProductionExceptionHandlerResponse.CONTINUE; } @Override + @Deprecated public ProductionExceptionHandlerResponse handleSerializationException(final ProducerRecord record, final Exception exception) { return ProductionExceptionHandlerResponse.CONTINUE; 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 448ceaf67014a..5d4a67c84375c 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 @@ -24,11 +24,15 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.LogContext; +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.test.InternalMockProcessorContext; import org.junit.Test; import java.util.Optional; -import static org.junit.Assert.assertEquals; +import static org.junit.Assert.*; public class RecordDeserializerTest { @@ -45,6 +49,8 @@ public class RecordDeserializerTest { headers, Optional.empty()); + private final InternalProcessorContext context = new InternalMockProcessorContext<>(); + @Test public void shouldReturnConsumerRecordWithDeserializedValueWhenNoExceptions() { final RecordDeserializer recordDeserializer = new RecordDeserializer( @@ -68,6 +74,70 @@ public void shouldReturnConsumerRecordWithDeserializedValueWhenNoExceptions() { assertEquals(rawRecord.headers(), record.headers()); } + @Test + public void shouldThrowExceptionWithKeyDeserializationAndFail() { + final RecordDeserializer recordDeserializer = new RecordDeserializer( + new TheSourceNode( + true, + false, + "key", "value" + ), + new LogAndFailExceptionHandler(), + new LogContext(), + new Metrics().sensor("dropped-records") + ); + StreamsException e = assertThrows(StreamsException.class, () -> recordDeserializer.deserialize(context, rawRecord)); + assertEquals(e.getMessage(), "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 default.deserialization.exception.handler appropriately."); + } + + @Test + public void shouldThrowExceptionWithValueDeserializationAndFail() { + final RecordDeserializer recordDeserializer = new RecordDeserializer( + new TheSourceNode( + false, + true, + "key", "value" + ), + new LogAndFailExceptionHandler(), + new LogContext(), + new Metrics().sensor("dropped-records") + ); + StreamsException e = assertThrows(StreamsException.class, () -> recordDeserializer.deserialize(context, rawRecord)); + assertEquals(e.getMessage(), "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 default.deserialization.exception.handler appropriately."); + } + + @Test + public void shouldThrowExceptionWithKeyDeserializationAndContinue() { + final RecordDeserializer recordDeserializer = new RecordDeserializer( + new TheSourceNode( + true, + false, + "key", "value" + ), + new LogAndContinueExceptionHandler(), + new LogContext(), + new Metrics().sensor("dropped-records") + ); + final ConsumerRecord record = recordDeserializer.deserialize(context, rawRecord); + assertNull(record); + } + + @Test + public void shouldThrowExceptionWithValueDeserializationAndContinue() { + final RecordDeserializer recordDeserializer = new RecordDeserializer( + new TheSourceNode( + false, + true, + "key", "value" + ), + new LogAndContinueExceptionHandler(), + new LogContext(), + new Metrics().sensor("dropped-records") + ); + final ConsumerRecord record = recordDeserializer.deserialize(context, rawRecord); + assertNull(record); + } + static class TheSourceNode extends SourceNode { private final boolean keyThrowsException; private final boolean valueThrowsException; From 5f621c2343d7829d9d04f11d10e9123f7f25190e Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Thu, 16 May 2024 13:29:13 +0200 Subject: [PATCH 09/28] KAFKA-16448: fix checkstyle --- gradle.properties | 2 +- .../processor/ErrorHandlerContext.java | 3 +- .../internals/ErrorHandlerContextImpl.java | 20 +++---- .../internals/ProcessorContextImpl.java | 2 +- .../internals/RecordCollectorImpl.java | 4 +- .../internals/RecordDeserializer.java | 52 +++++++++--------- .../RestrictiveProcessorContext.java | 9 +++- ...aysContinueProductionExceptionHandler.java | 13 ----- .../internals/RecordDeserializerTest.java | 53 ++++++++++--------- 9 files changed, 76 insertions(+), 82 deletions(-) diff --git a/gradle.properties b/gradle.properties index aa2a3d5f25ada..4880248cac320 100644 --- a/gradle.properties +++ b/gradle.properties @@ -23,7 +23,7 @@ group=org.apache.kafka # - streams/quickstart/pom.xml # - streams/quickstart/java/src/main/resources/archetype-resources/pom.xml # - streams/quickstart/java/pom.xml -version=3.8.0-SNAPSHOT-KIP-1033 +version=3.8.0-SNAPSHOT scalaVersion=2.13.12 # Adding swaggerVersion in gradle.properties to have a single version in place for swagger # New version of Swagger 2.2.14 requires minimum JDK 11. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java index b763a97deb93b..694892c21de4f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java @@ -18,8 +18,7 @@ import org.apache.kafka.common.header.Headers; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; -import org.apache.kafka.streams.processor.api.ProcessorContext; -import org.apache.kafka.streams.processor.internals.InternalProcessorContext; + /** * ErrorHandlerContext interface diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java index 4a18005b436da..a67a58482e8d7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java @@ -34,15 +34,15 @@ public class ErrorHandlerContextImpl implements ErrorHandlerContext { private final String processorNodeId; private final TaskId taskId; - public ErrorHandlerContextImpl(InternalProcessorContext processorContext, - String topic, - int partition, - long offset, - Headers headers, - byte[] sourceRawKey, - byte[] sourceRawValue, - String processorNodeId, - TaskId taskId) { + public ErrorHandlerContextImpl(final InternalProcessorContext processorContext, + final String topic, + final int partition, + final long offset, + final Headers headers, + final byte[] sourceRawKey, + final byte[] sourceRawValue, + final String processorNodeId, + final TaskId taskId) { this.processorContext = processorContext; this.topic = topic; this.partition = partition; @@ -71,7 +71,7 @@ public long offset() { @Override public Headers headers() { - return this.headers; + return this.headers; } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index d3188bda80ca0..69e24d80ac81f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -303,7 +303,7 @@ private void forwardInternal(final ProcessorNode child, if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { throw new StreamsException("Processing exception handler is set to fail upon" + " a processing error. If you would rather have the streaming pipeline" + - " continue after a deserialization error, please set the " + + " continue after a processing error, please set the " + PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.", e); } 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 0cade6644ddf9..7850c9f1eca3b 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 @@ -219,7 +219,7 @@ public void send(final String topic, log.debug(String.format("Error serializing record to topic %s", topic), exception); try { - ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( + final ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( context, record.topic(), record.partition(), @@ -320,7 +320,7 @@ private void recordSendError(final String topic, final Exception exception, fina "`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 { - ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( + final ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( context, serializedRecord.topic(), serializedRecord.partition(), 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 65640470ef936..4ba183a68e664 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 @@ -55,17 +55,17 @@ ConsumerRecord deserialize(final ProcessorContext processo try { return new ConsumerRecord<>( - rawRecord.topic(), - rawRecord.partition(), - rawRecord.offset(), - rawRecord.timestamp(), - TimestampType.CREATE_TIME, - rawRecord.serializedKeySize(), - rawRecord.serializedValueSize(), - sourceNode.deserializeKey(rawRecord.topic(), rawRecord.headers(), rawRecord.key()), - sourceNode.deserializeValue(rawRecord.topic(), rawRecord.headers(), rawRecord.value()), - rawRecord.headers(), - Optional.empty() + rawRecord.topic(), + rawRecord.partition(), + rawRecord.offset(), + rawRecord.timestamp(), + TimestampType.CREATE_TIME, + rawRecord.serializedKeySize(), + rawRecord.serializedValueSize(), + sourceNode.deserializeKey(rawRecord.topic(), rawRecord.headers(), rawRecord.key()), + sourceNode.deserializeValue(rawRecord.topic(), rawRecord.headers(), rawRecord.value()), + rawRecord.headers(), + Optional.empty() ); } catch (final Exception deserializationException) { handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor); @@ -81,8 +81,7 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa final Sensor droppedRecordsSensor) { final DeserializationExceptionHandler.DeserializationHandlerResponse response; try { - - ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( + final ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( (InternalProcessorContext) processorContext, rawRecord.topic(), rawRecord.partition(), @@ -92,28 +91,27 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa rawRecord.value(), null, processorContext.taskId()); - response = deserializationExceptionHandler.handle(errorHandlerContext, rawRecord, deserializationException); - + response = deserializationExceptionHandler.handle(errorHandlerContext, rawRecord, deserializationException); } catch (final Exception fatalUserException) { log.error( - "Deserialization error callback failed after deserialization error for record {}", - rawRecord, - deserializationException); + "Deserialization error callback failed after deserialization error for record {}", + rawRecord, + deserializationException); throw new StreamsException("Fatal user code error in deserialization error callback", fatalUserException); } if (response == DeserializationExceptionHandler.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 " + - DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.", - deserializationException); + " a deserialization error. If you would rather have the streaming pipeline" + + " continue after a deserialization error, please set the " + + DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.", + deserializationException); } else { log.warn( - "Skipping record due to deserialization error. topic=[{}] partition=[{}] offset=[{}]", - rawRecord.topic(), - rawRecord.partition(), - rawRecord.offset(), - deserializationException + "Skipping record due to deserialization error. topic=[{}] partition=[{}] offset=[{}]", + rawRecord.topic(), + rawRecord.partition(), + rawRecord.offset(), + deserializationException ); droppedRecordsSensor.record(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RestrictiveProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RestrictiveProcessorContext.java index a590f8d6c0b8d..268fe237455da 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RestrictiveProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RestrictiveProcessorContext.java @@ -20,7 +20,14 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.StreamsMetrics; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.*; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.Cancellable; +import org.apache.kafka.streams.processor.PunctuationType; +import org.apache.kafka.streams.processor.Punctuator; +import org.apache.kafka.streams.processor.StateRestoreCallback; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.To; import java.io.File; import java.time.Duration; diff --git a/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java b/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java index 2641fe4339f3d..c247043a3d8cd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java +++ b/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.errors; import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.streams.processor.ErrorHandlerContext; import java.util.Map; @@ -27,28 +26,16 @@ */ public class AlwaysContinueProductionExceptionHandler implements ProductionExceptionHandler { @Override -<<<<<<< HEAD @Deprecated public ProductionExceptionHandlerResponse handle(final ProducerRecord record, -======= - public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, - final ProducerRecord record, ->>>>>>> 5ffca71fd6d93af521d32fc3a9d32824f0554a53 final Exception exception) { return ProductionExceptionHandlerResponse.CONTINUE; } @Override -<<<<<<< HEAD @Deprecated public ProductionExceptionHandlerResponse handleSerializationException(final ProducerRecord record, final Exception exception) { -======= - public ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context, - final ProducerRecord record, - final Exception exception, - final SerializationExceptionOrigin origin) { ->>>>>>> 5ffca71fd6d93af521d32fc3a9d32824f0554a53 return ProductionExceptionHandlerResponse.CONTINUE; } 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 5d4a67c84375c..db731118f4fa9 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 @@ -32,36 +32,39 @@ import java.util.Optional; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertNull; + public class RecordDeserializerTest { - private final RecordHeaders headers = new RecordHeaders(new Header[] {new RecordHeader("key", "value".getBytes())}); + private final RecordHeaders headers = new RecordHeaders(new Header[]{new RecordHeader("key", "value".getBytes())}); private final ConsumerRecord rawRecord = new ConsumerRecord<>("topic", - 1, - 1, - 10, - TimestampType.LOG_APPEND_TIME, - 3, - 5, - new byte[0], - new byte[0], - headers, - Optional.empty()); + 1, + 1, + 10, + TimestampType.LOG_APPEND_TIME, + 3, + 5, + new byte[0], + new byte[0], + headers, + Optional.empty()); private final InternalProcessorContext context = new InternalMockProcessorContext<>(); @Test public void shouldReturnConsumerRecordWithDeserializedValueWhenNoExceptions() { final RecordDeserializer recordDeserializer = new RecordDeserializer( - new TheSourceNode( - false, - false, - "key", "value" - ), - null, - new LogContext(), - new Metrics().sensor("dropped-records") + new TheSourceNode( + false, + false, + "key", "value" + ), + null, + new LogContext(), + new Metrics().sensor("dropped-records") ); final ConsumerRecord record = recordDeserializer.deserialize(null, rawRecord); assertEquals(rawRecord.topic(), record.topic()); @@ -86,8 +89,8 @@ public void shouldThrowExceptionWithKeyDeserializationAndFail() { new LogContext(), new Metrics().sensor("dropped-records") ); - StreamsException e = assertThrows(StreamsException.class, () -> recordDeserializer.deserialize(context, rawRecord)); - assertEquals(e.getMessage(), "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 default.deserialization.exception.handler appropriately."); + final StreamsException e = assertThrows(StreamsException.class, () -> recordDeserializer.deserialize(context, rawRecord)); + assertEquals(e.getMessage(), "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 default.deserialization.exception.handler appropriately."); } @Test @@ -102,7 +105,7 @@ public void shouldThrowExceptionWithValueDeserializationAndFail() { new LogContext(), new Metrics().sensor("dropped-records") ); - StreamsException e = assertThrows(StreamsException.class, () -> recordDeserializer.deserialize(context, rawRecord)); + final StreamsException e = assertThrows(StreamsException.class, () -> recordDeserializer.deserialize(context, rawRecord)); assertEquals(e.getMessage(), "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 default.deserialization.exception.handler appropriately."); } @@ -118,7 +121,7 @@ public void shouldThrowExceptionWithKeyDeserializationAndContinue() { new LogContext(), new Metrics().sensor("dropped-records") ); - final ConsumerRecord record = recordDeserializer.deserialize(context, rawRecord); + final ConsumerRecord record = recordDeserializer.deserialize(context, rawRecord); assertNull(record); } @@ -134,7 +137,7 @@ public void shouldThrowExceptionWithValueDeserializationAndContinue() { new LogContext(), new Metrics().sensor("dropped-records") ); - final ConsumerRecord record = recordDeserializer.deserialize(context, rawRecord); + final ConsumerRecord record = recordDeserializer.deserialize(context, rawRecord); assertNull(record); } From 2b7a528707bf3ad17e37f174d921f622fda60603 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Fri, 17 May 2024 13:47:05 +0200 Subject: [PATCH 10/28] KAFKA-16448: Provide rawSourceKey and rawSourceValue in the ErrorHandlerContext --- .../apache/kafka/streams/StreamsConfig.java | 1219 ++++++++--------- .../DeserializationExceptionHandler.java | 2 +- .../processor/ErrorHandlerContext.java | 4 +- .../processor/internals/CorruptedRecord.java | 2 +- .../internals/ProcessorContextImpl.java | 6 +- .../processor/internals/RecordQueue.java | 2 +- .../processor/internals/SourceNode.java | 1 - .../processor/internals/StampedRecord.java | 8 +- .../processor/internals/StreamTask.java | 4 + 9 files changed, 568 insertions(+), 680 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 32f81b243b118..b1a27056f473e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams; +import java.util.Arrays; +import java.util.HashSet; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AdminClientConfig; @@ -54,10 +56,8 @@ import java.io.File; import java.time.Duration; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -96,7 +96,7 @@ * * StreamsConfig streamsConfig = new StreamsConfig(streamsProperties); * } - *

+ * * This instance can also be used to pass in custom configurations to different modules (e.g. passing a special config in your customized serde class). * The consumer/producer/admin prefix can also be used to distinguish these custom config values passed to different clients with the same config name. * * Example: @@ -114,28 +114,28 @@ * // and "my.custom.config2" -> "boom" from the config map * StreamsConfig streamsConfig = new StreamsConfig(streamsProperties); * } - *

- *

+ * + * * When increasing {@link ProducerConfig#MAX_BLOCK_MS_CONFIG} to be more resilient to non-available brokers you should also * increase {@link ConsumerConfig#MAX_POLL_INTERVAL_MS_CONFIG} using the following guidance: *

  *     max.poll.interval.ms > max.block.ms
  * 
- *

- *

+ * + * * Kafka Streams requires at least the following properties to be set: *

- *

+ * * By default, Kafka Streams does not allow users to overwrite the following properties (Streams setting shown in parentheses): *

- *

+ * * If {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} is set to {@link #EXACTLY_ONCE_V2 "exactly_once_v2"}, * {@link #EXACTLY_ONCE "exactly_once"} (deprecated), or {@link #EXACTLY_ONCE_BETA "exactly_once_beta"} (deprecated), Kafka Streams does not * allow users to overwrite the following properties (Streams setting shown in parentheses): @@ -242,20 +242,17 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String CLIENT_TAG_PREFIX = "client.tag."; - /** - * {@code topology.optimization} - */ + /** {@code topology.optimization} */ public static final String TOPOLOGY_OPTIMIZATION_CONFIG = "topology.optimization"; - private static final String CONFIG_ERROR_MSG = "Acceptable values are:" - + " \"+NO_OPTIMIZATION+\", \"+OPTIMIZE+\", " - + "or a comma separated list of specific optimizations: " - + "(\"+REUSE_KTABLE_SOURCE_TOPICS+\", \"+MERGE_REPARTITION_TOPICS+\" + " - + "\"SINGLE_STORE_SELF_JOIN+\")."; + + " \"+NO_OPTIMIZATION+\", \"+OPTIMIZE+\", " + + "or a comma separated list of specific optimizations: " + + "(\"+REUSE_KTABLE_SOURCE_TOPICS+\", \"+MERGE_REPARTITION_TOPICS+\" + " + + "\"SINGLE_STORE_SELF_JOIN+\")."; private static final String TOPOLOGY_OPTIMIZATION_DOC = "A configuration telling Kafka " - + "Streams if it should optimize the topology and what optimizations to apply. " - + CONFIG_ERROR_MSG - + "\"NO_OPTIMIZATION\" by default."; + + "Streams if it should optimize the topology and what optimizations to apply. " + + CONFIG_ERROR_MSG + + "\"NO_OPTIMIZATION\" by default."; /** * Config value for parameter {@link #TOPOLOGY_OPTIMIZATION_CONFIG "topology.optimization"} for disabling topology optimization @@ -288,8 +285,8 @@ public class StreamsConfig extends AbstractConfig { public static final String SINGLE_STORE_SELF_JOIN = "single.store.self.join"; private static final List TOPOLOGY_OPTIMIZATION_CONFIGS = Arrays.asList( - OPTIMIZE, NO_OPTIMIZATION, REUSE_KTABLE_SOURCE_TOPICS, MERGE_REPARTITION_TOPICS, - SINGLE_STORE_SELF_JOIN); + OPTIMIZE, NO_OPTIMIZATION, REUSE_KTABLE_SOURCE_TOPICS, MERGE_REPARTITION_TOPICS, + SINGLE_STORE_SELF_JOIN); /** * Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 0.10.0.x}. @@ -474,131 +471,97 @@ public class StreamsConfig extends AbstractConfig { */ public static final String METRICS_LATEST = "latest"; - /** - * {@code acceptable.recovery.lag} - */ + /** {@code acceptable.recovery.lag} */ public static final String ACCEPTABLE_RECOVERY_LAG_CONFIG = "acceptable.recovery.lag"; private static final String ACCEPTABLE_RECOVERY_LAG_DOC = "The maximum acceptable lag (number of offsets to catch up) for a client to be considered caught-up enough" + - " to receive an active task assignment. Upon assignment, it will still restore the rest of the changelog" + - " before processing. To avoid a pause in processing during rebalances, this config" + - " should correspond to a recovery time of well under a minute for a given workload. Must be at least 0."; + " to receive an active task assignment. Upon assignment, it will still restore the rest of the changelog" + + " before processing. To avoid a pause in processing during rebalances, this config" + + " should correspond to a recovery time of well under a minute for a given workload. Must be at least 0."; - /** - * {@code application.id} - */ + /** {@code application.id} */ @SuppressWarnings("WeakerAccess") public static final String APPLICATION_ID_CONFIG = "application.id"; private static final String APPLICATION_ID_DOC = "An identifier for the stream processing application. Must be unique within the Kafka cluster. It is used as 1) the default client-id prefix, 2) the group-id for membership management, 3) the changelog topic prefix."; - /** - * {@code application.server} - */ + /**{@code application.server} */ @SuppressWarnings("WeakerAccess") public static final String APPLICATION_SERVER_CONFIG = "application.server"; private static final String APPLICATION_SERVER_DOC = "A host:port pair pointing to a user-defined endpoint that can be used for state store discovery and interactive queries on this KafkaStreams instance."; - /** - * {@code bootstrap.servers} - */ + /** {@code bootstrap.servers} */ @SuppressWarnings("WeakerAccess") public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; - /** - * {@code buffered.records.per.partition} - */ + /** {@code buffered.records.per.partition} */ @SuppressWarnings("WeakerAccess") public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition"; public static final String BUFFERED_RECORDS_PER_PARTITION_DOC = "Maximum number of records to buffer per partition."; - /** - * {@code built.in.metrics.version} - */ + /** {@code built.in.metrics.version} */ public static final String BUILT_IN_METRICS_VERSION_CONFIG = "built.in.metrics.version"; private static final String BUILT_IN_METRICS_VERSION_DOC = "Version of the built-in metrics to use."; - /** - * {@code cache.max.bytes.buffering} - * - * @deprecated since 3.4.0 Use {@link #STATESTORE_CACHE_MAX_BYTES_CONFIG "statestore.cache.max.bytes"} instead. - */ + /** {@code cache.max.bytes.buffering} + * @deprecated since 3.4.0 Use {@link #STATESTORE_CACHE_MAX_BYTES_CONFIG "statestore.cache.max.bytes"} instead. */ @SuppressWarnings("WeakerAccess") @Deprecated public static final String CACHE_MAX_BYTES_BUFFERING_CONFIG = "cache.max.bytes.buffering"; public static final String CACHE_MAX_BYTES_BUFFERING_DOC = "Maximum number of memory bytes to be used for buffering across all threads"; - /** - * {@code statestore.cache.max.bytes} - */ + /** {@code statestore.cache.max.bytes} */ @SuppressWarnings("WeakerAccess") public static final String STATESTORE_CACHE_MAX_BYTES_CONFIG = "statestore.cache.max.bytes"; public static final String STATESTORE_CACHE_MAX_BYTES_DOC = "Maximum number of memory bytes to be used for statestore cache across all threads"; - /** - * {@code client.id} - */ + /** {@code client.id} */ @SuppressWarnings("WeakerAccess") public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; private static final String CLIENT_ID_DOC = "An ID prefix string used for the client IDs of internal (main, restore, and global) consumers , producers, and admin clients" + - " with pattern <client.id>-[Global]StreamThread[-<threadSequenceNumber>]-<consumer|producer|restore-consumer|global-consumer>."; + " with pattern <client.id>-[Global]StreamThread[-<threadSequenceNumber>]-<consumer|producer|restore-consumer|global-consumer>."; - /** - * {@code enable.metrics.push} - */ + /** {@code enable.metrics.push} */ @SuppressWarnings("WeakerAccess") - public static final String ENABLE_METRICS_PUSH_CONFIG = CommonClientConfigs.ENABLE_METRICS_PUSH_CONFIG; + public static final String ENABLE_METRICS_PUSH_CONFIG = CommonClientConfigs.ENABLE_METRICS_PUSH_CONFIG; public static final String ENABLE_METRICS_PUSH_DOC = "Whether to enable pushing of internal client metrics for (main, restore, and global) consumers, producers, and admin clients." + - " The cluster must have a client metrics subscription which corresponds to a client."; + " The cluster must have a client metrics subscription which corresponds to a client."; - /** - * {@code commit.interval.ms} - */ + /** {@code commit.interval.ms} */ @SuppressWarnings("WeakerAccess") public static final String COMMIT_INTERVAL_MS_CONFIG = "commit.interval.ms"; private static final String COMMIT_INTERVAL_MS_DOC = "The frequency in milliseconds with which to commit processing progress." + - " For at-least-once processing, committing means to save the position (ie, offsets) of the processor." + - " For exactly-once processing, it means to commit the transaction which includes to save the position and to make the committed data in the output topic visible to consumers with isolation level read_committed." + - " (Note, if processing.guarantee is set to " + EXACTLY_ONCE_V2 + ", " + EXACTLY_ONCE + ",the default value is " + EOS_DEFAULT_COMMIT_INTERVAL_MS + "," + - " otherwise the default value is " + DEFAULT_COMMIT_INTERVAL_MS + "."; + " For at-least-once processing, committing means to save the position (ie, offsets) of the processor." + + " For exactly-once processing, it means to commit the transaction which includes to save the position and to make the committed data in the output topic visible to consumers with isolation level read_committed." + + " (Note, if processing.guarantee is set to " + EXACTLY_ONCE_V2 + ", " + EXACTLY_ONCE + ",the default value is " + EOS_DEFAULT_COMMIT_INTERVAL_MS + "," + + " otherwise the default value is " + DEFAULT_COMMIT_INTERVAL_MS + "."; - /** - * {@code repartition.purge.interval.ms} - */ + /** {@code repartition.purge.interval.ms} */ @SuppressWarnings("WeakerAccess") public static final String REPARTITION_PURGE_INTERVAL_MS_CONFIG = "repartition.purge.interval.ms"; private static final String REPARTITION_PURGE_INTERVAL_MS_DOC = "The frequency in milliseconds with which to delete fully consumed records from repartition topics." + - " Purging will occur after at least this value since the last purge, but may be delayed until later." + - " (Note, unlike commit.interval.ms, the default for this value remains unchanged when processing.guarantee is set to " + EXACTLY_ONCE_V2 + ")."; + " Purging will occur after at least this value since the last purge, but may be delayed until later." + + " (Note, unlike commit.interval.ms, the default for this value remains unchanged when processing.guarantee is set to " + EXACTLY_ONCE_V2 + ")."; - /** - * {@code connections.max.idle.ms} - */ + /** {@code connections.max.idle.ms} */ @SuppressWarnings("WeakerAccess") public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; - /** - * {@code default.deserialization.exception.handler} - */ + /** {@code default.deserialization.exception.handler} */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG = "default.deserialization.exception.handler"; public static final String DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the org.apache.kafka.streams.errors.DeserializationExceptionHandler interface."; - /** - * {@code default.production.exception.handler} - */ + /** {@code default.production.exception.handler} */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG = "default.production.exception.handler"; private static final String DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the org.apache.kafka.streams.errors.ProductionExceptionHandler interface."; - /** - * {@code default.deserialization.exception.handler} - */ + /** {@code default.deserialization.exception.handler} */ @SuppressWarnings("WeakerAccess") public static final String PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG = "processing.exception.handler"; public static final String PROCESSING_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the org.apache.kafka.streams.errors.ProcessingExceptionHandler interface."; - /** - * {@code default.dsl.store} - */ + + /** {@code default.dsl.store} */ @Deprecated @SuppressWarnings("WeakerAccess") public static final String DEFAULT_DSL_STORE_CONFIG = "default.dsl.store"; @@ -612,212 +575,159 @@ public class StreamsConfig extends AbstractConfig { @Deprecated public static final String DEFAULT_DSL_STORE = ROCKS_DB; - /** - * {@code dsl.store.suppliers.class } - */ + /** {@code dsl.store.suppliers.class } */ public static final String DSL_STORE_SUPPLIERS_CLASS_CONFIG = "dsl.store.suppliers.class"; static final String DSL_STORE_SUPPLIERS_CLASS_DOC = "Defines which store implementations to plug in to DSL operators. Must implement the org.apache.kafka.streams.state.DslStoreSuppliers interface."; static final Class DSL_STORE_SUPPLIERS_CLASS_DEFAULT = BuiltInDslStoreSuppliers.RocksDBDslStoreSuppliers.class; - /** - * {@code default.windowed.key.serde.inner - * - * @deprecated since 3.0.0 Use {@link #WINDOWED_INNER_CLASS_SERDE "windowed.inner.class.serde"} instead.} - */ + /** {@code default.windowed.key.serde.inner} + * @deprecated since 3.0.0 Use {@link #WINDOWED_INNER_CLASS_SERDE "windowed.inner.class.serde"} instead. */ @SuppressWarnings("WeakerAccess") @Deprecated public static final String DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS = "default.windowed.key.serde.inner"; private static final String DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS_DOC = "Default serializer / deserializer for the inner class of a windowed key. Must implement the " + - "org.apache.kafka.common.serialization.Serde interface."; + "org.apache.kafka.common.serialization.Serde interface."; - /** - * {@code default.windowed.value.serde.inner - * - * @deprecated since 3.0.0 Use {@link #WINDOWED_INNER_CLASS_SERDE "windowed.inner.class.serde"} instead.} - */ + /** {@code default.windowed.value.serde.inner} + * @deprecated since 3.0.0 Use {@link #WINDOWED_INNER_CLASS_SERDE "windowed.inner.class.serde"} instead. */ @SuppressWarnings("WeakerAccess") @Deprecated public static final String DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS = "default.windowed.value.serde.inner"; private static final String DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS_DOC = "Default serializer / deserializer for the inner class of a windowed value. Must implement the " + - "org.apache.kafka.common.serialization.Serde interface."; + "org.apache.kafka.common.serialization.Serde interface."; public static final String WINDOWED_INNER_CLASS_SERDE = "windowed.inner.class.serde"; private static final String WINDOWED_INNER_CLASS_SERDE_DOC = " Default serializer / deserializer for the inner class of a windowed record. Must implement the " + - "org.apache.kafka.common.serialization.Serde interface. Note that setting this config in KafkaStreams application would result " + - "in an error as it is meant to be used only from Plain consumer client."; + "org.apache.kafka.common.serialization.Serde interface. Note that setting this config in KafkaStreams application would result " + + "in an error as it is meant to be used only from Plain consumer client."; - /** - * {@code default key.serde} - */ + /** {@code default key.serde} */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_KEY_SERDE_CLASS_CONFIG = "default.key.serde"; private static final String DEFAULT_KEY_SERDE_CLASS_DOC = "Default serializer / deserializer class for key that implements the org.apache.kafka.common.serialization.Serde interface. " - + "Note when windowed serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" - + DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS + "' as well"; + + "Note when windowed serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + + DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS + "' as well"; - /** - * {@code default value.serde} - */ + /** {@code default value.serde} */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_VALUE_SERDE_CLASS_CONFIG = "default.value.serde"; private static final String DEFAULT_VALUE_SERDE_CLASS_DOC = "Default serializer / deserializer class for value that implements the org.apache.kafka.common.serialization.Serde interface. " - + "Note when windowed serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" - + DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS + "' as well"; + + "Note when windowed serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + + DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS + "' or '" + DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS + "' as well"; - /** - * {@code default.timestamp.extractor} - */ + /** {@code default.timestamp.extractor} */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG = "default.timestamp.extractor"; public static final String DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC = "Default timestamp extractor class that implements the org.apache.kafka.streams.processor.TimestampExtractor interface."; - /** - * {@code max.task.idle.ms} - */ + /** {@code max.task.idle.ms} */ public static final String MAX_TASK_IDLE_MS_CONFIG = "max.task.idle.ms"; public static final String MAX_TASK_IDLE_MS_DOC = "This config controls whether joins and merges" - + " may produce out-of-order results." - + " The config value is the maximum amount of time in milliseconds a stream task will stay idle" - + " when it is fully caught up on some (but not all) input partitions" - + " to wait for producers to send additional records and avoid potential" - + " out-of-order record processing across multiple input streams." - + " The default (zero) does not wait for producers to send more records," - + " but it does wait to fetch data that is already present on the brokers." - + " This default means that for records that are already present on the brokers," - + " Streams will process them in timestamp order." - + " Set to -1 to disable idling entirely and process any locally available data," - + " even though doing so may produce out-of-order processing."; - - /** - * {@code max.warmup.replicas} - */ + + " may produce out-of-order results." + + " The config value is the maximum amount of time in milliseconds a stream task will stay idle" + + " when it is fully caught up on some (but not all) input partitions" + + " to wait for producers to send additional records and avoid potential" + + " out-of-order record processing across multiple input streams." + + " The default (zero) does not wait for producers to send more records," + + " but it does wait to fetch data that is already present on the brokers." + + " This default means that for records that are already present on the brokers," + + " Streams will process them in timestamp order." + + " Set to -1 to disable idling entirely and process any locally available data," + + " even though doing so may produce out-of-order processing."; + + /** {@code max.warmup.replicas} */ public static final String MAX_WARMUP_REPLICAS_CONFIG = "max.warmup.replicas"; private static final String MAX_WARMUP_REPLICAS_DOC = "The maximum number of warmup replicas (extra standbys beyond the configured num.standbys) that can be assigned at once for the purpose of keeping " + - " the task available on one instance while it is warming up on another instance it has been reassigned to. Used to throttle how much extra broker " + - " traffic and cluster state can be used for high availability. Must be at least 1." + - "Note that one warmup replica corresponds to one Stream Task. Furthermore, note that each warmup replica can only be promoted to an active task " + - "during a rebalance (normally during a so-called probing rebalance, which occur at a frequency specified by the `probing.rebalance.interval.ms` config). This means " + - "that the maximum rate at which active tasks can be migrated from one Kafka Streams Instance to another instance can be determined by " + - "(`max.warmup.replicas` / `probing.rebalance.interval.ms`)."; + " the task available on one instance while it is warming up on another instance it has been reassigned to. Used to throttle how much extra broker " + + " traffic and cluster state can be used for high availability. Must be at least 1." + + "Note that one warmup replica corresponds to one Stream Task. Furthermore, note that each warmup replica can only be promoted to an active task " + + "during a rebalance (normally during a so-called probing rebalance, which occur at a frequency specified by the `probing.rebalance.interval.ms` config). This means " + + "that the maximum rate at which active tasks can be migrated from one Kafka Streams Instance to another instance can be determined by " + + "(`max.warmup.replicas` / `probing.rebalance.interval.ms`)."; - /** - * {@code metadata.max.age.ms} - */ + /** {@code metadata.max.age.ms} */ @SuppressWarnings("WeakerAccess") public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; - /** - * {@code metrics.num.samples} - */ + /** {@code metrics.num.samples} */ @SuppressWarnings("WeakerAccess") public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG; - /** - * {@code metrics.record.level} - */ + /** {@code metrics.record.level} */ @SuppressWarnings("WeakerAccess") public static final String METRICS_RECORDING_LEVEL_CONFIG = CommonClientConfigs.METRICS_RECORDING_LEVEL_CONFIG; - /** - * {@code metric.reporters} - */ + /** {@code metric.reporters} */ @SuppressWarnings("WeakerAccess") public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG; - /** - * {@code metrics.sample.window.ms} - */ + /** {@code metrics.sample.window.ms} */ @SuppressWarnings("WeakerAccess") public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG; - /** - * {@code auto.include.jmx.reporter - * - * @deprecated and will removed in 4.0.0 Use {@link JMX_REPORTER "jmx.reporter"} instead.} - */ + /** {@code auto.include.jmx.reporter} + * @deprecated and will be removed in 4.0.0 */ @Deprecated public static final String AUTO_INCLUDE_JMX_REPORTER_CONFIG = CommonClientConfigs.AUTO_INCLUDE_JMX_REPORTER_CONFIG; - /** - * {@code num.standby.replicas} - */ + /** {@code num.standby.replicas} */ @SuppressWarnings("WeakerAccess") public static final String NUM_STANDBY_REPLICAS_CONFIG = "num.standby.replicas"; private static final String NUM_STANDBY_REPLICAS_DOC = "The number of standby replicas for each task."; - /** - * {@code num.stream.threads} - */ + /** {@code num.stream.threads} */ @SuppressWarnings("WeakerAccess") public static final String NUM_STREAM_THREADS_CONFIG = "num.stream.threads"; private static final String NUM_STREAM_THREADS_DOC = "The number of threads to execute stream processing."; - /** - * {@code poll.ms} - */ + /** {@code poll.ms} */ @SuppressWarnings("WeakerAccess") public static final String POLL_MS_CONFIG = "poll.ms"; private static final String POLL_MS_DOC = "The amount of time in milliseconds to block waiting for input."; - /** - * {@code probing.rebalance.interval.ms} - */ + /** {@code probing.rebalance.interval.ms} */ public static final String PROBING_REBALANCE_INTERVAL_MS_CONFIG = "probing.rebalance.interval.ms"; private static final String PROBING_REBALANCE_INTERVAL_MS_DOC = "The maximum time in milliseconds to wait before triggering a rebalance to probe for warmup replicas that have finished warming up and are ready to become active." + - " Probing rebalances will continue to be triggered until the assignment is balanced. Must be at least 1 minute."; + " Probing rebalances will continue to be triggered until the assignment is balanced. Must be at least 1 minute."; - /** - * {@code processing.guarantee} - */ + /** {@code processing.guarantee} */ @SuppressWarnings("WeakerAccess") public static final String PROCESSING_GUARANTEE_CONFIG = "processing.guarantee"; private static final String PROCESSING_GUARANTEE_DOC = "The processing guarantee that should be used. " + - "Possible values are " + AT_LEAST_ONCE + " (default) " + - "and " + EXACTLY_ONCE_V2 + " (requires brokers version 2.5 or higher). " + - "Deprecated options are " + EXACTLY_ONCE + " (requires brokers version 0.11.0 or higher) " + - "and " + EXACTLY_ONCE_BETA + " (requires brokers version 2.5 or higher). " + - "Note that exactly-once processing requires a cluster of at least three brokers by default what is the " + - "recommended setting for production; for development you can change this, by adjusting broker setting " + - "transaction.state.log.replication.factor and transaction.state.log.min.isr."; + "Possible values are " + AT_LEAST_ONCE + " (default) " + + "and " + EXACTLY_ONCE_V2 + " (requires brokers version 2.5 or higher). " + + "Deprecated options are " + EXACTLY_ONCE + " (requires brokers version 0.11.0 or higher) " + + "and " + EXACTLY_ONCE_BETA + " (requires brokers version 2.5 or higher). " + + "Note that exactly-once processing requires a cluster of at least three brokers by default what is the " + + "recommended setting for production; for development you can change this, by adjusting broker setting " + + "transaction.state.log.replication.factor and transaction.state.log.min.isr."; - /** - * {@code receive.buffer.bytes} - */ + /** {@code receive.buffer.bytes} */ @SuppressWarnings("WeakerAccess") public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG; - /** - * {@code rack.aware.assignment.tags} - */ + /** {@code rack.aware.assignment.tags} */ @SuppressWarnings("WeakerAccess") public static final String RACK_AWARE_ASSIGNMENT_TAGS_CONFIG = "rack.aware.assignment.tags"; private static final String RACK_AWARE_ASSIGNMENT_TAGS_DOC = "List of client tag keys used to distribute standby replicas across Kafka Streams instances." + - " When configured, Kafka Streams will make a best-effort to distribute" + - " the standby tasks over each client tag dimension."; + " When configured, Kafka Streams will make a best-effort to distribute" + + " the standby tasks over each client tag dimension."; - /** - * {@code reconnect.backoff.ms} - */ + /** {@code reconnect.backoff.ms} */ @SuppressWarnings("WeakerAccess") public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; - /** - * {@code reconnect.backoff.max} - */ + /** {@code reconnect.backoff.max} */ @SuppressWarnings("WeakerAccess") public static final String RECONNECT_BACKOFF_MAX_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG; - /** - * {@code replication.factor} - */ + /** {@code replication.factor} */ @SuppressWarnings("WeakerAccess") public static final String REPLICATION_FACTOR_CONFIG = "replication.factor"; private static final String REPLICATION_FACTOR_DOC = "The replication factor for change log topics and repartition topics created by the stream processing application." - + " The default of -1 (meaning: use broker default replication factor) requires broker version 2.4 or newer"; + + " The default of -1 (meaning: use broker default replication factor) requires broker version 2.4 or newer"; - /** - * {@code request.timeout.ms} - */ + /** {@code request.timeout.ms} */ @SuppressWarnings("WeakerAccess") public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG; @@ -832,87 +742,65 @@ public class StreamsConfig extends AbstractConfig { @Deprecated public static final String RETRIES_CONFIG = CommonClientConfigs.RETRIES_CONFIG; - /** - * {@code retry.backoff.ms} - */ + /** {@code retry.backoff.ms} */ @SuppressWarnings("WeakerAccess") public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG; - /** - * {@code rocksdb.config.setter} - */ + /** {@code rocksdb.config.setter} */ @SuppressWarnings("WeakerAccess") public static final String ROCKSDB_CONFIG_SETTER_CLASS_CONFIG = "rocksdb.config.setter"; private static final String ROCKSDB_CONFIG_SETTER_CLASS_DOC = "A Rocks DB config setter class or class name that implements the org.apache.kafka.streams.state.RocksDBConfigSetter interface"; - /** - * {@code security.protocol} - */ + /** {@code security.protocol} */ @SuppressWarnings("WeakerAccess") public static final String SECURITY_PROTOCOL_CONFIG = CommonClientConfigs.SECURITY_PROTOCOL_CONFIG; - /** - * {@code send.buffer.bytes} - */ + /** {@code send.buffer.bytes} */ @SuppressWarnings("WeakerAccess") public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG; - /** - * {@code state.cleanup.delay} - */ + /** {@code state.cleanup.delay} */ @SuppressWarnings("WeakerAccess") public static final String STATE_CLEANUP_DELAY_MS_CONFIG = "state.cleanup.delay.ms"; private static final String STATE_CLEANUP_DELAY_MS_DOC = "The amount of time in milliseconds to wait before deleting state when a partition has migrated. Only state directories that have not been modified for at least state.cleanup.delay.ms will be removed"; - /** - * {@code state.dir} - */ + /** {@code state.dir} */ @SuppressWarnings("WeakerAccess") public static final String STATE_DIR_CONFIG = "state.dir"; private static final String STATE_DIR_DOC = "Directory location for state store. This path must be unique for each streams instance sharing the same underlying filesystem. Note that if not configured, then the default location will be different in each environment as it is computed using System.getProperty(\"java.io.tmpdir\")"; - /** - * {@code task.timeout.ms} - */ + /** {@code task.timeout.ms} */ public static final String TASK_TIMEOUT_MS_CONFIG = "task.timeout.ms"; public static final String TASK_TIMEOUT_MS_DOC = "The maximum amount of time in milliseconds a task might stall due to internal errors and retries until an error is raised. " + - "For a timeout of 0ms, a task would raise an error for the first internal error. " + - "For any timeout larger than 0ms, a task will retry at least once before an error is raised."; + "For a timeout of 0ms, a task would raise an error for the first internal error. " + + "For any timeout larger than 0ms, a task will retry at least once before an error is raised."; - /** - * {@code window.size.ms} - */ + /** {@code window.size.ms} */ public static final String WINDOW_SIZE_MS_CONFIG = "window.size.ms"; private static final String WINDOW_SIZE_MS_DOC = "Sets window size for the deserializer in order to calculate window end times."; - /** - * {@code upgrade.from} - */ + /** {@code upgrade.from} */ @SuppressWarnings("WeakerAccess") public static final String UPGRADE_FROM_CONFIG = "upgrade.from"; private static final String UPGRADE_FROM_DOC = "Allows upgrading in a backward compatible way. " + - "This is needed when upgrading from [0.10.0, 1.1] to 2.0+, or when upgrading from [2.0, 2.3] to 2.4+. " + - "When upgrading from 3.3 to a newer version it is not required to specify this config. Default is `null`. " + - "Accepted values are \"" + UPGRADE_FROM_0100 + "\", \"" + UPGRADE_FROM_0101 + "\", \"" + - UPGRADE_FROM_0102 + "\", \"" + UPGRADE_FROM_0110 + "\", \"" + UPGRADE_FROM_10 + "\", \"" + - UPGRADE_FROM_11 + "\", \"" + UPGRADE_FROM_20 + "\", \"" + UPGRADE_FROM_21 + "\", \"" + - UPGRADE_FROM_22 + "\", \"" + UPGRADE_FROM_23 + "\", \"" + UPGRADE_FROM_24 + "\", \"" + - UPGRADE_FROM_25 + "\", \"" + UPGRADE_FROM_26 + "\", \"" + UPGRADE_FROM_27 + "\", \"" + - UPGRADE_FROM_28 + "\", \"" + UPGRADE_FROM_30 + "\", \"" + UPGRADE_FROM_31 + "\", \"" + - UPGRADE_FROM_32 + "\", \"" + UPGRADE_FROM_33 + "\", \"" + UPGRADE_FROM_34 + "\", \"" + - UPGRADE_FROM_35 + "\", \"" + UPGRADE_FROM_36 + "\", \"" + UPGRADE_FROM_37 + "(for upgrading from the corresponding old version)."; - - /** - * {@code windowstore.changelog.additional.retention.ms} - */ + "This is needed when upgrading from [0.10.0, 1.1] to 2.0+, or when upgrading from [2.0, 2.3] to 2.4+. " + + "When upgrading from 3.3 to a newer version it is not required to specify this config. Default is `null`. " + + "Accepted values are \"" + UPGRADE_FROM_0100 + "\", \"" + UPGRADE_FROM_0101 + "\", \"" + + UPGRADE_FROM_0102 + "\", \"" + UPGRADE_FROM_0110 + "\", \"" + UPGRADE_FROM_10 + "\", \"" + + UPGRADE_FROM_11 + "\", \"" + UPGRADE_FROM_20 + "\", \"" + UPGRADE_FROM_21 + "\", \"" + + UPGRADE_FROM_22 + "\", \"" + UPGRADE_FROM_23 + "\", \"" + UPGRADE_FROM_24 + "\", \"" + + UPGRADE_FROM_25 + "\", \"" + UPGRADE_FROM_26 + "\", \"" + UPGRADE_FROM_27 + "\", \"" + + UPGRADE_FROM_28 + "\", \"" + UPGRADE_FROM_30 + "\", \"" + UPGRADE_FROM_31 + "\", \"" + + UPGRADE_FROM_32 + "\", \"" + UPGRADE_FROM_33 + "\", \"" + UPGRADE_FROM_34 + "\", \"" + + UPGRADE_FROM_35 + "\", \"" + UPGRADE_FROM_36 + "\", \"" + UPGRADE_FROM_37 + "(for upgrading from the corresponding old version)."; + + /** {@code windowstore.changelog.additional.retention.ms} */ @SuppressWarnings("WeakerAccess") public static final String WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG = "windowstore.changelog.additional.retention.ms"; private static final String WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC = "Added to a windows maintainMs to ensure data is not deleted from the log prematurely. Allows for clock drift. Default is 1 day"; - /** - * {@code default.client.supplier} - */ + /** {@code default.client.supplier} */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_CLIENT_SUPPLIER_CONFIG = "default.client.supplier"; public static final String DEFAULT_CLIENT_SUPPLIER_DOC = "Client supplier class that implements the org.apache.kafka.streams.KafkaClientSupplier interface."; @@ -921,31 +809,28 @@ public class StreamsConfig extends AbstractConfig { public static final String RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC = "min_traffic"; public static final String RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY = "balance_subtopology"; - /** - * {@code } rack.aware.assignment.strategy - */ + /** {@code } rack.aware.assignment.strategy */ @SuppressWarnings("WeakerAccess") public static final String RACK_AWARE_ASSIGNMENT_STRATEGY_CONFIG = "rack.aware.assignment.strategy"; public static final String RACK_AWARE_ASSIGNMENT_STRATEGY_DOC = "The strategy we use for rack aware assignment. Rack aware assignment will take client.rack and racks of TopicPartition into account when assigning" - + " tasks to minimize cross rack traffic. Valid settings are : " + RACK_AWARE_ASSIGNMENT_STRATEGY_NONE + " (default), which will disable rack aware assignment; " + RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC - + ", which will compute minimum cross rack traffic assignment; " + RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY + ", which will compute minimum cross rack traffic and try to balance the tasks of same subtopolgies across different clients"; + + " tasks to minimize cross rack traffic. Valid settings are : " + RACK_AWARE_ASSIGNMENT_STRATEGY_NONE + " (default), which will disable rack aware assignment; " + RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC + + ", which will compute minimum cross rack traffic assignment; " + RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY + ", which will compute minimum cross rack traffic and try to balance the tasks of same subtopolgies across different clients"; @SuppressWarnings("WeakerAccess") public static final String RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG = "rack.aware.assignment.traffic_cost"; public static final String RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_DOC = "Cost associated with cross rack traffic. This config and rack.aware.assignment.non_overlap_cost controls whether the " - + "optimization algorithm favors minimizing cross rack traffic or minimize the movement of tasks in existing assignment. If set a larger value " + RackAwareTaskAssignor.class.getName() + " will " - + "optimize for minimizing cross rack traffic. The default value is null which means it will use default traffic cost values in different assignors."; + + "optimization algorithm favors minimizing cross rack traffic or minimize the movement of tasks in existing assignment. If set a larger value " + RackAwareTaskAssignor.class.getName() + " will " + + "optimize for minimizing cross rack traffic. The default value is null which means it will use default traffic cost values in different assignors."; @SuppressWarnings("WeakerAccess") public static final String RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG = "rack.aware.assignment.non_overlap_cost"; public static final String RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_DOC = "Cost associated with moving tasks from existing assignment. This config and " + RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG + " controls whether the " - + "optimization algorithm favors minimizing cross rack traffic or minimize the movement of tasks in existing assignment. If set a larger value " + RackAwareTaskAssignor.class.getName() + " will " - + "optimize to maintain the existing assignment. The default value is null which means it will use default non_overlap cost values in different assignors."; + + "optimization algorithm favors minimizing cross rack traffic or minimize the movement of tasks in existing assignment. If set a larger value " + RackAwareTaskAssignor.class.getName() + " will " + + "optimize to maintain the existing assignment. The default value is null which means it will use default non_overlap cost values in different assignors."; /** * {@code topology.optimization} - * * @deprecated since 2.7; use {@link #TOPOLOGY_OPTIMIZATION_CONFIG} instead */ @Deprecated @@ -953,371 +838,370 @@ public class StreamsConfig extends AbstractConfig { private static final String[] NON_CONFIGURABLE_CONSUMER_DEFAULT_CONFIGS = - new String[] {ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG}; + new String[] {ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG}; private static final String[] NON_CONFIGURABLE_CONSUMER_EOS_CONFIGS = - new String[] {ConsumerConfig.ISOLATION_LEVEL_CONFIG}; + new String[] {ConsumerConfig.ISOLATION_LEVEL_CONFIG}; private static final String[] NON_CONFIGURABLE_PRODUCER_EOS_CONFIGS = - new String[] { - ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, - ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, - ProducerConfig.TRANSACTIONAL_ID_CONFIG - }; + new String[] { + ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, + ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, + ProducerConfig.TRANSACTIONAL_ID_CONFIG + }; static { CONFIG = new ConfigDef() - // HIGH - - .define(APPLICATION_ID_CONFIG, // required with no default value - Type.STRING, - Importance.HIGH, - APPLICATION_ID_DOC) - .define(BOOTSTRAP_SERVERS_CONFIG, // required with no default value - Type.LIST, - Importance.HIGH, - CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) - .define(NUM_STANDBY_REPLICAS_CONFIG, - Type.INT, - 0, - Importance.HIGH, - NUM_STANDBY_REPLICAS_DOC) - .define(STATE_DIR_CONFIG, - Type.STRING, - System.getProperty("java.io.tmpdir") + File.separator + "kafka-streams", - Importance.HIGH, - STATE_DIR_DOC, - "${java.io.tmpdir}") - - // MEDIUM - - .define(ACCEPTABLE_RECOVERY_LAG_CONFIG, - Type.LONG, - 10_000L, - atLeast(0), - Importance.MEDIUM, - ACCEPTABLE_RECOVERY_LAG_DOC) - .define(CACHE_MAX_BYTES_BUFFERING_CONFIG, - Type.LONG, - 10 * 1024 * 1024L, - atLeast(0), - Importance.MEDIUM, - CACHE_MAX_BYTES_BUFFERING_DOC) - .define(STATESTORE_CACHE_MAX_BYTES_CONFIG, - Type.LONG, - 10 * 1024 * 1024L, - atLeast(0), - Importance.MEDIUM, - STATESTORE_CACHE_MAX_BYTES_DOC) - .define(CLIENT_ID_CONFIG, - Type.STRING, - "", - Importance.MEDIUM, - CLIENT_ID_DOC, - "<application.id>-<random-UUID>") - .define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, - Type.CLASS, - LogAndFailExceptionHandler.class.getName(), - Importance.MEDIUM, - DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC) - .define(DEFAULT_KEY_SERDE_CLASS_CONFIG, - Type.CLASS, - null, - Importance.MEDIUM, - DEFAULT_KEY_SERDE_CLASS_DOC) - .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC) - .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC) - .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC) - .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC) - .define(DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, - Type.CLASS, - DefaultProductionExceptionHandler.class.getName(), - Importance.MEDIUM, - DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC) - .define(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, - Type.CLASS, - ProcessingLogAndFailExceptionHandler.class.getName(), - Importance.MEDIUM, - PROCESSING_EXCEPTION_HANDLER_CLASS_DOC) - .define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, - Type.CLASS, - FailOnInvalidTimestamp.class.getName(), - Importance.MEDIUM, - DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC) - .define(DEFAULT_VALUE_SERDE_CLASS_CONFIG, - Type.CLASS, - null, - Importance.MEDIUM, - DEFAULT_VALUE_SERDE_CLASS_DOC) - .define(MAX_TASK_IDLE_MS_CONFIG, - Type.LONG, - 0L, - Importance.MEDIUM, - MAX_TASK_IDLE_MS_DOC) - .define(MAX_WARMUP_REPLICAS_CONFIG, - Type.INT, - 2, - atLeast(1), - Importance.MEDIUM, - MAX_WARMUP_REPLICAS_DOC) - .define(NUM_STREAM_THREADS_CONFIG, - Type.INT, - 1, - Importance.MEDIUM, - NUM_STREAM_THREADS_DOC) - .define(PROCESSING_GUARANTEE_CONFIG, - Type.STRING, - AT_LEAST_ONCE, - in(AT_LEAST_ONCE, EXACTLY_ONCE, EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2), - Importance.MEDIUM, - PROCESSING_GUARANTEE_DOC) - .define(RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG, - Type.INT, - null, - Importance.MEDIUM, - RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_DOC) - .define(RACK_AWARE_ASSIGNMENT_STRATEGY_CONFIG, - Type.STRING, - RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, - in(RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC, RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY), - Importance.MEDIUM, - RACK_AWARE_ASSIGNMENT_STRATEGY_DOC) - .define(RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, - Type.LIST, - Collections.emptyList(), - atMostOfSize(MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE), - Importance.MEDIUM, - RACK_AWARE_ASSIGNMENT_TAGS_DOC) - .define(RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG, - Type.INT, - null, - Importance.MEDIUM, - RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_DOC) - .define(REPLICATION_FACTOR_CONFIG, - Type.INT, - -1, - Importance.MEDIUM, - REPLICATION_FACTOR_DOC) - .define(SECURITY_PROTOCOL_CONFIG, - Type.STRING, - CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL, - ConfigDef.CaseInsensitiveValidString.in(Utils.enumOptions(SecurityProtocol.class)), - Importance.MEDIUM, - CommonClientConfigs.SECURITY_PROTOCOL_DOC) - .define(TASK_TIMEOUT_MS_CONFIG, - Type.LONG, - Duration.ofMinutes(5L).toMillis(), - atLeast(0L), - Importance.MEDIUM, - TASK_TIMEOUT_MS_DOC) - .define(TOPOLOGY_OPTIMIZATION_CONFIG, - Type.STRING, - NO_OPTIMIZATION, - (name, value) -> verifyTopologyOptimizationConfigs((String) value), - Importance.MEDIUM, - TOPOLOGY_OPTIMIZATION_DOC) - - // LOW - - .define(APPLICATION_SERVER_CONFIG, - Type.STRING, - "", - Importance.LOW, - APPLICATION_SERVER_DOC) - .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, - Type.INT, - 1000, - Importance.LOW, - BUFFERED_RECORDS_PER_PARTITION_DOC) - .define(BUILT_IN_METRICS_VERSION_CONFIG, - Type.STRING, - METRICS_LATEST, - in( - METRICS_LATEST - ), - Importance.LOW, - BUILT_IN_METRICS_VERSION_DOC) - .define(COMMIT_INTERVAL_MS_CONFIG, - Type.LONG, - DEFAULT_COMMIT_INTERVAL_MS, - atLeast(0), - Importance.LOW, - COMMIT_INTERVAL_MS_DOC) - .define(ENABLE_METRICS_PUSH_CONFIG, - Type.BOOLEAN, - true, - Importance.LOW, - ENABLE_METRICS_PUSH_DOC) - .define(REPARTITION_PURGE_INTERVAL_MS_CONFIG, - Type.LONG, - DEFAULT_COMMIT_INTERVAL_MS, - atLeast(0), - Importance.LOW, - REPARTITION_PURGE_INTERVAL_MS_DOC) - .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, - Type.LONG, - 9 * 60 * 1000L, - Importance.LOW, - CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC) - .define(DEFAULT_DSL_STORE_CONFIG, - Type.STRING, - DEFAULT_DSL_STORE, - in(ROCKS_DB, IN_MEMORY), - Importance.LOW, - DEFAULT_DSL_STORE_DOC) - .define(DSL_STORE_SUPPLIERS_CLASS_CONFIG, - Type.CLASS, - DSL_STORE_SUPPLIERS_CLASS_DEFAULT, - Importance.LOW, - DSL_STORE_SUPPLIERS_CLASS_DOC) - .define(DEFAULT_CLIENT_SUPPLIER_CONFIG, - Type.CLASS, - DefaultKafkaClientSupplier.class.getName(), - Importance.LOW, - DEFAULT_CLIENT_SUPPLIER_DOC) - .define(METADATA_MAX_AGE_CONFIG, - Type.LONG, - 5 * 60 * 1000L, - atLeast(0), - Importance.LOW, - CommonClientConfigs.METADATA_MAX_AGE_DOC) - .define(METRICS_NUM_SAMPLES_CONFIG, - Type.INT, - 2, - atLeast(1), - Importance.LOW, - CommonClientConfigs.METRICS_NUM_SAMPLES_DOC) - .define(METRIC_REPORTER_CLASSES_CONFIG, - Type.LIST, - "", - Importance.LOW, - CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) - .define(METRICS_RECORDING_LEVEL_CONFIG, - Type.STRING, - Sensor.RecordingLevel.INFO.toString(), - in(Sensor.RecordingLevel.INFO.toString(), Sensor.RecordingLevel.DEBUG.toString(), RecordingLevel.TRACE.toString()), - Importance.LOW, - CommonClientConfigs.METRICS_RECORDING_LEVEL_DOC) - .define(METRICS_SAMPLE_WINDOW_MS_CONFIG, - Type.LONG, - 30000L, - atLeast(0), - Importance.LOW, - CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC) - .define(AUTO_INCLUDE_JMX_REPORTER_CONFIG, - Type.BOOLEAN, - true, - Importance.LOW, - CommonClientConfigs.AUTO_INCLUDE_JMX_REPORTER_DOC) - .define(POLL_MS_CONFIG, - Type.LONG, - 100L, - Importance.LOW, - POLL_MS_DOC) - .define(PROBING_REBALANCE_INTERVAL_MS_CONFIG, - Type.LONG, - 10 * 60 * 1000L, - atLeast(60 * 1000L), - Importance.LOW, - PROBING_REBALANCE_INTERVAL_MS_DOC) - .define(RECEIVE_BUFFER_CONFIG, - Type.INT, - 32 * 1024, - atLeast(CommonClientConfigs.RECEIVE_BUFFER_LOWER_BOUND), - Importance.LOW, - CommonClientConfigs.RECEIVE_BUFFER_DOC) - .define(RECONNECT_BACKOFF_MS_CONFIG, - Type.LONG, - 50L, - atLeast(0L), - Importance.LOW, - CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC) - .define(RECONNECT_BACKOFF_MAX_MS_CONFIG, - Type.LONG, - 1000L, - atLeast(0L), - Importance.LOW, - CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_DOC) - .define(RETRIES_CONFIG, - Type.INT, - 0, - between(0, Integer.MAX_VALUE), - Importance.LOW, - CommonClientConfigs.RETRIES_DOC) - .define(RETRY_BACKOFF_MS_CONFIG, - Type.LONG, - 100L, - atLeast(0L), - Importance.LOW, - CommonClientConfigs.RETRY_BACKOFF_MS_DOC) - .define(REQUEST_TIMEOUT_MS_CONFIG, - Type.INT, - 40 * 1000, - atLeast(0), - Importance.LOW, - CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC) - .define(ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, - Type.CLASS, - null, - Importance.LOW, - ROCKSDB_CONFIG_SETTER_CLASS_DOC) - .define(SEND_BUFFER_CONFIG, - Type.INT, - 128 * 1024, - atLeast(CommonClientConfigs.SEND_BUFFER_LOWER_BOUND), - Importance.LOW, - CommonClientConfigs.SEND_BUFFER_DOC) - .define(STATE_CLEANUP_DELAY_MS_CONFIG, - Type.LONG, - 10 * 60 * 1000L, - Importance.LOW, - STATE_CLEANUP_DELAY_MS_DOC) - .define(UPGRADE_FROM_CONFIG, - Type.STRING, - null, - in(Stream.concat( - Stream.of((String) null), - Arrays.stream(UpgradeFromValues.values()).map(UpgradeFromValues::toString) - ).toArray(String[]::new) - ), - Importance.LOW, - UPGRADE_FROM_DOC) - .define(WINDOWED_INNER_CLASS_SERDE, - Type.STRING, - null, - Importance.LOW, - WINDOWED_INNER_CLASS_SERDE_DOC) - .define(WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, - Type.LONG, - 24 * 60 * 60 * 1000L, - Importance.LOW, - WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC) - .define(WINDOW_SIZE_MS_CONFIG, - Type.LONG, - null, - Importance.LOW, - WINDOW_SIZE_MS_DOC); + // HIGH + + .define(APPLICATION_ID_CONFIG, // required with no default value + Type.STRING, + Importance.HIGH, + APPLICATION_ID_DOC) + .define(BOOTSTRAP_SERVERS_CONFIG, // required with no default value + Type.LIST, + Importance.HIGH, + CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) + .define(NUM_STANDBY_REPLICAS_CONFIG, + Type.INT, + 0, + Importance.HIGH, + NUM_STANDBY_REPLICAS_DOC) + .define(STATE_DIR_CONFIG, + Type.STRING, + System.getProperty("java.io.tmpdir") + File.separator + "kafka-streams", + Importance.HIGH, + STATE_DIR_DOC, + "${java.io.tmpdir}") + + // MEDIUM + + .define(ACCEPTABLE_RECOVERY_LAG_CONFIG, + Type.LONG, + 10_000L, + atLeast(0), + Importance.MEDIUM, + ACCEPTABLE_RECOVERY_LAG_DOC) + .define(CACHE_MAX_BYTES_BUFFERING_CONFIG, + Type.LONG, + 10 * 1024 * 1024L, + atLeast(0), + Importance.MEDIUM, + CACHE_MAX_BYTES_BUFFERING_DOC) + .define(STATESTORE_CACHE_MAX_BYTES_CONFIG, + Type.LONG, + 10 * 1024 * 1024L, + atLeast(0), + Importance.MEDIUM, + STATESTORE_CACHE_MAX_BYTES_DOC) + .define(CLIENT_ID_CONFIG, + Type.STRING, + "", + Importance.MEDIUM, + CLIENT_ID_DOC, + "<application.id>-<random-UUID>") + .define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, + Type.CLASS, + LogAndFailExceptionHandler.class.getName(), + Importance.MEDIUM, + DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC) + .define(DEFAULT_KEY_SERDE_CLASS_CONFIG, + Type.CLASS, + null, + Importance.MEDIUM, + DEFAULT_KEY_SERDE_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC) + .define(DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, + Type.CLASS, + DefaultProductionExceptionHandler.class.getName(), + Importance.MEDIUM, + DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC) + .define(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, + Type.CLASS, + ProcessingLogAndFailExceptionHandler.class.getName(), + Importance.MEDIUM, + PROCESSING_EXCEPTION_HANDLER_CLASS_DOC) + .define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, + Type.CLASS, + FailOnInvalidTimestamp.class.getName(), + Importance.MEDIUM, + DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC) + .define(DEFAULT_VALUE_SERDE_CLASS_CONFIG, + Type.CLASS, + null, + Importance.MEDIUM, + DEFAULT_VALUE_SERDE_CLASS_DOC) + .define(MAX_TASK_IDLE_MS_CONFIG, + Type.LONG, + 0L, + Importance.MEDIUM, + MAX_TASK_IDLE_MS_DOC) + .define(MAX_WARMUP_REPLICAS_CONFIG, + Type.INT, + 2, + atLeast(1), + Importance.MEDIUM, + MAX_WARMUP_REPLICAS_DOC) + .define(NUM_STREAM_THREADS_CONFIG, + Type.INT, + 1, + Importance.MEDIUM, + NUM_STREAM_THREADS_DOC) + .define(PROCESSING_GUARANTEE_CONFIG, + Type.STRING, + AT_LEAST_ONCE, + in(AT_LEAST_ONCE, EXACTLY_ONCE, EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2), + Importance.MEDIUM, + PROCESSING_GUARANTEE_DOC) + .define(RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG, + Type.INT, + null, + Importance.MEDIUM, + RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_DOC) + .define(RACK_AWARE_ASSIGNMENT_STRATEGY_CONFIG, + Type.STRING, + RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, + in(RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC, RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY), + Importance.MEDIUM, + RACK_AWARE_ASSIGNMENT_STRATEGY_DOC) + .define(RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, + Type.LIST, + Collections.emptyList(), + atMostOfSize(MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE), + Importance.MEDIUM, + RACK_AWARE_ASSIGNMENT_TAGS_DOC) + .define(RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG, + Type.INT, + null, + Importance.MEDIUM, + RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_DOC) + .define(REPLICATION_FACTOR_CONFIG, + Type.INT, + -1, + Importance.MEDIUM, + REPLICATION_FACTOR_DOC) + .define(SECURITY_PROTOCOL_CONFIG, + Type.STRING, + CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL, + ConfigDef.CaseInsensitiveValidString.in(Utils.enumOptions(SecurityProtocol.class)), + Importance.MEDIUM, + CommonClientConfigs.SECURITY_PROTOCOL_DOC) + .define(TASK_TIMEOUT_MS_CONFIG, + Type.LONG, + Duration.ofMinutes(5L).toMillis(), + atLeast(0L), + Importance.MEDIUM, + TASK_TIMEOUT_MS_DOC) + .define(TOPOLOGY_OPTIMIZATION_CONFIG, + Type.STRING, + NO_OPTIMIZATION, + (name, value) -> verifyTopologyOptimizationConfigs((String) value), + Importance.MEDIUM, + TOPOLOGY_OPTIMIZATION_DOC) + + // LOW + + .define(APPLICATION_SERVER_CONFIG, + Type.STRING, + "", + Importance.LOW, + APPLICATION_SERVER_DOC) + .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, + Type.INT, + 1000, + Importance.LOW, + BUFFERED_RECORDS_PER_PARTITION_DOC) + .define(BUILT_IN_METRICS_VERSION_CONFIG, + Type.STRING, + METRICS_LATEST, + in( + METRICS_LATEST + ), + Importance.LOW, + BUILT_IN_METRICS_VERSION_DOC) + .define(COMMIT_INTERVAL_MS_CONFIG, + Type.LONG, + DEFAULT_COMMIT_INTERVAL_MS, + atLeast(0), + Importance.LOW, + COMMIT_INTERVAL_MS_DOC) + .define(ENABLE_METRICS_PUSH_CONFIG, + Type.BOOLEAN, + true, + Importance.LOW, + ENABLE_METRICS_PUSH_DOC) + .define(REPARTITION_PURGE_INTERVAL_MS_CONFIG, + Type.LONG, + DEFAULT_COMMIT_INTERVAL_MS, + atLeast(0), + Importance.LOW, + REPARTITION_PURGE_INTERVAL_MS_DOC) + .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, + Type.LONG, + 9 * 60 * 1000L, + Importance.LOW, + CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC) + .define(DEFAULT_DSL_STORE_CONFIG, + Type.STRING, + DEFAULT_DSL_STORE, + in(ROCKS_DB, IN_MEMORY), + Importance.LOW, + DEFAULT_DSL_STORE_DOC) + .define(DSL_STORE_SUPPLIERS_CLASS_CONFIG, + Type.CLASS, + DSL_STORE_SUPPLIERS_CLASS_DEFAULT, + Importance.LOW, + DSL_STORE_SUPPLIERS_CLASS_DOC) + .define(DEFAULT_CLIENT_SUPPLIER_CONFIG, + Type.CLASS, + DefaultKafkaClientSupplier.class.getName(), + Importance.LOW, + DEFAULT_CLIENT_SUPPLIER_DOC) + .define(METADATA_MAX_AGE_CONFIG, + Type.LONG, + 5 * 60 * 1000L, + atLeast(0), + Importance.LOW, + CommonClientConfigs.METADATA_MAX_AGE_DOC) + .define(METRICS_NUM_SAMPLES_CONFIG, + Type.INT, + 2, + atLeast(1), + Importance.LOW, + CommonClientConfigs.METRICS_NUM_SAMPLES_DOC) + .define(METRIC_REPORTER_CLASSES_CONFIG, + Type.LIST, + "", + Importance.LOW, + CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) + .define(METRICS_RECORDING_LEVEL_CONFIG, + Type.STRING, + Sensor.RecordingLevel.INFO.toString(), + in(Sensor.RecordingLevel.INFO.toString(), Sensor.RecordingLevel.DEBUG.toString(), RecordingLevel.TRACE.toString()), + Importance.LOW, + CommonClientConfigs.METRICS_RECORDING_LEVEL_DOC) + .define(METRICS_SAMPLE_WINDOW_MS_CONFIG, + Type.LONG, + 30000L, + atLeast(0), + Importance.LOW, + CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC) + .define(AUTO_INCLUDE_JMX_REPORTER_CONFIG, + Type.BOOLEAN, + true, + Importance.LOW, + CommonClientConfigs.AUTO_INCLUDE_JMX_REPORTER_DOC) + .define(POLL_MS_CONFIG, + Type.LONG, + 100L, + Importance.LOW, + POLL_MS_DOC) + .define(PROBING_REBALANCE_INTERVAL_MS_CONFIG, + Type.LONG, + 10 * 60 * 1000L, + atLeast(60 * 1000L), + Importance.LOW, + PROBING_REBALANCE_INTERVAL_MS_DOC) + .define(RECEIVE_BUFFER_CONFIG, + Type.INT, + 32 * 1024, + atLeast(CommonClientConfigs.RECEIVE_BUFFER_LOWER_BOUND), + Importance.LOW, + CommonClientConfigs.RECEIVE_BUFFER_DOC) + .define(RECONNECT_BACKOFF_MS_CONFIG, + Type.LONG, + 50L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC) + .define(RECONNECT_BACKOFF_MAX_MS_CONFIG, + Type.LONG, + 1000L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_DOC) + .define(RETRIES_CONFIG, + Type.INT, + 0, + between(0, Integer.MAX_VALUE), + Importance.LOW, + CommonClientConfigs.RETRIES_DOC) + .define(RETRY_BACKOFF_MS_CONFIG, + Type.LONG, + 100L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RETRY_BACKOFF_MS_DOC) + .define(REQUEST_TIMEOUT_MS_CONFIG, + Type.INT, + 40 * 1000, + atLeast(0), + Importance.LOW, + CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC) + .define(ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, + Type.CLASS, + null, + Importance.LOW, + ROCKSDB_CONFIG_SETTER_CLASS_DOC) + .define(SEND_BUFFER_CONFIG, + Type.INT, + 128 * 1024, + atLeast(CommonClientConfigs.SEND_BUFFER_LOWER_BOUND), + Importance.LOW, + CommonClientConfigs.SEND_BUFFER_DOC) + .define(STATE_CLEANUP_DELAY_MS_CONFIG, + Type.LONG, + 10 * 60 * 1000L, + Importance.LOW, + STATE_CLEANUP_DELAY_MS_DOC) + .define(UPGRADE_FROM_CONFIG, + Type.STRING, + null, + in(Stream.concat( + Stream.of((String) null), + Arrays.stream(UpgradeFromValues.values()).map(UpgradeFromValues::toString) + ).toArray(String[]::new) + ), + Importance.LOW, + UPGRADE_FROM_DOC) + .define(WINDOWED_INNER_CLASS_SERDE, + Type.STRING, + null, + Importance.LOW, + WINDOWED_INNER_CLASS_SERDE_DOC) + .define(WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, + Type.LONG, + 24 * 60 * 60 * 1000L, + Importance.LOW, + WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC) + .define(WINDOW_SIZE_MS_CONFIG, + Type.LONG, + null, + Importance.LOW, + WINDOW_SIZE_MS_DOC); } // this is the list of configs for underlying clients // that streams prefer different default values private static final Map PRODUCER_DEFAULT_OVERRIDES; - static { final Map tempProducerDefaultOverrides = new HashMap<>(); tempProducerDefaultOverrides.put(ProducerConfig.LINGER_MS_CONFIG, "100"); @@ -1325,7 +1209,6 @@ public class StreamsConfig extends AbstractConfig { } private static final Map PRODUCER_EOS_OVERRIDES; - static { final Map tempProducerDefaultOverrides = new HashMap<>(PRODUCER_DEFAULT_OVERRIDES); tempProducerDefaultOverrides.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, Integer.MAX_VALUE); @@ -1337,7 +1220,6 @@ public class StreamsConfig extends AbstractConfig { } private static final Map CONSUMER_DEFAULT_OVERRIDES; - static { final Map tempConsumerDefaultOverrides = new HashMap<>(); tempConsumerDefaultOverrides.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "1000"); @@ -1348,7 +1230,6 @@ public class StreamsConfig extends AbstractConfig { } private static final Map CONSUMER_EOS_OVERRIDES; - static { final Map tempConsumerDefaultOverrides = new HashMap<>(CONSUMER_DEFAULT_OVERRIDES); tempConsumerDefaultOverrides.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, READ_COMMITTED.toString()); @@ -1374,7 +1255,7 @@ public static class InternalConfig { // Private API used to control the usage of consistency offset vectors public static final String IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED = "__iq.consistency.offset" - + ".vector.enabled__"; + + ".vector.enabled__"; // Private API used to control the prefix of the auto created topics public static final String TOPIC_PREFIX_ALTERNATIVE = "__internal.override.topic.prefix__"; @@ -1550,12 +1431,12 @@ protected StreamsConfig(final Map props, final String processingModeConfig = getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); if (processingModeConfig.equals(EXACTLY_ONCE)) { log.warn("Configuration parameter `{}` is deprecated and will be removed in the 4.0.0 release. " + - "Please use `{}` instead. Note that this requires broker version 2.5+ so you should prepare " - + "to upgrade your brokers if necessary.", EXACTLY_ONCE, EXACTLY_ONCE_V2); + "Please use `{}` instead. Note that this requires broker version 2.5+ so you should prepare " + + "to upgrade your brokers if necessary.", EXACTLY_ONCE, EXACTLY_ONCE_V2); } if (processingModeConfig.equals(EXACTLY_ONCE_BETA)) { log.warn("Configuration parameter `{}` is deprecated and will be removed in the 4.0.0 release. " + - "Please use `{}` instead.", EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2); + "Please use `{}` instead.", EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2); } if (props.containsKey(RETRIES_CONFIG)) { @@ -1572,30 +1453,30 @@ private void verifyEOSTransactionTimeoutCompatibility() { final long commitInterval = getLong(COMMIT_INTERVAL_MS_CONFIG); final String transactionTimeoutConfigKey = producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG); final int transactionTimeout = - originals().containsKey(transactionTimeoutConfigKey) ? - (int) Objects.requireNonNull( - parseType(transactionTimeoutConfigKey, originals().get(transactionTimeoutConfigKey), Type.INT), - "Could not parse config `" + COMMIT_INTERVAL_MS_CONFIG + "` because it's set to `null`") : - DEFAULT_TRANSACTION_TIMEOUT; + originals().containsKey(transactionTimeoutConfigKey) ? + (int) Objects.requireNonNull( + parseType(transactionTimeoutConfigKey, originals().get(transactionTimeoutConfigKey), Type.INT), + "Could not parse config `" + COMMIT_INTERVAL_MS_CONFIG + "` because it's set to `null`") : + DEFAULT_TRANSACTION_TIMEOUT; if (transactionTimeout < commitInterval) { throw new IllegalArgumentException(String.format("Transaction timeout %d was set lower than " + - "streams commit interval %d. This will cause ongoing transaction always timeout due to inactivity " + - "caused by long commit interval. Consider reconfiguring commit interval to match " + - "transaction timeout by tuning 'commit.interval.ms' config, or increase the transaction timeout to match " + - "commit interval by tuning `producer.transaction.timeout.ms` config.", - transactionTimeout, commitInterval)); + "streams commit interval %d. This will cause ongoing transaction always timeout due to inactivity " + + "caused by long commit interval. Consider reconfiguring commit interval to match " + + "transaction timeout by tuning 'commit.interval.ms' config, or increase the transaction timeout to match " + + "commit interval by tuning `producer.transaction.timeout.ms` config.", + transactionTimeout, commitInterval)); } } @Override protected Map postProcessParsedConfig(final Map parsedValues) { final Map configUpdates = - CommonClientConfigs.postProcessReconnectBackoffConfigs(this, parsedValues); + CommonClientConfigs.postProcessReconnectBackoffConfigs(this, parsedValues); if (StreamsConfigUtils.eosEnabled(this) && !originals().containsKey(COMMIT_INTERVAL_MS_CONFIG)) { log.debug("Using {} default value of {} as exactly once is enabled.", - COMMIT_INTERVAL_MS_CONFIG, EOS_DEFAULT_COMMIT_INTERVAL_MS); + COMMIT_INTERVAL_MS_CONFIG, EOS_DEFAULT_COMMIT_INTERVAL_MS); configUpdates.put(COMMIT_INTERVAL_MS_CONFIG, EOS_DEFAULT_COMMIT_INTERVAL_MS); } @@ -1610,28 +1491,28 @@ private void validateRackAwarenessConfiguration() { if (clientTags.size() > MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE) { throw new ConfigException("At most " + MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE + " client tags " + - "can be specified using " + CLIENT_TAG_PREFIX + " prefix."); + "can be specified using " + CLIENT_TAG_PREFIX + " prefix."); } for (final String rackAwareAssignmentTag : rackAwareAssignmentTags) { if (!clientTags.containsKey(rackAwareAssignmentTag)) { throw new ConfigException(RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, - rackAwareAssignmentTags, - "Contains invalid value [" + rackAwareAssignmentTag + "] " + - "which doesn't have corresponding tag set via [" + CLIENT_TAG_PREFIX + "] prefix."); + rackAwareAssignmentTags, + "Contains invalid value [" + rackAwareAssignmentTag + "] " + + "which doesn't have corresponding tag set via [" + CLIENT_TAG_PREFIX + "] prefix."); } } clientTags.forEach((tagKey, tagValue) -> { if (tagKey.length() > MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH) { throw new ConfigException(CLIENT_TAG_PREFIX, - tagKey, - "Tag key exceeds maximum length of " + MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH + "."); + tagKey, + "Tag key exceeds maximum length of " + MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH + "."); } if (tagValue.length() > MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH) { throw new ConfigException(CLIENT_TAG_PREFIX, - tagValue, - "Tag value exceeds maximum length of " + MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH + "."); + tagValue, + "Tag value exceeds maximum length of " + MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH + "."); } }); } @@ -1666,30 +1547,30 @@ private void checkIfUnexpectedUserSpecifiedConsumerConfig(final Map-")); + "producer", config, eosMessage, clientProvidedProps.get(config), "-")); clientProvidedProps.remove(config); } } @@ -1726,15 +1607,15 @@ private void verifyMaxInFlightRequestPerConnection(final Object maxInFlightReque * Get the configs to the {@link KafkaConsumer main consumer}. * Properties using the prefix {@link #MAIN_CONSUMER_PREFIX} will be used in favor over * the properties prefixed with {@link #CONSUMER_PREFIX} and the non-prefixed versions - * (read the override precedence ordering in {@link #MAIN_CONSUMER_PREFIX} + * (read the override precedence ordering in {@link #MAIN_CONSUMER_PREFIX}) * except in the case of {@link ConsumerConfig#BOOTSTRAP_SERVERS_CONFIG} where we always use the non-prefixed * version as we only support reading/writing from/to the same Kafka Cluster. * If not specified by {@link #MAIN_CONSUMER_PREFIX}, main consumer will share the general consumer configs * prefixed by {@link #CONSUMER_PREFIX}. * - * @param groupId consumer groupId - * @param clientId clientId - * @param threadIdx stream thread index + * @param groupId consumer groupId + * @param clientId clientId + * @param threadIdx stream thread index * @return Map of the consumer configuration. */ @SuppressWarnings("WeakerAccess") @@ -1780,14 +1661,14 @@ public Map getMainConsumerConfigs(final String groupId, final St final Map producerProps = getClientPropsWithPrefix(PRODUCER_PREFIX, ProducerConfig.configNames()); if (topicProps.containsKey(topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG)) && - producerProps.containsKey(ProducerConfig.BATCH_SIZE_CONFIG)) { + producerProps.containsKey(ProducerConfig.BATCH_SIZE_CONFIG)) { final int segmentSize = Integer.parseInt(topicProps.get(topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG)).toString()); final int batchSize = Integer.parseInt(producerProps.get(ProducerConfig.BATCH_SIZE_CONFIG).toString()); if (segmentSize < batchSize) { throw new IllegalArgumentException(String.format("Specified topic segment size %d is is smaller than the configured producer batch size %d, this will cause produced batch not able to be appended to the topic", - segmentSize, - batchSize)); + segmentSize, + batchSize)); } } @@ -1800,7 +1681,7 @@ public Map getMainConsumerConfigs(final String groupId, final St * Get the configs for the {@link KafkaConsumer restore-consumer}. * Properties using the prefix {@link #RESTORE_CONSUMER_PREFIX} will be used in favor over * the properties prefixed with {@link #CONSUMER_PREFIX} and the non-prefixed versions - * (read the override precedence ordering in {@link #RESTORE_CONSUMER_PREFIX} + * (read the override precedence ordering in {@link #RESTORE_CONSUMER_PREFIX}) * except in the case of {@link ConsumerConfig#BOOTSTRAP_SERVERS_CONFIG} where we always use the non-prefixed * version as we only support reading/writing from/to the same Kafka Cluster. * If not specified by {@link #RESTORE_CONSUMER_PREFIX}, restore consumer will share the general consumer configs @@ -1833,7 +1714,7 @@ public Map getRestoreConsumerConfigs(final String clientId) { * Get the configs for the {@link KafkaConsumer global consumer}. * Properties using the prefix {@link #GLOBAL_CONSUMER_PREFIX} will be used in favor over * the properties prefixed with {@link #CONSUMER_PREFIX} and the non-prefixed versions - * (read the override precedence ordering in {@link #GLOBAL_CONSUMER_PREFIX} + * (read the override precedence ordering in {@link #GLOBAL_CONSUMER_PREFIX}) * except in the case of {@link ConsumerConfig#BOOTSTRAP_SERVERS_CONFIG} where we always use the non-prefixed * version as we only support reading/writing from/to the same Kafka Cluster. * If not specified by {@link #GLOBAL_CONSUMER_PREFIX}, global consumer will share the general consumer configs @@ -1896,7 +1777,6 @@ public Map getProducerConfigs(final String clientId) { /** * Get the configs for the {@link Admin admin client}. - * * @param clientId clientId * @return Map of the admin client configuration. */ @@ -1922,10 +1802,10 @@ public Map getAdminConfigs(final String clientId) { @SuppressWarnings("WeakerAccess") public Map getClientTags() { return originalsWithPrefix(CLIENT_TAG_PREFIX).entrySet().stream().collect( - Collectors.toMap( - Map.Entry::getKey, - tagEntry -> Objects.toString(tagEntry.getValue()) - ) + Collectors.toMap( + Map.Entry::getKey, + tagEntry -> Objects.toString(tagEntry.getValue()) + ) ); } @@ -1965,7 +1845,7 @@ public static Set verifyTopologyOptimizationConfigs(final String config) throw new ConfigException("\"" + config + "\" is not a valid optimization config. " + CONFIG_ERROR_MSG); } } - for (final String conf : configs) { + for (final String conf: configs) { if (!TOPOLOGY_OPTIMIZATION_CONFIGS.contains(conf)) { throw new ConfigException("Unrecognized config. " + CONFIG_ERROR_MSG); } @@ -1982,24 +1862,23 @@ public static Set verifyTopologyOptimizationConfigs(final String config) /** * Return configured KafkaClientSupplier - * * @return Configured KafkaClientSupplier */ public KafkaClientSupplier getKafkaClientSupplier() { return getConfiguredInstance(StreamsConfig.DEFAULT_CLIENT_SUPPLIER_CONFIG, - KafkaClientSupplier.class); + KafkaClientSupplier.class); } /** * Return an {@link Serde#configure(Map, boolean) configured} instance of {@link #DEFAULT_KEY_SERDE_CLASS_CONFIG key Serde * class}. * - * @return an configured instance of key Serde class + * @return a configured instance of key Serde class */ @SuppressWarnings("WeakerAccess") public Serde defaultKeySerde() { final Object keySerdeConfigSetting = get(DEFAULT_KEY_SERDE_CLASS_CONFIG); - if (keySerdeConfigSetting == null) { + if (keySerdeConfigSetting == null) { throw new ConfigException("Please specify a key serde or set one through StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG"); } try { @@ -2008,7 +1887,7 @@ public Serde defaultKeySerde() { return serde; } catch (final Exception e) { throw new StreamsException( - String.format("Failed to configure key serde %s", keySerdeConfigSetting), e); + String.format("Failed to configure key serde %s", keySerdeConfigSetting), e); } } @@ -2030,7 +1909,7 @@ public Serde defaultValueSerde() { return serde; } catch (final Exception e) { throw new StreamsException( - String.format("Failed to configure value serde %s", valueSerdeConfigSetting), e); + String.format("Failed to configure value serde %s", valueSerdeConfigSetting), e); } } @@ -2066,7 +1945,7 @@ private Map clientProps(final Set configNames, // iterate all client config names, filter out non-client configs from the original // property map and use the overridden values when they are not specified by users final Map parsed = new HashMap<>(); - for (final String configName : configNames) { + for (final String configName: configNames) { if (originals.containsKey(configName)) { parsed.put(configName, originals.get(configName)); } @@ -2078,4 +1957,4 @@ private Map clientProps(final Set configNames, public static void main(final String[] args) { System.out.println(CONFIG.toHtml(4, config -> "streamsconfigs_" + 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 1ae121e746458..6b2cc335c34e8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/DeserializationExceptionHandler.java @@ -38,7 +38,7 @@ public interface DeserializationExceptionHandler extends Configurable { * @param context processor context * @param record record that failed deserialization * @param exception the actual exception - * @deprecated Please {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception) + * @deprecated Please {@link #handle(ErrorHandlerContext, ConsumerRecord, Exception)} */ @Deprecated default DeserializationHandlerResponse handle(final ProcessorContext context, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java index 694892c21de4f..5a852d415a94c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ErrorHandlerContext.java @@ -100,7 +100,7 @@ public interface ErrorHandlerContext { * to the repartition topic. * *

Always returns null if this method is invoked within a - * {@link ProductionExceptionHandler.handle(ErrorHandlerContext, ProducerRecord, Exception, SerializationExceptionOrigin)} + * {@link org.apache.kafka.streams.errors.ProductionExceptionHandler#handle(ErrorHandlerContext, org.apache.kafka.clients.producer.ProducerRecord, Exception)} * * @return the raw byte of the key of the source message */ @@ -117,7 +117,7 @@ public interface ErrorHandlerContext { * to the repartition topic. * *

Always returns null if this method is invoked within a - * {@link ProductionExceptionHandler.handle(ErrorHandlerContext, ProducerRecord, Exception, SerializationExceptionOrigin)} + * {@link org.apache.kafka.streams.errors.ProductionExceptionHandler#handle(ErrorHandlerContext, org.apache.kafka.clients.producer.ProducerRecord, Exception)} * * @return the raw byte of the value of the source message */ diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java index 4a7646acb02d3..a7fae321ebff2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java @@ -27,7 +27,7 @@ public class CorruptedRecord extends StampedRecord { CorruptedRecord(final ConsumerRecord rawRecord) { - super(rawRecord, ConsumerRecord.NO_TIMESTAMP); + super(rawRecord, ConsumerRecord.NO_TIMESTAMP, rawRecord); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index 69e24d80ac81f..c0458a5dabe01 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -293,9 +293,10 @@ private void forwardInternal(final ProcessorNode child, try { child.process(record); - } catch (final Exception e) { + } catch (final Exception e) { final ErrorHandlerContext errorHandlerContext = new ErrorHandlerContextImpl(null, topic(), - partition(), offset(), headers(), null, null, child.name(), taskId()); + partition(), offset(), headers(), streamTask.rawRecord().key(), streamTask.rawRecord().value(), + child.name(), taskId()); final ProcessingExceptionHandler.ProcessingHandlerResponse response = streamTask.config .processingExceptionHandler .handle(errorHandlerContext, record, e); @@ -309,7 +310,6 @@ private void forwardInternal(final ProcessorNode child, } } - if (child.isTerminalNode()) { streamTask.maybeRecordE2ELatency(record.timestamp(), currentSystemTimeMs(), child.name()); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java index 297c287673892..412ca1804b3a8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -230,7 +230,7 @@ private void updateHead() { droppedRecordsSensor.record(); continue; } - headRecord = new StampedRecord(deserialized, timestamp); + headRecord = new StampedRecord(deserialized, timestamp, raw); headRecordSizeInBytes = consumerRecordSizeInBytes(raw); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java index 5d0c04b96a89f..81098d8a1b314 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java @@ -78,7 +78,6 @@ public void init(final InternalProcessorContext context) { valDeserializer = prepareValueDeserializer(valDeserializer, context, name()); } - @Override public void process(final Record record) { context.forward(record); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java index 71e3ca2e3ceca..ea4b236d28632 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java @@ -20,9 +20,11 @@ import org.apache.kafka.common.header.Headers; public class StampedRecord extends Stamped> { + private final ConsumerRecord rawRecord; - public StampedRecord(final ConsumerRecord record, final long timestamp) { + public StampedRecord(final ConsumerRecord record, final long timestamp, final ConsumerRecord rawRecord) { super(record, timestamp); + this.rawRecord = rawRecord; } public String topic() { @@ -49,6 +51,10 @@ public Headers headers() { return value.headers(); } + public ConsumerRecord rawRecord() { + return rawRecord; + } + @Override public String toString() { return value.toString() + ", timestamp = " + timestamp; 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 f6ff7f35ad7d5..12bf5a6401e3f 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 @@ -1314,6 +1314,10 @@ long streamTime() { return partitionGroup.streamTime(); } + ConsumerRecord rawRecord() { + return record.rawRecord(); + } + private class RecordQueueCreator { private final LogContext logContext; private final TimestampExtractor defaultTimestampExtractor; From 76cc4be88367e3d86c283609f35192bdbb3b9baa Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Fri, 17 May 2024 16:07:13 +0200 Subject: [PATCH 11/28] KAFKA-16448: add boolean to catch only one time an exception in the processor --- .../internals/ProcessorContextImpl.java | 32 +++++++++++-------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index c0458a5dabe01..dc43d0168f904 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -60,6 +60,8 @@ public class ProcessorContextImpl extends AbstractProcessorContext cacheNameToFlushListener = new HashMap<>(); @SuppressWarnings("this-escape") @@ -74,6 +76,7 @@ public ProcessorContextImpl(final TaskId id, appConfigs(), IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED, false); + } @Override @@ -294,19 +297,22 @@ private void forwardInternal(final ProcessorNode child, try { child.process(record); } catch (final Exception e) { - final ErrorHandlerContext errorHandlerContext = new ErrorHandlerContextImpl(null, topic(), - partition(), offset(), headers(), streamTask.rawRecord().key(), streamTask.rawRecord().value(), - child.name(), taskId()); - final ProcessingExceptionHandler.ProcessingHandlerResponse response = streamTask.config - .processingExceptionHandler - .handle(errorHandlerContext, record, e); - - if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { - throw new StreamsException("Processing exception handler is set to fail upon" + - " a processing error. If you would rather have the streaming pipeline" + - " continue after a processing error, please set the " + - PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.", - e); + if(!processingExceptionOccurred) { + processingExceptionOccurred = true; + final ErrorHandlerContext errorHandlerContext = new ErrorHandlerContextImpl(null, topic(), + partition(), offset(), headers(), streamTask.rawRecord().key(), streamTask.rawRecord().value(), + child.name(), taskId()); + final ProcessingExceptionHandler.ProcessingHandlerResponse response = streamTask.config + .processingExceptionHandler + .handle(errorHandlerContext, record, e); + + if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { + throw new StreamsException("Processing exception handler is set to fail upon" + + " a processing error. If you would rather have the streaming pipeline" + + " continue after a processing error, please set the " + + PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.", + e); + } } } From 6906dd7e1d9beae3fad9d495b309f8424167b71a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Fri, 17 May 2024 20:42:05 +0200 Subject: [PATCH 12/28] KAFKA-16448: Fix deprecation in unit tests --- .../processor/internals/ProcessorContextImpl.java | 2 +- .../AlwaysContinueProductionExceptionHandler.java | 10 +++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index dc43d0168f904..eb3eac8baffa1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -297,7 +297,7 @@ private void forwardInternal(final ProcessorNode child, try { child.process(record); } catch (final Exception e) { - if(!processingExceptionOccurred) { + if (!processingExceptionOccurred) { processingExceptionOccurred = true; final ErrorHandlerContext errorHandlerContext = new ErrorHandlerContextImpl(null, topic(), partition(), offset(), headers(), streamTask.rawRecord().key(), streamTask.rawRecord().value(), diff --git a/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java b/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java index c247043a3d8cd..32c685d7f0caf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java +++ b/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.errors; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.streams.processor.ErrorHandlerContext; import java.util.Map; @@ -27,15 +28,18 @@ public class AlwaysContinueProductionExceptionHandler implements ProductionExceptionHandler { @Override @Deprecated - public ProductionExceptionHandlerResponse handle(final ProducerRecord record, + public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, + final ProducerRecord record, final Exception exception) { return ProductionExceptionHandlerResponse.CONTINUE; } @Override @Deprecated - public ProductionExceptionHandlerResponse handleSerializationException(final ProducerRecord record, - final Exception exception) { + public ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context, + final ProducerRecord record, + final Exception exception, + final SerializationExceptionOrigin origin) { return ProductionExceptionHandlerResponse.CONTINUE; } From 259f01a118537d5e96b84a831ee4509c71751756 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Sat, 18 May 2024 14:09:17 +0200 Subject: [PATCH 13/28] KAFKA-16448: Move raw record out of StampedRecord --- .../internals/AbstractPartitionGroup.java | 7 ++++-- .../processor/internals/CorruptedRecord.java | 5 +++-- .../processor/internals/PartitionGroup.java | 12 ++++++++++ .../processor/internals/RecordQueue.java | 22 +++++++++++++++---- .../processor/internals/StampedRecord.java | 9 +------- .../processor/internals/StreamTask.java | 5 ++++- .../internals/SynchronizedPartitionGroup.java | 10 +++++++-- 7 files changed, 51 insertions(+), 19 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractPartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractPartitionGroup.java index 12af39ca4dbd3..808e4db9544c7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractPartitionGroup.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractPartitionGroup.java @@ -16,11 +16,12 @@ */ package org.apache.kafka.streams.processor.internals; -import java.util.Set; -import java.util.function.Function; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import java.util.Set; +import java.util.function.Function; + abstract class AbstractPartitionGroup { abstract boolean readyToProcess(long wallClockTime); @@ -55,6 +56,8 @@ abstract class AbstractPartitionGroup { abstract Long headRecordOffset(final TopicPartition partition); + abstract ConsumerRecord rawHeadRecord(); + abstract int numBuffered(); abstract int numBuffered(TopicPartition tp); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java index a7fae321ebff2..d31a29883cabe 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java @@ -16,9 +16,10 @@ */ package org.apache.kafka.streams.processor.internals; -import java.util.Objects; import org.apache.kafka.clients.consumer.ConsumerRecord; +import java.util.Objects; + /** * This class represents a version of a {@link StampedRecord} that failed to deserialize. We need * a special record type so that {@link StreamTask} can update consumed offsets. See KAFKA-6502 @@ -27,7 +28,7 @@ public class CorruptedRecord extends StampedRecord { CorruptedRecord(final ConsumerRecord rawRecord) { - super(rawRecord, ConsumerRecord.NO_TIMESTAMP, rawRecord); + super(rawRecord, ConsumerRecord.NO_TIMESTAMP); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java index 4852ba97932e9..37716e51544a9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java @@ -71,6 +71,7 @@ class PartitionGroup extends AbstractPartitionGroup { private boolean allBuffered; private final Map idlePartitionDeadlines = new HashMap<>(); private final Map fetchedLags = new HashMap<>(); + private ConsumerRecord rawHeadRecord; PartitionGroup(final LogContext logContext, final Map partitionQueues, @@ -249,6 +250,7 @@ StampedRecord nextRecord(final RecordInfo info, final long wallClockTime) { if (queue != null) { // get the first record from this queue. record = queue.poll(wallClockTime); + rawHeadRecord = queue.rawHeadRecord(); if (record != null) { --totalBuffered; @@ -321,6 +323,16 @@ Long headRecordOffset(final TopicPartition partition) { return recordQueue.headRecordOffset(); } + /** + * Returns the raw head record + * + * @return the raw head record + */ + @Override + ConsumerRecord rawHeadRecord() { + return rawHeadRecord; + } + /** * @throws IllegalStateException if the record's partition does not belong to this partition group */ diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java index 412ca1804b3a8..9ccdc1d89b98b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -22,11 +22,10 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; -import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.processor.internals.metrics.TopicMetrics; - import org.slf4j.Logger; import java.util.ArrayDeque; @@ -51,6 +50,8 @@ public class RecordQueue { private final ArrayDeque> fifoQueue; private StampedRecord headRecord = null; + private ConsumerRecord rawHeadRecord; + private long partitionTime = UNKNOWN; private final Sensor droppedRecordsSensor; @@ -114,6 +115,15 @@ public TopicPartition partition() { return partition; } + /** + * Returns the head record of the queue + * + * @return ConsumerRecord + */ + public ConsumerRecord rawHeadRecord() { + return rawHeadRecord; + } + /** * Add a batch of {@link ConsumerRecord} into the queue * @@ -142,6 +152,7 @@ public StampedRecord poll(final long wallClockTime) { headRecord = null; headRecordSizeInBytes = 0L; + rawHeadRecord = null; partitionTime = Math.max(partitionTime, recordToReturn.timestamp); updateHead(); @@ -188,6 +199,7 @@ public void clear() { fifoQueue.clear(); headRecord = null; headRecordSizeInBytes = 0L; + rawHeadRecord = null; partitionTime = UNKNOWN; } @@ -230,14 +242,16 @@ private void updateHead() { droppedRecordsSensor.record(); continue; } - headRecord = new StampedRecord(deserialized, timestamp, raw); + headRecord = new StampedRecord(deserialized, timestamp); headRecordSizeInBytes = consumerRecordSizeInBytes(raw); + rawHeadRecord = raw; } // if all records in the FIFO queue are corrupted, make the last one the headRecord // This record is used to update the offsets. See KAFKA-6502 for more details. if (headRecord == null && lastCorruptedRecord != null) { headRecord = new CorruptedRecord(lastCorruptedRecord); + rawHeadRecord = lastCorruptedRecord; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java index ea4b236d28632..d50fbee68260f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java @@ -20,11 +20,8 @@ import org.apache.kafka.common.header.Headers; public class StampedRecord extends Stamped> { - private final ConsumerRecord rawRecord; - - public StampedRecord(final ConsumerRecord record, final long timestamp, final ConsumerRecord rawRecord) { + public StampedRecord(final ConsumerRecord record, final long timestamp) { super(record, timestamp); - this.rawRecord = rawRecord; } public String topic() { @@ -51,10 +48,6 @@ public Headers headers() { return value.headers(); } - public ConsumerRecord rawRecord() { - return rawRecord; - } - @Override public String toString() { return value.toString() + ", timestamp = " + timestamp; 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 12bf5a6401e3f..6594051d822fc 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 @@ -107,6 +107,8 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator, protected final InternalProcessorContext processorContext; private StampedRecord record; + private ConsumerRecord rawRecord; + private boolean commitNeeded = false; private boolean commitRequested = false; private boolean hasPendingTxCommit = false; @@ -764,6 +766,7 @@ public boolean process(final long wallClockTime) { // get the next record to process record = partitionGroup.nextRecord(recordInfo, wallClockTime); + rawRecord = partitionGroup.rawHeadRecord(); // if there is no record to process, return immediately if (record == null) { @@ -1315,7 +1318,7 @@ long streamTime() { } ConsumerRecord rawRecord() { - return record.rawRecord(); + return rawRecord; } private class RecordQueueCreator { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SynchronizedPartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SynchronizedPartitionGroup.java index 3e544c432a285..086b9d4201212 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SynchronizedPartitionGroup.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SynchronizedPartitionGroup.java @@ -16,11 +16,12 @@ */ package org.apache.kafka.streams.processor.internals; -import java.util.Set; -import java.util.function.Function; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import java.util.Set; +import java.util.function.Function; + class SynchronizedPartitionGroup extends AbstractPartitionGroup { private final AbstractPartitionGroup wrapped; @@ -69,6 +70,11 @@ synchronized Long headRecordOffset(final TopicPartition partition) { return wrapped.headRecordOffset(partition); } + @Override + ConsumerRecord rawHeadRecord() { + return wrapped.rawHeadRecord(); + } + @Override synchronized int numBuffered() { return wrapped.numBuffered(); From 521a51d2712808ebc47ff9f7bbe2e5ba68a3bc7a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Sat, 18 May 2024 19:20:10 +0200 Subject: [PATCH 14/28] KAFKA-16448: Add tests for ProcessorContextImpl --- .../apache/kafka/streams/TopologyConfig.java | 18 +-- .../internals/ProcessorContextImpl.java | 7 +- .../internals/ProcessorContextImplTest.java | 124 ++++++++++++++++++ 3 files changed, 130 insertions(+), 19 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java index c3a9dcb2d50c7..9bb09c4eb3f78 100644 --- a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; -import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.internals.StreamsConfigUtils; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.internals.MaterializedInternal; @@ -53,7 +52,6 @@ import static org.apache.kafka.streams.StreamsConfig.IN_MEMORY; import static org.apache.kafka.streams.StreamsConfig.MAX_TASK_IDLE_MS_CONFIG; import static org.apache.kafka.streams.StreamsConfig.MAX_TASK_IDLE_MS_DOC; -import static org.apache.kafka.streams.StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG; import static org.apache.kafka.streams.StreamsConfig.ROCKS_DB; import static org.apache.kafka.streams.StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG; import static org.apache.kafka.streams.StreamsConfig.STATESTORE_CACHE_MAX_BYTES_DOC; @@ -137,7 +135,6 @@ public class TopologyConfig extends AbstractConfig { public final Class dslStoreSuppliers; public final Supplier timestampExtractorSupplier; public final Supplier deserializationExceptionHandlerSupplier; - public final Supplier processingExceptionHandlerSupplier; public TopologyConfig(final StreamsConfig globalAppConfigs) { this(null, globalAppConfigs, new Properties()); @@ -228,13 +225,6 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo deserializationExceptionHandlerSupplier = () -> globalAppConfigs.getConfiguredInstance(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, DeserializationExceptionHandler.class); } - if (isTopologyOverride(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, topologyOverrides)) { - processingExceptionHandlerSupplier = () -> getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class); - log.info("Topology {} is overriding {} to {}", topologyName, PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, getClass(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG)); - } else { - processingExceptionHandlerSupplier = () -> globalAppConfigs.getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class); - } - if (isTopologyOverride(DEFAULT_DSL_STORE_CONFIG, topologyOverrides)) { storeType = getString(DEFAULT_DSL_STORE_CONFIG); log.info("Topology {} is overriding {} to {}", topologyName, DEFAULT_DSL_STORE_CONFIG, storeType); @@ -290,8 +280,7 @@ public TaskConfig getTaskConfig() { maxBufferedSize, timestampExtractorSupplier.get(), deserializationExceptionHandlerSupplier.get(), - eosEnabled, - processingExceptionHandlerSupplier.get() + eosEnabled ); } @@ -302,22 +291,19 @@ public static class TaskConfig { public final TimestampExtractor timestampExtractor; public final DeserializationExceptionHandler deserializationExceptionHandler; public final boolean eosEnabled; - public final ProcessingExceptionHandler processingExceptionHandler; private TaskConfig(final long maxTaskIdleMs, final long taskTimeoutMs, final int maxBufferedSize, final TimestampExtractor timestampExtractor, final DeserializationExceptionHandler deserializationExceptionHandler, - final boolean eosEnabled, - final ProcessingExceptionHandler processingExceptionHandler) { + final boolean eosEnabled) { this.maxTaskIdleMs = maxTaskIdleMs; this.taskTimeoutMs = taskTimeoutMs; this.maxBufferedSize = maxBufferedSize; this.timestampExtractor = timestampExtractor; this.deserializationExceptionHandler = deserializationExceptionHandler; this.eosEnabled = eosEnabled; - this.processingExceptionHandler = processingExceptionHandler; } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index eb3eac8baffa1..ecfb60909678e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -60,6 +60,7 @@ public class ProcessorContextImpl extends AbstractProcessorContext cacheNameToFlushListener = new HashMap<>(); @@ -76,7 +77,7 @@ public ProcessorContextImpl(final TaskId id, appConfigs(), IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED, false); - + processingExceptionHandler = config.processingExceptionHandler(); } @Override @@ -297,13 +298,13 @@ private void forwardInternal(final ProcessorNode child, try { child.process(record); } catch (final Exception e) { + // prevent parent nodes to throw exception if (!processingExceptionOccurred) { processingExceptionOccurred = true; final ErrorHandlerContext errorHandlerContext = new ErrorHandlerContextImpl(null, topic(), partition(), offset(), headers(), streamTask.rawRecord().key(), streamTask.rawRecord().value(), child.name(), taskId()); - final ProcessingExceptionHandler.ProcessingHandlerResponse response = streamTask.config - .processingExceptionHandler + final ProcessingExceptionHandler.ProcessingHandlerResponse response = processingExceptionHandler .handle(errorHandlerContext, record, e); if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java index 6bee7f32c24fa..8d44f7f2730f5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeader; @@ -23,13 +24,18 @@ import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsConfig.InternalConfig; +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.ErrorHandlerContext; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.query.Position; @@ -63,6 +69,7 @@ import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.streams.processor.internals.ProcessorContextImpl.BYTEARRAY_VALUE_SERIALIZER; import static org.apache.kafka.streams.processor.internals.ProcessorContextImpl.BYTES_KEY_SERIALIZER; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; @@ -597,6 +604,123 @@ public void shouldSetAndGetProcessorMetaData() { assertThrows(NullPointerException.class, () -> context.setProcessorMetadata(null)); } + @Test + public void shouldContinueOnProcessingExceptions() { + when(streamsConfig.processingExceptionHandler()).thenReturn(new ProcessingExceptionHandler() { + @Override + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + assertArrayEquals(new byte[] {107, 101, 121}, context.sourceRawKey()); + assertArrayEquals(new byte[] {118, 97, 108, 117, 101}, context.sourceRawValue()); + assertEquals("fakeChild", context.processorNodeId()); + assertEquals("key", record.key()); + assertEquals("value", record.value()); + assertEquals("Exception should be handled by processing exception handler", exception.getMessage()); + + return ProcessingHandlerResponse.CONTINUE; + } + + @Override + public void configure(final Map configs) { + // No-op + } + }); + + context = new ProcessorContextImpl( + mock(TaskId.class), + streamsConfig, + stateManager, + mock(StreamsMetricsImpl.class), + mock(ThreadCache.class) + ); + + final StreamTask task = mock(StreamTask.class); + when(task.streamTime()).thenReturn(STREAM_TIME); + when(task.rawRecord()).thenReturn(new ConsumerRecord<>("topic", 0, 0, "key".getBytes(), "value".getBytes())); + context.transitionToActive(task, null, null); + + final ProcessorNode processorNode = new ProcessorNode<>( + "fake", + (org.apache.kafka.streams.processor.api.Processor) null, + Collections.emptySet() + ); + + final ProcessorNode childProcessorNode = new ProcessorNode<>( + "fakeChild", + (Processor) record -> { + throw new RuntimeException("Exception should be handled by processing exception handler"); + }, + Collections.emptySet() + ); + + processorNode.init(context); + childProcessorNode.init(context); + processorNode.addChild(childProcessorNode); + + context.setCurrentNode(processorNode); + + context.forward("key", "value"); + } + + @Test + public void shouldFailOnProcessingExceptions() { + when(streamsConfig.processingExceptionHandler()).thenReturn(new ProcessingExceptionHandler() { + @Override + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + assertArrayEquals(new byte[] {107, 101, 121}, context.sourceRawKey()); + assertArrayEquals(new byte[] {118, 97, 108, 117, 101}, context.sourceRawValue()); + assertEquals("fakeChild", context.processorNodeId()); + assertEquals("key", record.key()); + assertEquals("value", record.value()); + assertEquals("Exception should be handled by processing exception handler", exception.getMessage()); + + return ProcessingHandlerResponse.FAIL; + } + + @Override + public void configure(final Map configs) { + // No-op + } + }); + + context = new ProcessorContextImpl( + mock(TaskId.class), + streamsConfig, + stateManager, + mock(StreamsMetricsImpl.class), + mock(ThreadCache.class) + ); + + final StreamTask task = mock(StreamTask.class); + when(task.streamTime()).thenReturn(STREAM_TIME); + when(task.rawRecord()).thenReturn(new ConsumerRecord<>("topic", 0, 0, "key".getBytes(), "value".getBytes())); + context.transitionToActive(task, null, null); + + final ProcessorNode processorNode = new ProcessorNode<>( + "fake", + (org.apache.kafka.streams.processor.api.Processor) null, + Collections.emptySet() + ); + + final ProcessorNode childProcessorNode = new ProcessorNode<>( + "fakeChild", + (Processor) record -> { + throw new RuntimeException("Exception should be handled by processing exception handler"); + }, + Collections.emptySet() + ); + + processorNode.init(context); + childProcessorNode.init(context); + processorNode.addChild(childProcessorNode); + + context.setCurrentNode(processorNode); + + final StreamsException exception = assertThrows(StreamsException.class, () -> context.forward("key", "value")); + assertEquals("Processing exception handler is set to fail upon a processing error. " + + "If you would rather have the streaming pipeline continue after a processing error, " + + "please set the processing.exception.handler appropriately.", exception.getMessage()); + } + @SuppressWarnings("unchecked") private KeyValueStore keyValueStoreMock() { final KeyValueStore keyValueStoreMock = mock(KeyValueStore.class); From ea13eeae13238d3988f198b871eb2da5962c686d Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Mon, 20 May 2024 18:59:11 +0200 Subject: [PATCH 15/28] KAFKA-16448: split deserializer key and value to get origin --- .../internals/RecordCollectorImpl.java | 217 ++++++++++-------- 1 file changed, 123 insertions(+), 94 deletions(-) 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 7850c9f1eca3b..0ebaf774f2601 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 @@ -96,14 +96,14 @@ public RecordCollectorImpl(final LogContext logContext, for (final String topic : topology.sinkTopics()) { final String processorNodeId = topology.sink(topic).name(); producedSensorByTopic.put( - topic, - TopicMetrics.producedSensor( - threadId, - taskId.toString(), - processorNodeId, topic, - streamsMetrics - )); + TopicMetrics.producedSensor( + threadId, + taskId.toString(), + processorNodeId, + topic, + streamsMetrics + )); } this.offsets = new HashMap<>(); @@ -117,7 +117,7 @@ public void initialize() { } /** - * @throws StreamsException fatal error that should cause the thread to die + * @throws StreamsException fatal error that should cause the thread to die * @throws TaskMigratedException recoverable error that would cause the task to be removed */ @Override @@ -145,8 +145,8 @@ public void send(final String topic, // here we cannot drop the message on the floor even if it is a transient timeout exception, // so we treat everything the same as a fatal exception throw new StreamsException("Could not determine the number of partitions for topic '" + topic + - "' for task " + taskId + " due to " + fatal, - fatal + "' for task " + taskId + " due to " + fatal, + fatal ); } if (partitions.size() > 0) { @@ -162,14 +162,14 @@ public void send(final String topic, + "topic=[{}]", topic); droppedRecordsSensor.record(); } else { - for (final int multicastPartition: multicastPartitions) { + for (final int multicastPartition : multicastPartitions) { send(topic, key, value, headers, multicastPartition, timestamp, keySerializer, valueSerializer, processorNodeId, context); } } } } else { throw new StreamsException("Could not get partition information for topic " + topic + " for task " + taskId + - ". This can happen if the topic does not exist."); + ". This can happen if the topic does not exist."); } } else { send(topic, key, value, headers, null, timestamp, keySerializer, valueSerializer, processorNodeId, context); @@ -190,71 +190,22 @@ public void send(final String topic, final InternalProcessorContext context) { checkForException(); - final byte[] keyBytes; - final byte[] valBytes; + byte[] keyBytes = null; + byte[] valBytes = null; try { keyBytes = keySerializer.serialize(topic, headers, key); + } catch (final ClassCastException exception) { + this.manageClassCastException(topic, key, value, keySerializer, valueSerializer, exception); + } catch (final Exception exception) { + this.manageOtherExceptions(topic, key, value, headers, partition, timestamp, processorNodeId, context, exception, ProductionExceptionHandler.SerializationExceptionOrigin.KEY); + return; + } + try { valBytes = valueSerializer.serialize(topic, headers, value); } catch (final ClassCastException exception) { - final String keyClass = key == null ? "unknown because key is null" : key.getClass().getName(); - final String valueClass = value == null ? "unknown because value is null" : value.getClass().getName(); - throw new StreamsException( - String.format( - "ClassCastException while producing data to topic %s. " + - "A serializer (key: %s / value: %s) is not compatible to the actual key or value type " + - "(key type: %s / value type: %s). " + - "Change the default Serdes in StreamConfig or provide correct Serdes via method parameters " + - "(for example if using the DSL, `#to(String topic, Produced produced)` with " + - "`Produced.keySerde(WindowedSerdes.timeWindowedSerdeFrom(String.class))`).", - topic, - keySerializer.getClass().getName(), - valueSerializer.getClass().getName(), - keyClass, - valueClass), - exception); + this.manageClassCastException(topic, key, value, keySerializer, valueSerializer, exception); } catch (final Exception exception) { - final ProducerRecord record = new ProducerRecord<>(topic, partition, timestamp, key, value, headers); - final ProductionExceptionHandler.ProductionExceptionHandlerResponse response; - - log.debug(String.format("Error serializing record to topic %s", topic), exception); - - try { - final ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( - context, - record.topic(), - record.partition(), - context.offset(), - record.headers(), - null, - null, - processorNodeId, - taskId); - response = productionExceptionHandler.handleSerializationException(errorHandlerContext, record, exception, ProductionExceptionHandler.SerializationExceptionOrigin.VALUE); - } catch (final Exception e) { - log.error("Fatal when handling serialization exception", e); - recordSendError(topic, e, null, context, processorNodeId); - return; - } - - if (response == ProductionExceptionHandlerResponse.FAIL) { - throw new StreamsException( - String.format( - "Unable to serialize record. ProducerRecord(topic=[%s], partition=[%d], timestamp=[%d]", - topic, - partition, - timestamp), - exception - ); - } - - log.warn("Unable to serialize record, continue processing. " + - "ProducerRecord(topic=[{}], partition=[{}], timestamp=[{}])", - topic, - partition, - timestamp); - - droppedRecordsSensor.record(); - + this.manageOtherExceptions(topic, key, value, headers, partition, timestamp, processorNodeId, context, exception, ProductionExceptionHandler.SerializationExceptionOrigin.VALUE); return; } @@ -278,14 +229,14 @@ public void send(final String topic, // 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, - t -> TopicMetrics.producedSensor( - Thread.currentThread().getName(), - taskId.toString(), - processorNodeId, topic, - context.metrics() - ) + t -> TopicMetrics.producedSensor( + Thread.currentThread().getName(), + taskId.toString(), + processorNodeId, + topic, + context.metrics() + ) ); final long bytesProduced = producerRecordSizeInBytes(serializedRecord); topicProducedSensor.record(bytesProduced, context.currentSystemTimeMs()); @@ -299,6 +250,84 @@ public void send(final String topic, }); } + private void manageClassCastException(final String topic, + final K key, + final V value, + final Serializer keySerializer, + final Serializer valueSerializer, + final ClassCastException exception) { + final String keyClass = key == null ? "unknown because key is null" : key.getClass().getName(); + final String valueClass = value == null ? "unknown because value is null" : value.getClass().getName(); + throw new StreamsException( + String.format( + "ClassCastException while producing data to topic %s. " + + "A serializer (key: %s / value: %s) is not compatible to the actual key or value type " + + "(key type: %s / value type: %s). " + + "Change the default Serdes in StreamConfig or provide correct Serdes via method parameters " + + "(for example if using the DSL, `#to(String topic, Produced produced)` with " + + "`Produced.keySerde(WindowedSerdes.timeWindowedSerdeFrom(String.class))`).", + topic, + keySerializer.getClass().getName(), + valueSerializer.getClass().getName(), + keyClass, + valueClass), + exception); + } + + private void manageOtherExceptions(final String topic, + final K key, + final V value, + final Headers headers, + final Integer partition, + final Long timestamp, + final String processorNodeId, + final InternalProcessorContext context, + final Exception exception, + final ProductionExceptionHandler.SerializationExceptionOrigin origin) { + final ProducerRecord record = new ProducerRecord<>(topic, partition, timestamp, key, value, headers); + final ProductionExceptionHandler.ProductionExceptionHandlerResponse response; + + log.debug(String.format("Error serializing record to topic %s", topic), exception); + + try { + final ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( + context, + record.topic(), + record.partition(), + context.offset(), + record.headers(), + null, + null, + processorNodeId, + taskId); + response = productionExceptionHandler.handleSerializationException(errorHandlerContext, record, exception, origin); + } catch (final Exception e) { + log.error("Fatal when handling serialization exception", e); + recordSendError(topic, e, null, context, processorNodeId); + return; + } + + if (response == ProductionExceptionHandlerResponse.FAIL) { + throw new StreamsException( + String.format( + "Unable to serialize record. ProducerRecord(topic=[%s], partition=[%d], timestamp=[%d]", + topic, + partition, + timestamp), + exception + ); + } + + log.warn("Unable to serialize record, continue processing. " + + "ProducerRecord(topic=[{}], partition=[{}], timestamp=[{}])", + topic, + partition, + timestamp); + + droppedRecordsSensor.record(); + + } + private void recordSendError(final String topic, final Exception exception, final ProducerRecord serializedRecord, final InternalProcessorContext context, final String processorNodeId) { String errorMessage = String.format(SEND_EXCEPTION_MESSAGE, topic, taskId, exception.toString()); @@ -310,21 +339,21 @@ private void recordSendError(final String topic, final Exception exception, fina exception instanceof InvalidProducerEpochException || exception instanceof OutOfOrderSequenceException) { errorMessage += "\nWritten offsets would not be recorded and no more records would be sent since the producer is fenced, " + - "indicating the task may be migrated out"; + "indicating the task may be migrated out"; sendException.set(new TaskMigratedException(errorMessage, exception)); } else { if (exception instanceof RetriableException) { 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"; + "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 { final ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( - context, + null, serializedRecord.topic(), serializedRecord.partition(), - context.offset(), + context != null ? context.offset() : -1, serializedRecord.headers(), null, null, @@ -346,20 +375,20 @@ private void recordSendError(final String topic, final Exception exception, fina private boolean isFatalException(final Exception exception) { final boolean securityException = exception instanceof AuthenticationException || - exception instanceof AuthorizationException || - exception instanceof SecurityDisabledException; + exception instanceof AuthorizationException || + exception instanceof SecurityDisabledException; final boolean communicationException = exception instanceof InvalidTopicException || - exception instanceof UnknownServerException || - exception instanceof SerializationException || - exception instanceof OffsetMetadataTooLarge || - exception instanceof IllegalStateException; + exception instanceof UnknownServerException || + exception instanceof SerializationException || + exception instanceof OffsetMetadataTooLarge || + exception instanceof IllegalStateException; return securityException || communicationException; } /** - * @throws StreamsException fatal error that should cause the thread to die + * @throws StreamsException fatal error that should cause the thread to die * @throws TaskMigratedException recoverable error that would cause the task to be removed */ @Override @@ -370,7 +399,7 @@ public void flush() { } /** - * @throws StreamsException fatal error that should cause the thread to die + * @throws StreamsException fatal error that should cause the thread to die * @throws TaskMigratedException recoverable error that would cause the task to be removed */ @Override @@ -387,7 +416,7 @@ public void closeClean() { } /** - * @throws StreamsException fatal error that should cause the thread to die + * @throws StreamsException fatal error that should cause the thread to die * @throws TaskMigratedException recoverable error that would cause the task to be removed */ @Override From db45b6b41d8ffdb08739d2d204327a69fcaa7a09 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Mon, 20 May 2024 20:29:17 +0200 Subject: [PATCH 16/28] KAFKA-16448: fix TU after classCastException --- .../streams/processor/internals/ProcessorContextImpl.java | 5 ++++- .../kafka/streams/processor/internals/ProcessorNodeTest.java | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index ecfb60909678e..55bf9ea4cd88b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -301,8 +301,11 @@ private void forwardInternal(final ProcessorNode child, // prevent parent nodes to throw exception if (!processingExceptionOccurred) { processingExceptionOccurred = true; + final byte[] rawKey = streamTask.rawRecord() != null ? streamTask.rawRecord().key() : null; + final byte[] rawValue = streamTask.rawRecord() != null ? streamTask.rawRecord().value() : null; + final ErrorHandlerContext errorHandlerContext = new ErrorHandlerContextImpl(null, topic(), - partition(), offset(), headers(), streamTask.rawRecord().key(), streamTask.rawRecord().value(), + partition(), offset(), headers(), rawKey, rawValue, child.name(), taskId()); final ProcessingExceptionHandler.ProcessingHandlerResponse response = processingExceptionHandler .handle(errorHandlerContext, record, e); 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 dfa3f9e422a75..eea4e63c1c777 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 @@ -139,7 +139,7 @@ public void testTopologyLevelClassCastException() { final TestInputTopic topic = testDriver.createInputTopic("streams-plaintext-input", new StringSerializer(), new StringSerializer()); final StreamsException se = assertThrows(StreamsException.class, () -> topic.pipeInput("a-key", "a value")); - final String msg = se.getMessage(); + final String msg = se.getCause().getMessage(); assertTrue("Error about class cast with serdes", msg.contains("ClassCastException")); assertTrue("Error about class cast with serdes", msg.contains("Serdes")); } From 2171fbe205357c10879e41c80b8257b084cb6a8a Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Mon, 20 May 2024 21:48:09 +0200 Subject: [PATCH 17/28] KAFKA-16448: remove deprecated --- .../errors/AlwaysContinueProductionExceptionHandler.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java b/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java index 32c685d7f0caf..61fea5f27bb40 100644 --- a/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java +++ b/streams/src/test/java/org/apache/kafka/streams/errors/AlwaysContinueProductionExceptionHandler.java @@ -27,7 +27,6 @@ */ public class AlwaysContinueProductionExceptionHandler implements ProductionExceptionHandler { @Override - @Deprecated public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext context, final ProducerRecord record, final Exception exception) { @@ -35,7 +34,6 @@ public ProductionExceptionHandlerResponse handle(final ErrorHandlerContext conte } @Override - @Deprecated public ProductionExceptionHandlerResponse handleSerializationException(final ErrorHandlerContext context, final ProducerRecord record, final Exception exception, From 028ba33630271ace6de7486d9c1309f2f284d2f2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Wed, 22 May 2024 11:41:26 +0200 Subject: [PATCH 18/28] KAFKA-16448: Fix indent --- .../apache/kafka/streams/StreamsConfig.java | 913 +++++++++--------- .../apache/kafka/streams/TopologyConfig.java | 22 +- 2 files changed, 464 insertions(+), 471 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 b1a27056f473e..908b33af303b8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -36,13 +36,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; -import org.apache.kafka.streams.errors.DeserializationExceptionHandler; -import org.apache.kafka.streams.errors.LogAndFailExceptionHandler; -import org.apache.kafka.streams.errors.ProcessingExceptionHandler; -import org.apache.kafka.streams.errors.ProcessingLogAndFailExceptionHandler; -import org.apache.kafka.streams.errors.ProductionExceptionHandler; -import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.errors.*; import org.apache.kafka.streams.internals.StreamsConfigUtils; import org.apache.kafka.streams.internals.UpgradeFromValues; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; @@ -250,9 +244,9 @@ public class StreamsConfig extends AbstractConfig { + "(\"+REUSE_KTABLE_SOURCE_TOPICS+\", \"+MERGE_REPARTITION_TOPICS+\" + " + "\"SINGLE_STORE_SELF_JOIN+\")."; private static final String TOPOLOGY_OPTIMIZATION_DOC = "A configuration telling Kafka " - + "Streams if it should optimize the topology and what optimizations to apply. " - + CONFIG_ERROR_MSG - + "\"NO_OPTIMIZATION\" by default."; + + "Streams if it should optimize the topology and what optimizations to apply. " + + CONFIG_ERROR_MSG + + "\"NO_OPTIMIZATION\" by default."; /** * Config value for parameter {@link #TOPOLOGY_OPTIMIZATION_CONFIG "topology.optimization"} for disabling topology optimization @@ -285,8 +279,8 @@ public class StreamsConfig extends AbstractConfig { public static final String SINGLE_STORE_SELF_JOIN = "single.store.self.join"; private static final List TOPOLOGY_OPTIMIZATION_CONFIGS = Arrays.asList( - OPTIMIZE, NO_OPTIMIZATION, REUSE_KTABLE_SOURCE_TOPICS, MERGE_REPARTITION_TOPICS, - SINGLE_STORE_SELF_JOIN); + OPTIMIZE, NO_OPTIMIZATION, REUSE_KTABLE_SOURCE_TOPICS, MERGE_REPARTITION_TOPICS, + SINGLE_STORE_SELF_JOIN); /** * Config value for parameter {@link #UPGRADE_FROM_CONFIG "upgrade.from"} for upgrading an application from version {@code 0.10.0.x}. @@ -474,9 +468,9 @@ public class StreamsConfig extends AbstractConfig { /** {@code acceptable.recovery.lag} */ public static final String ACCEPTABLE_RECOVERY_LAG_CONFIG = "acceptable.recovery.lag"; private static final String ACCEPTABLE_RECOVERY_LAG_DOC = "The maximum acceptable lag (number of offsets to catch up) for a client to be considered caught-up enough" + - " to receive an active task assignment. Upon assignment, it will still restore the rest of the changelog" + - " before processing. To avoid a pause in processing during rebalances, this config" + - " should correspond to a recovery time of well under a minute for a given workload. Must be at least 0."; + " to receive an active task assignment. Upon assignment, it will still restore the rest of the changelog" + + " before processing. To avoid a pause in processing during rebalances, this config" + + " should correspond to a recovery time of well under a minute for a given workload. Must be at least 0."; /** {@code application.id} */ @SuppressWarnings("WeakerAccess") @@ -517,22 +511,22 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; private static final String CLIENT_ID_DOC = "An ID prefix string used for the client IDs of internal (main, restore, and global) consumers , producers, and admin clients" + - " with pattern <client.id>-[Global]StreamThread[-<threadSequenceNumber>]-<consumer|producer|restore-consumer|global-consumer>."; + " with pattern <client.id>-[Global]StreamThread[-<threadSequenceNumber>]-<consumer|producer|restore-consumer|global-consumer>."; /** {@code enable.metrics.push} */ @SuppressWarnings("WeakerAccess") public static final String ENABLE_METRICS_PUSH_CONFIG = CommonClientConfigs.ENABLE_METRICS_PUSH_CONFIG; - public static final String ENABLE_METRICS_PUSH_DOC = "Whether to enable pushing of internal client metrics for (main, restore, and global) consumers, producers, and admin clients." + - " The cluster must have a client metrics subscription which corresponds to a client."; + public static final String ENABLE_METRICS_PUSH_DOC = "Whether to enable pushing of internal client metrics for (main, restore, and global) consumers, producers, and admin clients." + + " The cluster must have a client metrics subscription which corresponds to a client."; /** {@code commit.interval.ms} */ @SuppressWarnings("WeakerAccess") public static final String COMMIT_INTERVAL_MS_CONFIG = "commit.interval.ms"; private static final String COMMIT_INTERVAL_MS_DOC = "The frequency in milliseconds with which to commit processing progress." + - " For at-least-once processing, committing means to save the position (ie, offsets) of the processor." + - " For exactly-once processing, it means to commit the transaction which includes to save the position and to make the committed data in the output topic visible to consumers with isolation level read_committed." + - " (Note, if processing.guarantee is set to " + EXACTLY_ONCE_V2 + ", " + EXACTLY_ONCE + ",the default value is " + EOS_DEFAULT_COMMIT_INTERVAL_MS + "," + - " otherwise the default value is " + DEFAULT_COMMIT_INTERVAL_MS + "."; + " For at-least-once processing, committing means to save the position (ie, offsets) of the processor." + + " For exactly-once processing, it means to commit the transaction which includes to save the position and to make the committed data in the output topic visible to consumers with isolation level read_committed." + + " (Note, if processing.guarantee is set to " + EXACTLY_ONCE_V2 + ", " + EXACTLY_ONCE + ",the default value is " + EOS_DEFAULT_COMMIT_INTERVAL_MS + "," + + " otherwise the default value is " + DEFAULT_COMMIT_INTERVAL_MS + "."; /** {@code repartition.purge.interval.ms} */ @SuppressWarnings("WeakerAccess") @@ -555,12 +549,11 @@ public class StreamsConfig extends AbstractConfig { public static final String DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG = "default.production.exception.handler"; private static final String DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the org.apache.kafka.streams.errors.ProductionExceptionHandler interface."; - /** {@code default.deserialization.exception.handler} */ + /** {@code processing.exception.handler} */ @SuppressWarnings("WeakerAccess") public static final String PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG = "processing.exception.handler"; public static final String PROCESSING_EXCEPTION_HANDLER_CLASS_DOC = "Exception handling class that implements the org.apache.kafka.streams.errors.ProcessingExceptionHandler interface."; - /** {@code default.dsl.store} */ @Deprecated @SuppressWarnings("WeakerAccess") @@ -586,7 +579,7 @@ public class StreamsConfig extends AbstractConfig { @Deprecated public static final String DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS = "default.windowed.key.serde.inner"; private static final String DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS_DOC = "Default serializer / deserializer for the inner class of a windowed key. Must implement the " + - "org.apache.kafka.common.serialization.Serde interface."; + "org.apache.kafka.common.serialization.Serde interface."; /** {@code default.windowed.value.serde.inner} * @deprecated since 3.0.0 Use {@link #WINDOWED_INNER_CLASS_SERDE "windowed.inner.class.serde"} instead. */ @@ -594,13 +587,13 @@ public class StreamsConfig extends AbstractConfig { @Deprecated public static final String DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS = "default.windowed.value.serde.inner"; private static final String DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS_DOC = "Default serializer / deserializer for the inner class of a windowed value. Must implement the " + - "org.apache.kafka.common.serialization.Serde interface."; + "org.apache.kafka.common.serialization.Serde interface."; public static final String WINDOWED_INNER_CLASS_SERDE = "windowed.inner.class.serde"; private static final String WINDOWED_INNER_CLASS_SERDE_DOC = " Default serializer / deserializer for the inner class of a windowed record. Must implement the " + - "org.apache.kafka.common.serialization.Serde interface. Note that setting this config in KafkaStreams application would result " + - "in an error as it is meant to be used only from Plain consumer client."; - + "org.apache.kafka.common.serialization.Serde interface. Note that setting this config in KafkaStreams application would result " + + "in an error as it is meant to be used only from Plain consumer client."; + /** {@code default key.serde} */ @SuppressWarnings("WeakerAccess") public static final String DEFAULT_KEY_SERDE_CLASS_CONFIG = "default.key.serde"; @@ -623,27 +616,27 @@ public class StreamsConfig extends AbstractConfig { /** {@code max.task.idle.ms} */ public static final String MAX_TASK_IDLE_MS_CONFIG = "max.task.idle.ms"; public static final String MAX_TASK_IDLE_MS_DOC = "This config controls whether joins and merges" - + " may produce out-of-order results." - + " The config value is the maximum amount of time in milliseconds a stream task will stay idle" - + " when it is fully caught up on some (but not all) input partitions" - + " to wait for producers to send additional records and avoid potential" - + " out-of-order record processing across multiple input streams." - + " The default (zero) does not wait for producers to send more records," - + " but it does wait to fetch data that is already present on the brokers." - + " This default means that for records that are already present on the brokers," - + " Streams will process them in timestamp order." - + " Set to -1 to disable idling entirely and process any locally available data," - + " even though doing so may produce out-of-order processing."; + + " may produce out-of-order results." + + " The config value is the maximum amount of time in milliseconds a stream task will stay idle" + + " when it is fully caught up on some (but not all) input partitions" + + " to wait for producers to send additional records and avoid potential" + + " out-of-order record processing across multiple input streams." + + " The default (zero) does not wait for producers to send more records," + + " but it does wait to fetch data that is already present on the brokers." + + " This default means that for records that are already present on the brokers," + + " Streams will process them in timestamp order." + + " Set to -1 to disable idling entirely and process any locally available data," + + " even though doing so may produce out-of-order processing."; /** {@code max.warmup.replicas} */ public static final String MAX_WARMUP_REPLICAS_CONFIG = "max.warmup.replicas"; private static final String MAX_WARMUP_REPLICAS_DOC = "The maximum number of warmup replicas (extra standbys beyond the configured num.standbys) that can be assigned at once for the purpose of keeping " + - " the task available on one instance while it is warming up on another instance it has been reassigned to. Used to throttle how much extra broker " + - " traffic and cluster state can be used for high availability. Must be at least 1." + - "Note that one warmup replica corresponds to one Stream Task. Furthermore, note that each warmup replica can only be promoted to an active task " + - "during a rebalance (normally during a so-called probing rebalance, which occur at a frequency specified by the `probing.rebalance.interval.ms` config). This means " + - "that the maximum rate at which active tasks can be migrated from one Kafka Streams Instance to another instance can be determined by " + - "(`max.warmup.replicas` / `probing.rebalance.interval.ms`)."; + " the task available on one instance while it is warming up on another instance it has been reassigned to. Used to throttle how much extra broker " + + " traffic and cluster state can be used for high availability. Must be at least 1." + + "Note that one warmup replica corresponds to one Stream Task. Furthermore, note that each warmup replica can only be promoted to an active task " + + "during a rebalance (normally during a so-called probing rebalance, which occur at a frequency specified by the `probing.rebalance.interval.ms` config). This means " + + "that the maximum rate at which active tasks can be migrated from one Kafka Streams Instance to another instance can be determined by " + + "(`max.warmup.replicas` / `probing.rebalance.interval.ms`)."; /** {@code metadata.max.age.ms} */ @SuppressWarnings("WeakerAccess") @@ -688,19 +681,19 @@ public class StreamsConfig extends AbstractConfig { /** {@code probing.rebalance.interval.ms} */ public static final String PROBING_REBALANCE_INTERVAL_MS_CONFIG = "probing.rebalance.interval.ms"; private static final String PROBING_REBALANCE_INTERVAL_MS_DOC = "The maximum time in milliseconds to wait before triggering a rebalance to probe for warmup replicas that have finished warming up and are ready to become active." + - " Probing rebalances will continue to be triggered until the assignment is balanced. Must be at least 1 minute."; + " Probing rebalances will continue to be triggered until the assignment is balanced. Must be at least 1 minute."; /** {@code processing.guarantee} */ @SuppressWarnings("WeakerAccess") public static final String PROCESSING_GUARANTEE_CONFIG = "processing.guarantee"; private static final String PROCESSING_GUARANTEE_DOC = "The processing guarantee that should be used. " + - "Possible values are " + AT_LEAST_ONCE + " (default) " + - "and " + EXACTLY_ONCE_V2 + " (requires brokers version 2.5 or higher). " + - "Deprecated options are " + EXACTLY_ONCE + " (requires brokers version 0.11.0 or higher) " + - "and " + EXACTLY_ONCE_BETA + " (requires brokers version 2.5 or higher). " + - "Note that exactly-once processing requires a cluster of at least three brokers by default what is the " + - "recommended setting for production; for development you can change this, by adjusting broker setting " + - "transaction.state.log.replication.factor and transaction.state.log.min.isr."; + "Possible values are " + AT_LEAST_ONCE + " (default) " + + "and " + EXACTLY_ONCE_V2 + " (requires brokers version 2.5 or higher). " + + "Deprecated options are " + EXACTLY_ONCE + " (requires brokers version 0.11.0 or higher) " + + "and " + EXACTLY_ONCE_BETA + " (requires brokers version 2.5 or higher). " + + "Note that exactly-once processing requires a cluster of at least three brokers by default what is the " + + "recommended setting for production; for development you can change this, by adjusting broker setting " + + "transaction.state.log.replication.factor and transaction.state.log.min.isr."; /** {@code receive.buffer.bytes} */ @SuppressWarnings("WeakerAccess") @@ -710,8 +703,8 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String RACK_AWARE_ASSIGNMENT_TAGS_CONFIG = "rack.aware.assignment.tags"; private static final String RACK_AWARE_ASSIGNMENT_TAGS_DOC = "List of client tag keys used to distribute standby replicas across Kafka Streams instances." + - " When configured, Kafka Streams will make a best-effort to distribute" + - " the standby tasks over each client tag dimension."; + " When configured, Kafka Streams will make a best-effort to distribute" + + " the standby tasks over each client tag dimension."; /** {@code reconnect.backoff.ms} */ @SuppressWarnings("WeakerAccess") @@ -725,7 +718,7 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String REPLICATION_FACTOR_CONFIG = "replication.factor"; private static final String REPLICATION_FACTOR_DOC = "The replication factor for change log topics and repartition topics created by the stream processing application." - + " The default of -1 (meaning: use broker default replication factor) requires broker version 2.4 or newer"; + + " The default of -1 (meaning: use broker default replication factor) requires broker version 2.4 or newer"; /** {@code request.timeout.ms} */ @SuppressWarnings("WeakerAccess") @@ -772,8 +765,8 @@ public class StreamsConfig extends AbstractConfig { /** {@code task.timeout.ms} */ public static final String TASK_TIMEOUT_MS_CONFIG = "task.timeout.ms"; public static final String TASK_TIMEOUT_MS_DOC = "The maximum amount of time in milliseconds a task might stall due to internal errors and retries until an error is raised. " + - "For a timeout of 0ms, a task would raise an error for the first internal error. " + - "For any timeout larger than 0ms, a task will retry at least once before an error is raised."; + "For a timeout of 0ms, a task would raise an error for the first internal error. " + + "For any timeout larger than 0ms, a task will retry at least once before an error is raised."; /** {@code window.size.ms} */ @@ -784,16 +777,16 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String UPGRADE_FROM_CONFIG = "upgrade.from"; private static final String UPGRADE_FROM_DOC = "Allows upgrading in a backward compatible way. " + - "This is needed when upgrading from [0.10.0, 1.1] to 2.0+, or when upgrading from [2.0, 2.3] to 2.4+. " + - "When upgrading from 3.3 to a newer version it is not required to specify this config. Default is `null`. " + - "Accepted values are \"" + UPGRADE_FROM_0100 + "\", \"" + UPGRADE_FROM_0101 + "\", \"" + - UPGRADE_FROM_0102 + "\", \"" + UPGRADE_FROM_0110 + "\", \"" + UPGRADE_FROM_10 + "\", \"" + - UPGRADE_FROM_11 + "\", \"" + UPGRADE_FROM_20 + "\", \"" + UPGRADE_FROM_21 + "\", \"" + - UPGRADE_FROM_22 + "\", \"" + UPGRADE_FROM_23 + "\", \"" + UPGRADE_FROM_24 + "\", \"" + - UPGRADE_FROM_25 + "\", \"" + UPGRADE_FROM_26 + "\", \"" + UPGRADE_FROM_27 + "\", \"" + - UPGRADE_FROM_28 + "\", \"" + UPGRADE_FROM_30 + "\", \"" + UPGRADE_FROM_31 + "\", \"" + - UPGRADE_FROM_32 + "\", \"" + UPGRADE_FROM_33 + "\", \"" + UPGRADE_FROM_34 + "\", \"" + - UPGRADE_FROM_35 + "\", \"" + UPGRADE_FROM_36 + "\", \"" + UPGRADE_FROM_37 + "(for upgrading from the corresponding old version)."; + "This is needed when upgrading from [0.10.0, 1.1] to 2.0+, or when upgrading from [2.0, 2.3] to 2.4+. " + + "When upgrading from 3.3 to a newer version it is not required to specify this config. Default is `null`. " + + "Accepted values are \"" + UPGRADE_FROM_0100 + "\", \"" + UPGRADE_FROM_0101 + "\", \"" + + UPGRADE_FROM_0102 + "\", \"" + UPGRADE_FROM_0110 + "\", \"" + UPGRADE_FROM_10 + "\", \"" + + UPGRADE_FROM_11 + "\", \"" + UPGRADE_FROM_20 + "\", \"" + UPGRADE_FROM_21 + "\", \"" + + UPGRADE_FROM_22 + "\", \"" + UPGRADE_FROM_23 + "\", \"" + UPGRADE_FROM_24 + "\", \"" + + UPGRADE_FROM_25 + "\", \"" + UPGRADE_FROM_26 + "\", \"" + UPGRADE_FROM_27 + "\", \"" + + UPGRADE_FROM_28 + "\", \"" + UPGRADE_FROM_30 + "\", \"" + UPGRADE_FROM_31 + "\", \"" + + UPGRADE_FROM_32 + "\", \"" + UPGRADE_FROM_33 + "\", \"" + UPGRADE_FROM_34 + "\", \"" + + UPGRADE_FROM_35 + "\", \"" + UPGRADE_FROM_36 + "\", \"" + UPGRADE_FROM_37 + "(for upgrading from the corresponding old version)."; /** {@code windowstore.changelog.additional.retention.ms} */ @SuppressWarnings("WeakerAccess") @@ -813,20 +806,20 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String RACK_AWARE_ASSIGNMENT_STRATEGY_CONFIG = "rack.aware.assignment.strategy"; public static final String RACK_AWARE_ASSIGNMENT_STRATEGY_DOC = "The strategy we use for rack aware assignment. Rack aware assignment will take client.rack and racks of TopicPartition into account when assigning" - + " tasks to minimize cross rack traffic. Valid settings are : " + RACK_AWARE_ASSIGNMENT_STRATEGY_NONE + " (default), which will disable rack aware assignment; " + RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC - + ", which will compute minimum cross rack traffic assignment; " + RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY + ", which will compute minimum cross rack traffic and try to balance the tasks of same subtopolgies across different clients"; + + " tasks to minimize cross rack traffic. Valid settings are : " + RACK_AWARE_ASSIGNMENT_STRATEGY_NONE + " (default), which will disable rack aware assignment; " + RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC + + ", which will compute minimum cross rack traffic assignment; " + RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY + ", which will compute minimum cross rack traffic and try to balance the tasks of same subtopolgies across different clients"; @SuppressWarnings("WeakerAccess") public static final String RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG = "rack.aware.assignment.traffic_cost"; public static final String RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_DOC = "Cost associated with cross rack traffic. This config and rack.aware.assignment.non_overlap_cost controls whether the " - + "optimization algorithm favors minimizing cross rack traffic or minimize the movement of tasks in existing assignment. If set a larger value " + RackAwareTaskAssignor.class.getName() + " will " - + "optimize for minimizing cross rack traffic. The default value is null which means it will use default traffic cost values in different assignors."; + + "optimization algorithm favors minimizing cross rack traffic or minimize the movement of tasks in existing assignment. If set a larger value " + RackAwareTaskAssignor.class.getName() + " will " + + "optimize for minimizing cross rack traffic. The default value is null which means it will use default traffic cost values in different assignors."; @SuppressWarnings("WeakerAccess") public static final String RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG = "rack.aware.assignment.non_overlap_cost"; public static final String RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_DOC = "Cost associated with moving tasks from existing assignment. This config and " + RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG + " controls whether the " - + "optimization algorithm favors minimizing cross rack traffic or minimize the movement of tasks in existing assignment. If set a larger value " + RackAwareTaskAssignor.class.getName() + " will " - + "optimize to maintain the existing assignment. The default value is null which means it will use default non_overlap cost values in different assignors."; + + "optimization algorithm favors minimizing cross rack traffic or minimize the movement of tasks in existing assignment. If set a larger value " + RackAwareTaskAssignor.class.getName() + " will " + + "optimize to maintain the existing assignment. The default value is null which means it will use default non_overlap cost values in different assignors."; /** @@ -838,365 +831,365 @@ public class StreamsConfig extends AbstractConfig { private static final String[] NON_CONFIGURABLE_CONSUMER_DEFAULT_CONFIGS = - new String[] {ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG}; + new String[] {ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG}; private static final String[] NON_CONFIGURABLE_CONSUMER_EOS_CONFIGS = - new String[] {ConsumerConfig.ISOLATION_LEVEL_CONFIG}; + new String[] {ConsumerConfig.ISOLATION_LEVEL_CONFIG}; private static final String[] NON_CONFIGURABLE_PRODUCER_EOS_CONFIGS = - new String[] { - ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, - ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, - ProducerConfig.TRANSACTIONAL_ID_CONFIG - }; + new String[] { + ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, + ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, + ProducerConfig.TRANSACTIONAL_ID_CONFIG + }; static { CONFIG = new ConfigDef() - // HIGH - - .define(APPLICATION_ID_CONFIG, // required with no default value - Type.STRING, - Importance.HIGH, - APPLICATION_ID_DOC) - .define(BOOTSTRAP_SERVERS_CONFIG, // required with no default value - Type.LIST, - Importance.HIGH, - CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) - .define(NUM_STANDBY_REPLICAS_CONFIG, - Type.INT, - 0, - Importance.HIGH, - NUM_STANDBY_REPLICAS_DOC) - .define(STATE_DIR_CONFIG, - Type.STRING, - System.getProperty("java.io.tmpdir") + File.separator + "kafka-streams", - Importance.HIGH, - STATE_DIR_DOC, - "${java.io.tmpdir}") - - // MEDIUM - - .define(ACCEPTABLE_RECOVERY_LAG_CONFIG, - Type.LONG, - 10_000L, - atLeast(0), - Importance.MEDIUM, - ACCEPTABLE_RECOVERY_LAG_DOC) - .define(CACHE_MAX_BYTES_BUFFERING_CONFIG, - Type.LONG, - 10 * 1024 * 1024L, - atLeast(0), - Importance.MEDIUM, - CACHE_MAX_BYTES_BUFFERING_DOC) - .define(STATESTORE_CACHE_MAX_BYTES_CONFIG, - Type.LONG, - 10 * 1024 * 1024L, - atLeast(0), - Importance.MEDIUM, - STATESTORE_CACHE_MAX_BYTES_DOC) - .define(CLIENT_ID_CONFIG, - Type.STRING, - "", - Importance.MEDIUM, - CLIENT_ID_DOC, - "<application.id>-<random-UUID>") - .define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, - Type.CLASS, - LogAndFailExceptionHandler.class.getName(), - Importance.MEDIUM, - DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC) - .define(DEFAULT_KEY_SERDE_CLASS_CONFIG, - Type.CLASS, - null, - Importance.MEDIUM, - DEFAULT_KEY_SERDE_CLASS_DOC) - .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC) - .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC) - .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC) - .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, - Type.CLASS, - null, - Importance.MEDIUM, - CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC) - .define(DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, - Type.CLASS, - DefaultProductionExceptionHandler.class.getName(), - Importance.MEDIUM, - DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC) - .define(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, - Type.CLASS, - ProcessingLogAndFailExceptionHandler.class.getName(), - Importance.MEDIUM, - PROCESSING_EXCEPTION_HANDLER_CLASS_DOC) - .define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, - Type.CLASS, - FailOnInvalidTimestamp.class.getName(), - Importance.MEDIUM, - DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC) - .define(DEFAULT_VALUE_SERDE_CLASS_CONFIG, - Type.CLASS, - null, - Importance.MEDIUM, - DEFAULT_VALUE_SERDE_CLASS_DOC) - .define(MAX_TASK_IDLE_MS_CONFIG, - Type.LONG, - 0L, - Importance.MEDIUM, - MAX_TASK_IDLE_MS_DOC) - .define(MAX_WARMUP_REPLICAS_CONFIG, - Type.INT, - 2, - atLeast(1), - Importance.MEDIUM, - MAX_WARMUP_REPLICAS_DOC) - .define(NUM_STREAM_THREADS_CONFIG, - Type.INT, - 1, - Importance.MEDIUM, - NUM_STREAM_THREADS_DOC) - .define(PROCESSING_GUARANTEE_CONFIG, - Type.STRING, - AT_LEAST_ONCE, - in(AT_LEAST_ONCE, EXACTLY_ONCE, EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2), - Importance.MEDIUM, - PROCESSING_GUARANTEE_DOC) - .define(RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG, - Type.INT, - null, - Importance.MEDIUM, - RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_DOC) - .define(RACK_AWARE_ASSIGNMENT_STRATEGY_CONFIG, - Type.STRING, - RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, - in(RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC, RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY), - Importance.MEDIUM, - RACK_AWARE_ASSIGNMENT_STRATEGY_DOC) - .define(RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, - Type.LIST, - Collections.emptyList(), - atMostOfSize(MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE), - Importance.MEDIUM, - RACK_AWARE_ASSIGNMENT_TAGS_DOC) - .define(RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG, - Type.INT, - null, - Importance.MEDIUM, - RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_DOC) - .define(REPLICATION_FACTOR_CONFIG, - Type.INT, - -1, - Importance.MEDIUM, - REPLICATION_FACTOR_DOC) - .define(SECURITY_PROTOCOL_CONFIG, - Type.STRING, - CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL, - ConfigDef.CaseInsensitiveValidString.in(Utils.enumOptions(SecurityProtocol.class)), - Importance.MEDIUM, - CommonClientConfigs.SECURITY_PROTOCOL_DOC) - .define(TASK_TIMEOUT_MS_CONFIG, - Type.LONG, - Duration.ofMinutes(5L).toMillis(), - atLeast(0L), - Importance.MEDIUM, - TASK_TIMEOUT_MS_DOC) - .define(TOPOLOGY_OPTIMIZATION_CONFIG, - Type.STRING, - NO_OPTIMIZATION, - (name, value) -> verifyTopologyOptimizationConfigs((String) value), - Importance.MEDIUM, - TOPOLOGY_OPTIMIZATION_DOC) - - // LOW - - .define(APPLICATION_SERVER_CONFIG, - Type.STRING, - "", - Importance.LOW, - APPLICATION_SERVER_DOC) - .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, - Type.INT, - 1000, - Importance.LOW, - BUFFERED_RECORDS_PER_PARTITION_DOC) - .define(BUILT_IN_METRICS_VERSION_CONFIG, - Type.STRING, - METRICS_LATEST, - in( - METRICS_LATEST - ), - Importance.LOW, - BUILT_IN_METRICS_VERSION_DOC) - .define(COMMIT_INTERVAL_MS_CONFIG, - Type.LONG, - DEFAULT_COMMIT_INTERVAL_MS, - atLeast(0), - Importance.LOW, - COMMIT_INTERVAL_MS_DOC) - .define(ENABLE_METRICS_PUSH_CONFIG, - Type.BOOLEAN, - true, - Importance.LOW, - ENABLE_METRICS_PUSH_DOC) - .define(REPARTITION_PURGE_INTERVAL_MS_CONFIG, - Type.LONG, - DEFAULT_COMMIT_INTERVAL_MS, - atLeast(0), - Importance.LOW, - REPARTITION_PURGE_INTERVAL_MS_DOC) - .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, - Type.LONG, - 9 * 60 * 1000L, - Importance.LOW, - CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC) - .define(DEFAULT_DSL_STORE_CONFIG, - Type.STRING, - DEFAULT_DSL_STORE, - in(ROCKS_DB, IN_MEMORY), - Importance.LOW, - DEFAULT_DSL_STORE_DOC) - .define(DSL_STORE_SUPPLIERS_CLASS_CONFIG, - Type.CLASS, - DSL_STORE_SUPPLIERS_CLASS_DEFAULT, - Importance.LOW, - DSL_STORE_SUPPLIERS_CLASS_DOC) - .define(DEFAULT_CLIENT_SUPPLIER_CONFIG, - Type.CLASS, - DefaultKafkaClientSupplier.class.getName(), - Importance.LOW, - DEFAULT_CLIENT_SUPPLIER_DOC) - .define(METADATA_MAX_AGE_CONFIG, - Type.LONG, - 5 * 60 * 1000L, - atLeast(0), - Importance.LOW, - CommonClientConfigs.METADATA_MAX_AGE_DOC) - .define(METRICS_NUM_SAMPLES_CONFIG, - Type.INT, - 2, - atLeast(1), - Importance.LOW, - CommonClientConfigs.METRICS_NUM_SAMPLES_DOC) - .define(METRIC_REPORTER_CLASSES_CONFIG, - Type.LIST, - "", - Importance.LOW, - CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) - .define(METRICS_RECORDING_LEVEL_CONFIG, - Type.STRING, - Sensor.RecordingLevel.INFO.toString(), - in(Sensor.RecordingLevel.INFO.toString(), Sensor.RecordingLevel.DEBUG.toString(), RecordingLevel.TRACE.toString()), - Importance.LOW, - CommonClientConfigs.METRICS_RECORDING_LEVEL_DOC) - .define(METRICS_SAMPLE_WINDOW_MS_CONFIG, - Type.LONG, - 30000L, - atLeast(0), - Importance.LOW, - CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC) - .define(AUTO_INCLUDE_JMX_REPORTER_CONFIG, - Type.BOOLEAN, - true, - Importance.LOW, - CommonClientConfigs.AUTO_INCLUDE_JMX_REPORTER_DOC) - .define(POLL_MS_CONFIG, - Type.LONG, - 100L, - Importance.LOW, - POLL_MS_DOC) - .define(PROBING_REBALANCE_INTERVAL_MS_CONFIG, - Type.LONG, - 10 * 60 * 1000L, - atLeast(60 * 1000L), - Importance.LOW, - PROBING_REBALANCE_INTERVAL_MS_DOC) - .define(RECEIVE_BUFFER_CONFIG, - Type.INT, - 32 * 1024, - atLeast(CommonClientConfigs.RECEIVE_BUFFER_LOWER_BOUND), - Importance.LOW, - CommonClientConfigs.RECEIVE_BUFFER_DOC) - .define(RECONNECT_BACKOFF_MS_CONFIG, - Type.LONG, - 50L, - atLeast(0L), - Importance.LOW, - CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC) - .define(RECONNECT_BACKOFF_MAX_MS_CONFIG, - Type.LONG, - 1000L, - atLeast(0L), - Importance.LOW, - CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_DOC) - .define(RETRIES_CONFIG, - Type.INT, - 0, - between(0, Integer.MAX_VALUE), - Importance.LOW, - CommonClientConfigs.RETRIES_DOC) - .define(RETRY_BACKOFF_MS_CONFIG, - Type.LONG, - 100L, - atLeast(0L), - Importance.LOW, - CommonClientConfigs.RETRY_BACKOFF_MS_DOC) - .define(REQUEST_TIMEOUT_MS_CONFIG, - Type.INT, - 40 * 1000, - atLeast(0), - Importance.LOW, - CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC) - .define(ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, - Type.CLASS, - null, - Importance.LOW, - ROCKSDB_CONFIG_SETTER_CLASS_DOC) - .define(SEND_BUFFER_CONFIG, - Type.INT, - 128 * 1024, - atLeast(CommonClientConfigs.SEND_BUFFER_LOWER_BOUND), - Importance.LOW, - CommonClientConfigs.SEND_BUFFER_DOC) - .define(STATE_CLEANUP_DELAY_MS_CONFIG, - Type.LONG, - 10 * 60 * 1000L, - Importance.LOW, - STATE_CLEANUP_DELAY_MS_DOC) - .define(UPGRADE_FROM_CONFIG, - Type.STRING, - null, - in(Stream.concat( - Stream.of((String) null), - Arrays.stream(UpgradeFromValues.values()).map(UpgradeFromValues::toString) - ).toArray(String[]::new) - ), - Importance.LOW, - UPGRADE_FROM_DOC) - .define(WINDOWED_INNER_CLASS_SERDE, - Type.STRING, - null, - Importance.LOW, - WINDOWED_INNER_CLASS_SERDE_DOC) - .define(WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, - Type.LONG, - 24 * 60 * 60 * 1000L, - Importance.LOW, - WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC) - .define(WINDOW_SIZE_MS_CONFIG, - Type.LONG, - null, - Importance.LOW, - WINDOW_SIZE_MS_DOC); + // HIGH + + .define(APPLICATION_ID_CONFIG, // required with no default value + Type.STRING, + Importance.HIGH, + APPLICATION_ID_DOC) + .define(BOOTSTRAP_SERVERS_CONFIG, // required with no default value + Type.LIST, + Importance.HIGH, + CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) + .define(NUM_STANDBY_REPLICAS_CONFIG, + Type.INT, + 0, + Importance.HIGH, + NUM_STANDBY_REPLICAS_DOC) + .define(STATE_DIR_CONFIG, + Type.STRING, + System.getProperty("java.io.tmpdir") + File.separator + "kafka-streams", + Importance.HIGH, + STATE_DIR_DOC, + "${java.io.tmpdir}") + + // MEDIUM + + .define(ACCEPTABLE_RECOVERY_LAG_CONFIG, + Type.LONG, + 10_000L, + atLeast(0), + Importance.MEDIUM, + ACCEPTABLE_RECOVERY_LAG_DOC) + .define(CACHE_MAX_BYTES_BUFFERING_CONFIG, + Type.LONG, + 10 * 1024 * 1024L, + atLeast(0), + Importance.MEDIUM, + CACHE_MAX_BYTES_BUFFERING_DOC) + .define(STATESTORE_CACHE_MAX_BYTES_CONFIG, + Type.LONG, + 10 * 1024 * 1024L, + atLeast(0), + Importance.MEDIUM, + STATESTORE_CACHE_MAX_BYTES_DOC) + .define(CLIENT_ID_CONFIG, + Type.STRING, + "", + Importance.MEDIUM, + CLIENT_ID_DOC, + "<application.id>-<random-UUID>") + .define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, + Type.CLASS, + LogAndFailExceptionHandler.class.getName(), + Importance.MEDIUM, + DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC) + .define(DEFAULT_KEY_SERDE_CLASS_CONFIG, + Type.CLASS, + null, + Importance.MEDIUM, + DEFAULT_KEY_SERDE_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC) + .define(DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, + Type.CLASS, + DefaultProductionExceptionHandler.class.getName(), + Importance.MEDIUM, + DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC) + .define(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, + Type.CLASS, + ProcessingLogAndFailExceptionHandler.class.getName(), + Importance.MEDIUM, + PROCESSING_EXCEPTION_HANDLER_CLASS_DOC) + .define(DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, + Type.CLASS, + FailOnInvalidTimestamp.class.getName(), + Importance.MEDIUM, + DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC) + .define(DEFAULT_VALUE_SERDE_CLASS_CONFIG, + Type.CLASS, + null, + Importance.MEDIUM, + DEFAULT_VALUE_SERDE_CLASS_DOC) + .define(MAX_TASK_IDLE_MS_CONFIG, + Type.LONG, + 0L, + Importance.MEDIUM, + MAX_TASK_IDLE_MS_DOC) + .define(MAX_WARMUP_REPLICAS_CONFIG, + Type.INT, + 2, + atLeast(1), + Importance.MEDIUM, + MAX_WARMUP_REPLICAS_DOC) + .define(NUM_STREAM_THREADS_CONFIG, + Type.INT, + 1, + Importance.MEDIUM, + NUM_STREAM_THREADS_DOC) + .define(PROCESSING_GUARANTEE_CONFIG, + Type.STRING, + AT_LEAST_ONCE, + in(AT_LEAST_ONCE, EXACTLY_ONCE, EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2), + Importance.MEDIUM, + PROCESSING_GUARANTEE_DOC) + .define(RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_CONFIG, + Type.INT, + null, + Importance.MEDIUM, + RACK_AWARE_ASSIGNMENT_NON_OVERLAP_COST_DOC) + .define(RACK_AWARE_ASSIGNMENT_STRATEGY_CONFIG, + Type.STRING, + RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, + in(RACK_AWARE_ASSIGNMENT_STRATEGY_NONE, RACK_AWARE_ASSIGNMENT_STRATEGY_MIN_TRAFFIC, RACK_AWARE_ASSIGNMENT_STRATEGY_BALANCE_SUBTOPOLOGY), + Importance.MEDIUM, + RACK_AWARE_ASSIGNMENT_STRATEGY_DOC) + .define(RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, + Type.LIST, + Collections.emptyList(), + atMostOfSize(MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE), + Importance.MEDIUM, + RACK_AWARE_ASSIGNMENT_TAGS_DOC) + .define(RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_CONFIG, + Type.INT, + null, + Importance.MEDIUM, + RACK_AWARE_ASSIGNMENT_TRAFFIC_COST_DOC) + .define(REPLICATION_FACTOR_CONFIG, + Type.INT, + -1, + Importance.MEDIUM, + REPLICATION_FACTOR_DOC) + .define(SECURITY_PROTOCOL_CONFIG, + Type.STRING, + CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL, + ConfigDef.CaseInsensitiveValidString.in(Utils.enumOptions(SecurityProtocol.class)), + Importance.MEDIUM, + CommonClientConfigs.SECURITY_PROTOCOL_DOC) + .define(TASK_TIMEOUT_MS_CONFIG, + Type.LONG, + Duration.ofMinutes(5L).toMillis(), + atLeast(0L), + Importance.MEDIUM, + TASK_TIMEOUT_MS_DOC) + .define(TOPOLOGY_OPTIMIZATION_CONFIG, + Type.STRING, + NO_OPTIMIZATION, + (name, value) -> verifyTopologyOptimizationConfigs((String) value), + Importance.MEDIUM, + TOPOLOGY_OPTIMIZATION_DOC) + + // LOW + + .define(APPLICATION_SERVER_CONFIG, + Type.STRING, + "", + Importance.LOW, + APPLICATION_SERVER_DOC) + .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, + Type.INT, + 1000, + Importance.LOW, + BUFFERED_RECORDS_PER_PARTITION_DOC) + .define(BUILT_IN_METRICS_VERSION_CONFIG, + Type.STRING, + METRICS_LATEST, + in( + METRICS_LATEST + ), + Importance.LOW, + BUILT_IN_METRICS_VERSION_DOC) + .define(COMMIT_INTERVAL_MS_CONFIG, + Type.LONG, + DEFAULT_COMMIT_INTERVAL_MS, + atLeast(0), + Importance.LOW, + COMMIT_INTERVAL_MS_DOC) + .define(ENABLE_METRICS_PUSH_CONFIG, + Type.BOOLEAN, + true, + Importance.LOW, + ENABLE_METRICS_PUSH_DOC) + .define(REPARTITION_PURGE_INTERVAL_MS_CONFIG, + Type.LONG, + DEFAULT_COMMIT_INTERVAL_MS, + atLeast(0), + Importance.LOW, + REPARTITION_PURGE_INTERVAL_MS_DOC) + .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, + Type.LONG, + 9 * 60 * 1000L, + Importance.LOW, + CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC) + .define(DEFAULT_DSL_STORE_CONFIG, + Type.STRING, + DEFAULT_DSL_STORE, + in(ROCKS_DB, IN_MEMORY), + Importance.LOW, + DEFAULT_DSL_STORE_DOC) + .define(DSL_STORE_SUPPLIERS_CLASS_CONFIG, + Type.CLASS, + DSL_STORE_SUPPLIERS_CLASS_DEFAULT, + Importance.LOW, + DSL_STORE_SUPPLIERS_CLASS_DOC) + .define(DEFAULT_CLIENT_SUPPLIER_CONFIG, + Type.CLASS, + DefaultKafkaClientSupplier.class.getName(), + Importance.LOW, + DEFAULT_CLIENT_SUPPLIER_DOC) + .define(METADATA_MAX_AGE_CONFIG, + Type.LONG, + 5 * 60 * 1000L, + atLeast(0), + Importance.LOW, + CommonClientConfigs.METADATA_MAX_AGE_DOC) + .define(METRICS_NUM_SAMPLES_CONFIG, + Type.INT, + 2, + atLeast(1), + Importance.LOW, + CommonClientConfigs.METRICS_NUM_SAMPLES_DOC) + .define(METRIC_REPORTER_CLASSES_CONFIG, + Type.LIST, + "", + Importance.LOW, + CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) + .define(METRICS_RECORDING_LEVEL_CONFIG, + Type.STRING, + Sensor.RecordingLevel.INFO.toString(), + in(Sensor.RecordingLevel.INFO.toString(), Sensor.RecordingLevel.DEBUG.toString(), RecordingLevel.TRACE.toString()), + Importance.LOW, + CommonClientConfigs.METRICS_RECORDING_LEVEL_DOC) + .define(METRICS_SAMPLE_WINDOW_MS_CONFIG, + Type.LONG, + 30000L, + atLeast(0), + Importance.LOW, + CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC) + .define(AUTO_INCLUDE_JMX_REPORTER_CONFIG, + Type.BOOLEAN, + true, + Importance.LOW, + CommonClientConfigs.AUTO_INCLUDE_JMX_REPORTER_DOC) + .define(POLL_MS_CONFIG, + Type.LONG, + 100L, + Importance.LOW, + POLL_MS_DOC) + .define(PROBING_REBALANCE_INTERVAL_MS_CONFIG, + Type.LONG, + 10 * 60 * 1000L, + atLeast(60 * 1000L), + Importance.LOW, + PROBING_REBALANCE_INTERVAL_MS_DOC) + .define(RECEIVE_BUFFER_CONFIG, + Type.INT, + 32 * 1024, + atLeast(CommonClientConfigs.RECEIVE_BUFFER_LOWER_BOUND), + Importance.LOW, + CommonClientConfigs.RECEIVE_BUFFER_DOC) + .define(RECONNECT_BACKOFF_MS_CONFIG, + Type.LONG, + 50L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC) + .define(RECONNECT_BACKOFF_MAX_MS_CONFIG, + Type.LONG, + 1000L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_DOC) + .define(RETRIES_CONFIG, + Type.INT, + 0, + between(0, Integer.MAX_VALUE), + Importance.LOW, + CommonClientConfigs.RETRIES_DOC) + .define(RETRY_BACKOFF_MS_CONFIG, + Type.LONG, + 100L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RETRY_BACKOFF_MS_DOC) + .define(REQUEST_TIMEOUT_MS_CONFIG, + Type.INT, + 40 * 1000, + atLeast(0), + Importance.LOW, + CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC) + .define(ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, + Type.CLASS, + null, + Importance.LOW, + ROCKSDB_CONFIG_SETTER_CLASS_DOC) + .define(SEND_BUFFER_CONFIG, + Type.INT, + 128 * 1024, + atLeast(CommonClientConfigs.SEND_BUFFER_LOWER_BOUND), + Importance.LOW, + CommonClientConfigs.SEND_BUFFER_DOC) + .define(STATE_CLEANUP_DELAY_MS_CONFIG, + Type.LONG, + 10 * 60 * 1000L, + Importance.LOW, + STATE_CLEANUP_DELAY_MS_DOC) + .define(UPGRADE_FROM_CONFIG, + Type.STRING, + null, + in(Stream.concat( + Stream.of((String) null), + Arrays.stream(UpgradeFromValues.values()).map(UpgradeFromValues::toString) + ).toArray(String[]::new) + ), + Importance.LOW, + UPGRADE_FROM_DOC) + .define(WINDOWED_INNER_CLASS_SERDE, + Type.STRING, + null, + Importance.LOW, + WINDOWED_INNER_CLASS_SERDE_DOC) + .define(WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, + Type.LONG, + 24 * 60 * 60 * 1000L, + Importance.LOW, + WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC) + .define(WINDOW_SIZE_MS_CONFIG, + Type.LONG, + null, + Importance.LOW, + WINDOW_SIZE_MS_DOC); } // this is the list of configs for underlying clients @@ -1255,7 +1248,7 @@ public static class InternalConfig { // Private API used to control the usage of consistency offset vectors public static final String IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED = "__iq.consistency.offset" - + ".vector.enabled__"; + + ".vector.enabled__"; // Private API used to control the prefix of the auto created topics public static final String TOPIC_PREFIX_ALTERNATIVE = "__internal.override.topic.prefix__"; @@ -1266,7 +1259,7 @@ public static class InternalConfig { public static boolean getStateUpdaterEnabled(final Map configs) { return InternalConfig.getBoolean(configs, InternalConfig.STATE_UPDATER_ENABLED, false); } - + // Private API to enable processing threads (i.e. polling is decoupled from processing) public static final String PROCESSING_THREADS_ENABLED = "__processing.threads.enabled__"; @@ -1431,12 +1424,12 @@ protected StreamsConfig(final Map props, final String processingModeConfig = getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); if (processingModeConfig.equals(EXACTLY_ONCE)) { log.warn("Configuration parameter `{}` is deprecated and will be removed in the 4.0.0 release. " + - "Please use `{}` instead. Note that this requires broker version 2.5+ so you should prepare " - + "to upgrade your brokers if necessary.", EXACTLY_ONCE, EXACTLY_ONCE_V2); + "Please use `{}` instead. Note that this requires broker version 2.5+ so you should prepare " + + "to upgrade your brokers if necessary.", EXACTLY_ONCE, EXACTLY_ONCE_V2); } if (processingModeConfig.equals(EXACTLY_ONCE_BETA)) { log.warn("Configuration parameter `{}` is deprecated and will be removed in the 4.0.0 release. " + - "Please use `{}` instead.", EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2); + "Please use `{}` instead.", EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2); } if (props.containsKey(RETRIES_CONFIG)) { @@ -1454,25 +1447,25 @@ private void verifyEOSTransactionTimeoutCompatibility() { final String transactionTimeoutConfigKey = producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG); final int transactionTimeout = originals().containsKey(transactionTimeoutConfigKey) ? - (int) Objects.requireNonNull( - parseType(transactionTimeoutConfigKey, originals().get(transactionTimeoutConfigKey), Type.INT), - "Could not parse config `" + COMMIT_INTERVAL_MS_CONFIG + "` because it's set to `null`") : - DEFAULT_TRANSACTION_TIMEOUT; + (int) Objects.requireNonNull( + parseType(transactionTimeoutConfigKey, originals().get(transactionTimeoutConfigKey), Type.INT), + "Could not parse config `" + COMMIT_INTERVAL_MS_CONFIG + "` because it's set to `null`") : + DEFAULT_TRANSACTION_TIMEOUT; if (transactionTimeout < commitInterval) { throw new IllegalArgumentException(String.format("Transaction timeout %d was set lower than " + - "streams commit interval %d. This will cause ongoing transaction always timeout due to inactivity " + - "caused by long commit interval. Consider reconfiguring commit interval to match " + - "transaction timeout by tuning 'commit.interval.ms' config, or increase the transaction timeout to match " + - "commit interval by tuning `producer.transaction.timeout.ms` config.", - transactionTimeout, commitInterval)); + "streams commit interval %d. This will cause ongoing transaction always timeout due to inactivity " + + "caused by long commit interval. Consider reconfiguring commit interval to match " + + "transaction timeout by tuning 'commit.interval.ms' config, or increase the transaction timeout to match " + + "commit interval by tuning `producer.transaction.timeout.ms` config.", + transactionTimeout, commitInterval)); } } @Override protected Map postProcessParsedConfig(final Map parsedValues) { final Map configUpdates = - CommonClientConfigs.postProcessReconnectBackoffConfigs(this, parsedValues); + CommonClientConfigs.postProcessReconnectBackoffConfigs(this, parsedValues); if (StreamsConfigUtils.eosEnabled(this) && !originals().containsKey(COMMIT_INTERVAL_MS_CONFIG)) { log.debug("Using {} default value of {} as exactly once is enabled.", @@ -1491,28 +1484,28 @@ private void validateRackAwarenessConfiguration() { if (clientTags.size() > MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE) { throw new ConfigException("At most " + MAX_RACK_AWARE_ASSIGNMENT_TAG_LIST_SIZE + " client tags " + - "can be specified using " + CLIENT_TAG_PREFIX + " prefix."); + "can be specified using " + CLIENT_TAG_PREFIX + " prefix."); } for (final String rackAwareAssignmentTag : rackAwareAssignmentTags) { if (!clientTags.containsKey(rackAwareAssignmentTag)) { throw new ConfigException(RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, - rackAwareAssignmentTags, - "Contains invalid value [" + rackAwareAssignmentTag + "] " + - "which doesn't have corresponding tag set via [" + CLIENT_TAG_PREFIX + "] prefix."); + rackAwareAssignmentTags, + "Contains invalid value [" + rackAwareAssignmentTag + "] " + + "which doesn't have corresponding tag set via [" + CLIENT_TAG_PREFIX + "] prefix."); } } clientTags.forEach((tagKey, tagValue) -> { if (tagKey.length() > MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH) { throw new ConfigException(CLIENT_TAG_PREFIX, - tagKey, - "Tag key exceeds maximum length of " + MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH + "."); + tagKey, + "Tag key exceeds maximum length of " + MAX_RACK_AWARE_ASSIGNMENT_TAG_KEY_LENGTH + "."); } if (tagValue.length() > MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH) { throw new ConfigException(CLIENT_TAG_PREFIX, - tagValue, - "Tag value exceeds maximum length of " + MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH + "."); + tagValue, + "Tag value exceeds maximum length of " + MAX_RACK_AWARE_ASSIGNMENT_TAG_VALUE_LENGTH + "."); } }); } @@ -1570,7 +1563,7 @@ private void checkIfUnexpectedUserSpecifiedConsumerConfig(final Map-")); + "producer", config, eosMessage, clientProvidedProps.get(config), "-")); clientProvidedProps.remove(config); } } @@ -1661,7 +1654,7 @@ public Map getMainConsumerConfigs(final String groupId, final St final Map producerProps = getClientPropsWithPrefix(PRODUCER_PREFIX, ProducerConfig.configNames()); if (topicProps.containsKey(topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG)) && - producerProps.containsKey(ProducerConfig.BATCH_SIZE_CONFIG)) { + producerProps.containsKey(ProducerConfig.BATCH_SIZE_CONFIG)) { final int segmentSize = Integer.parseInt(topicProps.get(topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG)).toString()); final int batchSize = Integer.parseInt(producerProps.get(ProducerConfig.BATCH_SIZE_CONFIG).toString()); @@ -1802,10 +1795,10 @@ public Map getAdminConfigs(final String clientId) { @SuppressWarnings("WeakerAccess") public Map getClientTags() { return originalsWithPrefix(CLIENT_TAG_PREFIX).entrySet().stream().collect( - Collectors.toMap( - Map.Entry::getKey, - tagEntry -> Objects.toString(tagEntry.getValue()) - ) + Collectors.toMap( + Map.Entry::getKey, + tagEntry -> Objects.toString(tagEntry.getValue()) + ) ); } @@ -1866,7 +1859,7 @@ public static Set verifyTopologyOptimizationConfigs(final String config) */ public KafkaClientSupplier getKafkaClientSupplier() { return getConfiguredInstance(StreamsConfig.DEFAULT_CLIENT_SUPPLIER_CONFIG, - KafkaClientSupplier.class); + KafkaClientSupplier.class); } /** @@ -1887,7 +1880,7 @@ public Serde defaultKeySerde() { return serde; } catch (final Exception e) { throw new StreamsException( - String.format("Failed to configure key serde %s", keySerdeConfigSetting), e); + String.format("Failed to configure key serde %s", keySerdeConfigSetting), e); } } @@ -1909,7 +1902,7 @@ public Serde defaultValueSerde() { return serde; } catch (final Exception e) { throw new StreamsException( - String.format("Failed to configure value serde %s", valueSerdeConfigSetting), e); + String.format("Failed to configure value serde %s", valueSerdeConfigSetting), e); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java index 9bb09c4eb3f78..36de11c43358e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java @@ -26,11 +26,11 @@ import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.internals.MaterializedInternal; import org.apache.kafka.streams.processor.TimestampExtractor; + import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper; import org.apache.kafka.streams.state.DslStoreSuppliers; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.util.Optional; import java.util.Properties; import java.util.function.Supplier; @@ -40,23 +40,23 @@ import static org.apache.kafka.streams.StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_DOC; import static org.apache.kafka.streams.StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG; import static org.apache.kafka.streams.StreamsConfig.CACHE_MAX_BYTES_BUFFERING_DOC; +import static org.apache.kafka.streams.StreamsConfig.DSL_STORE_SUPPLIERS_CLASS_CONFIG; +import static org.apache.kafka.streams.StreamsConfig.DSL_STORE_SUPPLIERS_CLASS_DEFAULT; +import static org.apache.kafka.streams.StreamsConfig.DSL_STORE_SUPPLIERS_CLASS_DOC; +import static org.apache.kafka.streams.StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG; +import static org.apache.kafka.streams.StreamsConfig.STATESTORE_CACHE_MAX_BYTES_DOC; import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG; import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_DOC; -import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_CONFIG; -import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_DOC; import static org.apache.kafka.streams.StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG; import static org.apache.kafka.streams.StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_DOC; -import static org.apache.kafka.streams.StreamsConfig.DSL_STORE_SUPPLIERS_CLASS_CONFIG; -import static org.apache.kafka.streams.StreamsConfig.DSL_STORE_SUPPLIERS_CLASS_DEFAULT; -import static org.apache.kafka.streams.StreamsConfig.DSL_STORE_SUPPLIERS_CLASS_DOC; -import static org.apache.kafka.streams.StreamsConfig.IN_MEMORY; import static org.apache.kafka.streams.StreamsConfig.MAX_TASK_IDLE_MS_CONFIG; import static org.apache.kafka.streams.StreamsConfig.MAX_TASK_IDLE_MS_DOC; -import static org.apache.kafka.streams.StreamsConfig.ROCKS_DB; -import static org.apache.kafka.streams.StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG; -import static org.apache.kafka.streams.StreamsConfig.STATESTORE_CACHE_MAX_BYTES_DOC; import static org.apache.kafka.streams.StreamsConfig.TASK_TIMEOUT_MS_CONFIG; import static org.apache.kafka.streams.StreamsConfig.TASK_TIMEOUT_MS_DOC; +import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_CONFIG; +import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_DOC; +import static org.apache.kafka.streams.StreamsConfig.ROCKS_DB; +import static org.apache.kafka.streams.StreamsConfig.IN_MEMORY; import static org.apache.kafka.streams.internals.StreamsConfigUtils.getTotalCacheSize; /** @@ -306,4 +306,4 @@ private TaskConfig(final long maxTaskIdleMs, this.eosEnabled = eosEnabled; } } -} +} \ No newline at end of file From 655f274392ce2c9abd85b6bbf6f389089ffe5a3f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Wed, 22 May 2024 11:52:38 +0200 Subject: [PATCH 19/28] KAFKA-16448: Update from trunk --- .../src/main/java/org/apache/kafka/streams/StreamsConfig.java | 2 +- .../src/main/java/org/apache/kafka/streams/TopologyConfig.java | 2 +- .../kafka/streams/processor/internals/CorruptedRecord.java | 3 +-- .../kafka/streams/processor/internals/StampedRecord.java | 1 + 4 files changed, 4 insertions(+), 4 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 908b33af303b8..24c3d820e92da 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -1950,4 +1950,4 @@ private Map clientProps(final Set configNames, public static void main(final String[] args) { System.out.println(CONFIG.toHtml(4, config -> "streamsconfigs_" + config)); } -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java index 36de11c43358e..2d4157eba7885 100644 --- a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java @@ -306,4 +306,4 @@ private TaskConfig(final long maxTaskIdleMs, this.eosEnabled = eosEnabled; } } -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java index d31a29883cabe..4a7646acb02d3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/CorruptedRecord.java @@ -16,9 +16,8 @@ */ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.clients.consumer.ConsumerRecord; - import java.util.Objects; +import org.apache.kafka.clients.consumer.ConsumerRecord; /** * This class represents a version of a {@link StampedRecord} that failed to deserialize. We need diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java index d50fbee68260f..71e3ca2e3ceca 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.header.Headers; public class StampedRecord extends Stamped> { + public StampedRecord(final ConsumerRecord record, final long timestamp) { super(record, timestamp); } From 81b80f41383dabf18ea6e71fced684c51d8ca3bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Wed, 22 May 2024 12:01:23 +0200 Subject: [PATCH 20/28] KAFKA-16448: Fix bad indent --- .../main/java/org/apache/kafka/streams/StreamsConfig.java | 8 +++++++- .../kafka/streams/processor/internals/ProcessorNode.java | 5 ++++- .../kafka/streams/processor/internals/SourceNode.java | 1 + 3 files changed, 12 insertions(+), 2 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 24c3d820e92da..5685fc4b40720 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -36,7 +36,13 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.errors.*; +import org.apache.kafka.streams.errors.DefaultProductionExceptionHandler; +import org.apache.kafka.streams.errors.DeserializationExceptionHandler; +import org.apache.kafka.streams.errors.LogAndFailExceptionHandler; +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; +import org.apache.kafka.streams.errors.ProcessingLogAndFailExceptionHandler; +import org.apache.kafka.streams.errors.ProductionExceptionHandler; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.internals.StreamsConfigUtils; import org.apache.kafka.streams.internals.UpgradeFromValues; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; 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 7a8fff9dc6986..aa57c805881f9 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 @@ -53,6 +53,7 @@ public ProcessorNode(final String name) { public ProcessorNode(final String name, final Processor processor, final Set stateStores) { + this.name = name; this.processor = processor; this.fixedKeyProcessor = null; @@ -64,6 +65,7 @@ public ProcessorNode(final String name, public ProcessorNode(final String name, final FixedKeyProcessor processor, final Set stateStores) { + this.name = name; this.processor = null; this.fixedKeyProcessor = processor; @@ -141,6 +143,7 @@ protected void throwIfClosed() { } } + public void process(final Record record) { throwIfClosed(); @@ -206,4 +209,4 @@ public String toString(final String indent) { } return sb.toString(); } -} +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java index 81098d8a1b314..5d0c04b96a89f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java @@ -78,6 +78,7 @@ public void init(final InternalProcessorContext context) { valDeserializer = prepareValueDeserializer(valDeserializer, context, name()); } + @Override public void process(final Record record) { context.forward(record); From 99ae6749f73c1bdef0ae71c59c8b679406e65d8c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Wed, 22 May 2024 12:11:13 +0200 Subject: [PATCH 21/28] KAFKA-16448: Fix bad indent --- .../internals/AbstractPartitionGroup.java | 5 +- .../processor/internals/ProcessorNode.java | 2 +- .../internals/RecordCollectorImpl.java | 78 +++++++++---------- .../internals/RecordDeserializer.java | 64 +++++++-------- .../processor/internals/RecordQueue.java | 5 +- 5 files changed, 77 insertions(+), 77 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractPartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractPartitionGroup.java index 808e4db9544c7..7183526d27ca0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractPartitionGroup.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractPartitionGroup.java @@ -16,11 +16,10 @@ */ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.TopicPartition; - import java.util.Set; import java.util.function.Function; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; abstract class AbstractPartitionGroup { 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 aa57c805881f9..fbd976cb11274 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 @@ -209,4 +209,4 @@ public String toString(final String indent) { } return sb.toString(); } -} \ No newline at end of file +} 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 64bf7dd133272..74e1413d2bf38 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 @@ -97,14 +97,14 @@ public RecordCollectorImpl(final LogContext logContext, for (final String topic : topology.sinkTopics()) { final String processorNodeId = topology.sink(topic).name(); producedSensorByTopic.put( + topic, + TopicMetrics.producedSensor( + threadId, + taskId.toString(), + processorNodeId, topic, - TopicMetrics.producedSensor( - threadId, - taskId.toString(), - processorNodeId, - topic, - streamsMetrics - )); + streamsMetrics + )); } this.offsets = new HashMap<>(); @@ -146,8 +146,8 @@ public void send(final String topic, // here we cannot drop the message on the floor even if it is a transient timeout exception, // so we treat everything the same as a fatal exception throw new StreamsException("Could not determine the number of partitions for topic '" + topic + - "' for task " + taskId + " due to " + fatal, - fatal + "' for task " + taskId + " due to " + fatal, + fatal ); } if (partitions.size() > 0) { @@ -170,7 +170,7 @@ public void send(final String topic, } } else { throw new StreamsException("Could not get partition information for topic " + topic + " for task " + taskId + - ". This can happen if the topic does not exist."); + ". This can happen if the topic does not exist."); } } else { send(topic, key, value, headers, null, timestamp, keySerializer, valueSerializer, processorNodeId, context); @@ -230,14 +230,14 @@ public void send(final String topic, // 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, + t -> TopicMetrics.producedSensor( + Thread.currentThread().getName(), + taskId.toString(), + processorNodeId, topic, - t -> TopicMetrics.producedSensor( - Thread.currentThread().getName(), - taskId.toString(), - processorNodeId, - topic, - context.metrics() - ) + context.metrics() + ) ); final long bytesProduced = producerRecordSizeInBytes(serializedRecord); topicProducedSensor.record(bytesProduced, context.currentSystemTimeMs()); @@ -341,26 +341,26 @@ private void recordSendError(final String topic, final Exception exception, fina exception instanceof InvalidProducerEpochException || exception instanceof OutOfOrderSequenceException) { errorMessage += "\nWritten offsets would not be recorded and no more records would be sent since the producer is fenced, " + - "indicating the task may be migrated out"; + "indicating the task may be migrated out"; sendException.set(new TaskMigratedException(errorMessage, exception)); } else { if (exception instanceof RetriableException) { 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"; + "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 { final ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( - null, - serializedRecord.topic(), - serializedRecord.partition(), - context != null ? context.offset() : -1, - serializedRecord.headers(), - null, - null, - processorNodeId, - taskId); + null, + serializedRecord.topic(), + serializedRecord.partition(), + context != null ? context.offset() : -1, + serializedRecord.headers(), + null, + null, + processorNodeId, + taskId); if (productionExceptionHandler.handle(errorHandlerContext, serializedRecord, exception) == ProductionExceptionHandlerResponse.FAIL) { errorMessage += "\nException handler choose to FAIL the processing, no more records would be sent."; @@ -377,20 +377,20 @@ private void recordSendError(final String topic, final Exception exception, fina private boolean isFatalException(final Exception exception) { final boolean securityException = exception instanceof AuthenticationException || - exception instanceof AuthorizationException || - exception instanceof SecurityDisabledException; + exception instanceof AuthorizationException || + exception instanceof SecurityDisabledException; final boolean communicationException = exception instanceof InvalidTopicException || - exception instanceof UnknownServerException || - exception instanceof SerializationException || - exception instanceof OffsetMetadataTooLarge || - exception instanceof IllegalStateException; + exception instanceof UnknownServerException || + exception instanceof SerializationException || + exception instanceof OffsetMetadataTooLarge || + exception instanceof IllegalStateException; return securityException || communicationException; } /** - * @throws StreamsException fatal error that should cause the thread to die + * @throws StreamsException fatal error that should cause the thread to die * @throws TaskMigratedException recoverable error that would cause the task to be removed */ @Override @@ -401,7 +401,7 @@ public void flush() { } /** - * @throws StreamsException fatal error that should cause the thread to die + * @throws StreamsException fatal error that should cause the thread to die * @throws TaskMigratedException recoverable error that would cause the task to be removed */ @Override @@ -418,7 +418,7 @@ public void closeClean() { } /** - * @throws StreamsException fatal error that should cause the thread to die + * @throws StreamsException fatal error that should cause the thread to die * @throws TaskMigratedException recoverable error that would cause the task to be removed */ @Override 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 4ba183a68e664..9068d8ac4afde 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 @@ -55,17 +55,17 @@ ConsumerRecord deserialize(final ProcessorContext processo try { return new ConsumerRecord<>( - rawRecord.topic(), - rawRecord.partition(), - rawRecord.offset(), - rawRecord.timestamp(), - TimestampType.CREATE_TIME, - rawRecord.serializedKeySize(), - rawRecord.serializedValueSize(), - sourceNode.deserializeKey(rawRecord.topic(), rawRecord.headers(), rawRecord.key()), - sourceNode.deserializeValue(rawRecord.topic(), rawRecord.headers(), rawRecord.value()), - rawRecord.headers(), - Optional.empty() + rawRecord.topic(), + rawRecord.partition(), + rawRecord.offset(), + rawRecord.timestamp(), + TimestampType.CREATE_TIME, + rawRecord.serializedKeySize(), + rawRecord.serializedValueSize(), + sourceNode.deserializeKey(rawRecord.topic(), rawRecord.headers(), rawRecord.key()), + sourceNode.deserializeValue(rawRecord.topic(), rawRecord.headers(), rawRecord.value()), + rawRecord.headers(), + Optional.empty() ); } catch (final Exception deserializationException) { handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor); @@ -82,36 +82,36 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa final DeserializationExceptionHandler.DeserializationHandlerResponse response; try { final ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( - (InternalProcessorContext) processorContext, - rawRecord.topic(), - rawRecord.partition(), - rawRecord.offset(), - rawRecord.headers(), - rawRecord.key(), - rawRecord.value(), - null, - processorContext.taskId()); + (InternalProcessorContext) processorContext, + rawRecord.topic(), + rawRecord.partition(), + rawRecord.offset(), + rawRecord.headers(), + rawRecord.key(), + rawRecord.value(), + null, + processorContext.taskId()); response = deserializationExceptionHandler.handle(errorHandlerContext, rawRecord, deserializationException); } catch (final Exception fatalUserException) { log.error( - "Deserialization error callback failed after deserialization error for record {}", - rawRecord, - deserializationException); + "Deserialization error callback failed after deserialization error for record {}", + rawRecord, + deserializationException); throw new StreamsException("Fatal user code error in deserialization error callback", fatalUserException); } if (response == DeserializationExceptionHandler.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 " + - DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.", - deserializationException); + " a deserialization error. If you would rather have the streaming pipeline" + + " continue after a deserialization error, please set the " + + DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.", + deserializationException); } else { log.warn( - "Skipping record due to deserialization error. topic=[{}] partition=[{}] offset=[{}]", - rawRecord.topic(), - rawRecord.partition(), - rawRecord.offset(), - deserializationException + "Skipping record due to deserialization error. topic=[{}] partition=[{}] offset=[{}]", + rawRecord.topic(), + rawRecord.partition(), + rawRecord.offset(), + deserializationException ); droppedRecordsSensor.record(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java index 9ccdc1d89b98b..b57028d635ba8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -22,10 +22,11 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.TimestampExtractor; -import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.processor.internals.metrics.TopicMetrics; + import org.slf4j.Logger; import java.util.ArrayDeque; From a22f98358c0abce4b46ae4a3bcf04ca9d479a59b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Wed, 22 May 2024 12:16:47 +0200 Subject: [PATCH 22/28] KAFKA-16448: Fix bad indent --- .../errors/ProductionExceptionHandler.java | 14 +-- .../internals/RecordCollectorImpl.java | 2 +- .../processor/internals/StreamTask.java | 4 +- .../internals/SynchronizedPartitionGroup.java | 5 +- .../internals/RecordDeserializerTest.java | 100 +++++++++--------- 5 files changed, 62 insertions(+), 63 deletions(-) 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 c1b05ccc2755d..71b160b5d2d1c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/ProductionExceptionHandler.java @@ -29,7 +29,7 @@ public interface ProductionExceptionHandler extends Configurable { * Inspect a record that we attempted to produce, and the exception that resulted * from attempting to produce it and determine whether or not to continue processing. * - * @param record The record that failed to produce + * @param record The record that failed to produce * @param exception The exception that occurred during production * @deprecated Use {@link #handle(ErrorHandlerContext, ProducerRecord, Exception)} instead */ @@ -43,8 +43,8 @@ default ProductionExceptionHandlerResponse handle(final ProducerRecord rawRecord = new ConsumerRecord<>("topic", - 1, - 1, - 10, - TimestampType.LOG_APPEND_TIME, - 3, - 5, - new byte[0], - new byte[0], - headers, - Optional.empty()); + 1, + 1, + 10, + TimestampType.LOG_APPEND_TIME, + 3, + 5, + new byte[0], + new byte[0], + headers, + Optional.empty()); private final InternalProcessorContext context = new InternalMockProcessorContext<>(); @Test public void shouldReturnConsumerRecordWithDeserializedValueWhenNoExceptions() { final RecordDeserializer recordDeserializer = new RecordDeserializer( - new TheSourceNode( - false, - false, - "key", "value" - ), - null, - new LogContext(), - new Metrics().sensor("dropped-records") + new TheSourceNode( + false, + false, + "key", "value" + ), + null, + new LogContext(), + new Metrics().sensor("dropped-records") ); final ConsumerRecord record = recordDeserializer.deserialize(null, rawRecord); assertEquals(rawRecord.topic(), record.topic()); @@ -80,14 +80,14 @@ public void shouldReturnConsumerRecordWithDeserializedValueWhenNoExceptions() { @Test public void shouldThrowExceptionWithKeyDeserializationAndFail() { final RecordDeserializer recordDeserializer = new RecordDeserializer( - new TheSourceNode( - true, - false, - "key", "value" - ), - new LogAndFailExceptionHandler(), - new LogContext(), - new Metrics().sensor("dropped-records") + new TheSourceNode( + true, + false, + "key", "value" + ), + new LogAndFailExceptionHandler(), + new LogContext(), + new Metrics().sensor("dropped-records") ); final StreamsException e = assertThrows(StreamsException.class, () -> recordDeserializer.deserialize(context, rawRecord)); assertEquals(e.getMessage(), "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 default.deserialization.exception.handler appropriately."); @@ -96,14 +96,14 @@ public void shouldThrowExceptionWithKeyDeserializationAndFail() { @Test public void shouldThrowExceptionWithValueDeserializationAndFail() { final RecordDeserializer recordDeserializer = new RecordDeserializer( - new TheSourceNode( - false, - true, - "key", "value" - ), - new LogAndFailExceptionHandler(), - new LogContext(), - new Metrics().sensor("dropped-records") + new TheSourceNode( + false, + true, + "key", "value" + ), + new LogAndFailExceptionHandler(), + new LogContext(), + new Metrics().sensor("dropped-records") ); final StreamsException e = assertThrows(StreamsException.class, () -> recordDeserializer.deserialize(context, rawRecord)); assertEquals(e.getMessage(), "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 default.deserialization.exception.handler appropriately."); @@ -112,14 +112,14 @@ public void shouldThrowExceptionWithValueDeserializationAndFail() { @Test public void shouldThrowExceptionWithKeyDeserializationAndContinue() { final RecordDeserializer recordDeserializer = new RecordDeserializer( - new TheSourceNode( - true, - false, - "key", "value" - ), - new LogAndContinueExceptionHandler(), - new LogContext(), - new Metrics().sensor("dropped-records") + new TheSourceNode( + true, + false, + "key", "value" + ), + new LogAndContinueExceptionHandler(), + new LogContext(), + new Metrics().sensor("dropped-records") ); final ConsumerRecord record = recordDeserializer.deserialize(context, rawRecord); assertNull(record); @@ -128,14 +128,14 @@ public void shouldThrowExceptionWithKeyDeserializationAndContinue() { @Test public void shouldThrowExceptionWithValueDeserializationAndContinue() { final RecordDeserializer recordDeserializer = new RecordDeserializer( - new TheSourceNode( - false, - true, - "key", "value" - ), - new LogAndContinueExceptionHandler(), - new LogContext(), - new Metrics().sensor("dropped-records") + new TheSourceNode( + false, + true, + "key", "value" + ), + new LogAndContinueExceptionHandler(), + new LogContext(), + new Metrics().sensor("dropped-records") ); final ConsumerRecord record = recordDeserializer.deserialize(context, rawRecord); assertNull(record); From 50fa22dd108a91a60ec875c7a23ce10018a377bf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Fri, 24 May 2024 02:12:01 +0200 Subject: [PATCH 23/28] KAFKA-16448: Handle processing exception on punctuate --- .../apache/kafka/streams/TopologyConfig.java | 13 ++++++++++--- .../internals/ErrorHandlerContextImpl.java | 11 +++++++++++ .../internals/ProcessorContextImpl.java | 2 +- .../streams/processor/internals/StreamTask.java | 17 ++++++++++++++++- 4 files changed, 38 insertions(+), 5 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java index 2d4157eba7885..4687b9bd9afe3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java @@ -22,15 +22,16 @@ import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.internals.StreamsConfigUtils; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.internals.MaterializedInternal; import org.apache.kafka.streams.processor.TimestampExtractor; - import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper; import org.apache.kafka.streams.state.DslStoreSuppliers; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import java.util.Optional; import java.util.Properties; import java.util.function.Supplier; @@ -135,6 +136,7 @@ public class TopologyConfig extends AbstractConfig { public final Class dslStoreSuppliers; public final Supplier timestampExtractorSupplier; public final Supplier deserializationExceptionHandlerSupplier; + public final Supplier processingExceptionHandler; public TopologyConfig(final StreamsConfig globalAppConfigs) { this(null, globalAppConfigs, new Properties()); @@ -150,6 +152,7 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo this.applicationConfigs = globalAppConfigs; this.topologyOverrides = topologyOverrides; + this.processingExceptionHandler = () -> globalAppConfigs.getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class); if (isTopologyOverride(BUFFERED_RECORDS_PER_PARTITION_CONFIG, topologyOverrides)) { maxBufferedSize = getInt(BUFFERED_RECORDS_PER_PARTITION_CONFIG); @@ -280,7 +283,8 @@ public TaskConfig getTaskConfig() { maxBufferedSize, timestampExtractorSupplier.get(), deserializationExceptionHandlerSupplier.get(), - eosEnabled + eosEnabled, + processingExceptionHandler.get() ); } @@ -291,19 +295,22 @@ public static class TaskConfig { public final TimestampExtractor timestampExtractor; public final DeserializationExceptionHandler deserializationExceptionHandler; public final boolean eosEnabled; + public final ProcessingExceptionHandler processingExceptionHandler; private TaskConfig(final long maxTaskIdleMs, final long taskTimeoutMs, final int maxBufferedSize, final TimestampExtractor timestampExtractor, final DeserializationExceptionHandler deserializationExceptionHandler, - final boolean eosEnabled) { + final boolean eosEnabled, + final ProcessingExceptionHandler processingExceptionHandler) { this.maxTaskIdleMs = maxTaskIdleMs; this.taskTimeoutMs = taskTimeoutMs; this.maxBufferedSize = maxBufferedSize; this.timestampExtractor = timestampExtractor; this.deserializationExceptionHandler = deserializationExceptionHandler; this.eosEnabled = eosEnabled; + this.processingExceptionHandler = processingExceptionHandler; } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java index a67a58482e8d7..e5debd5977fd7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java @@ -34,6 +34,17 @@ public class ErrorHandlerContextImpl implements ErrorHandlerContext { private final String processorNodeId; private final TaskId taskId; + public ErrorHandlerContextImpl(final String topic, + final int partition, + final long offset, + final Headers headers, + final byte[] sourceRawKey, + final byte[] sourceRawValue, + final String processorNodeId, + final TaskId taskId) { + this(null, topic, partition, offset, headers, sourceRawKey, sourceRawValue, processorNodeId, taskId); + } + public ErrorHandlerContextImpl(final InternalProcessorContext processorContext, final String topic, final int partition, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index 55bf9ea4cd88b..264184da5d32d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -304,7 +304,7 @@ private void forwardInternal(final ProcessorNode child, final byte[] rawKey = streamTask.rawRecord() != null ? streamTask.rawRecord().key() : null; final byte[] rawValue = streamTask.rawRecord() != null ? streamTask.rawRecord().value() : null; - final ErrorHandlerContext errorHandlerContext = new ErrorHandlerContextImpl(null, topic(), + final ErrorHandlerContext errorHandlerContext = new ErrorHandlerContextImpl(topic(), partition(), offset(), headers(), rawKey, rawValue, child.name(), taskId()); final ProcessingExceptionHandler.ProcessingHandlerResponse response = processingExceptionHandler 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 1960b694f4a2b..e7ad7416f2770 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 @@ -29,11 +29,13 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.errors.DeserializationExceptionHandler; import org.apache.kafka.streams.errors.LockException; +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.processor.Cancellable; +import org.apache.kafka.streams.processor.ErrorHandlerContext; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.TaskId; @@ -62,6 +64,7 @@ import static java.util.Collections.singleton; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeRecordSensor; +import static org.apache.kafka.streams.StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; /** @@ -910,7 +913,19 @@ public void punctuate(final ProcessorNode node, } catch (final StreamsException e) { throw e; } catch (final RuntimeException e) { - throw new StreamsException(String.format("%sException caught while punctuating processor '%s'", logPrefix, node.name()), e); + final ErrorHandlerContext errorHandlerContext = new ErrorHandlerContextImpl(recordContext.topic(), + recordContext.partition(), recordContext.offset(), recordContext.headers(), null, null, node.name(), id); + final ProcessingExceptionHandler.ProcessingHandlerResponse response = config.processingExceptionHandler + .handle(errorHandlerContext, null, e); + + if (response == ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) { + throw new StreamsException(String.format("%sException caught while punctuating processor '%s'. " + + "Processing exception handler is set to fail upon" + + " a processing error. If you would rather have the streaming pipeline" + + " continue after a processing error, please set the " + + PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.", logPrefix, node.name()), + e); + } } finally { processorContext.setCurrentNode(null); } From 8f95697ff3c5273381689bb56c61c9ac77204a23 Mon Sep 17 00:00:00 2001 From: Sebastien Viale Date: Fri, 24 May 2024 22:43:57 +0200 Subject: [PATCH 24/28] KAFKA-16448 fix rawRecord and add unit test --- .../apache/kafka/streams/TopologyConfig.java | 1 + .../processor/internals/PartitionGroup.java | 2 +- .../internals/ProcessorContextImplTest.java | 500 +++++++++++++++++- .../testutil/LogAndContinueOnInvalidDsl.java | 37 ++ .../LogAndContinueOnInvalidProcessor.java | 37 ++ .../LogAndContinueOnInvalidPunctuate.java | 36 ++ .../testutil/LogAndFailOnInvalidDsl.java | 37 ++ .../LogAndFailOnInvalidProcessor.java | 38 ++ .../LogAndFailOnInvalidPunctuate.java | 37 ++ 9 files changed, 722 insertions(+), 3 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidDsl.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidProcessor.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidPunctuate.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidDsl.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidProcessor.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidPunctuate.java diff --git a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java index 4687b9bd9afe3..95b6ddd717727 100644 --- a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java @@ -58,6 +58,7 @@ import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_DOC; import static org.apache.kafka.streams.StreamsConfig.ROCKS_DB; import static org.apache.kafka.streams.StreamsConfig.IN_MEMORY; +import static org.apache.kafka.streams.StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG; import static org.apache.kafka.streams.internals.StreamsConfigUtils.getTotalCacheSize; /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java index 37716e51544a9..f52a1feb01785 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java @@ -249,8 +249,8 @@ StampedRecord nextRecord(final RecordInfo info, final long wallClockTime) { if (queue != null) { // get the first record from this queue. - record = queue.poll(wallClockTime); rawHeadRecord = queue.rawHeadRecord(); + record = queue.poll(wallClockTime); if (record != null) { --totalBuffered; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java index 8d44f7f2730f5..1d5d7dfd8b205 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java @@ -21,11 +21,19 @@ 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.IntegerSerializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsConfig.InternalConfig; import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.KeyValueTimestamp; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.TestInputTopic; +import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ErrorHandlerContext; import org.apache.kafka.streams.processor.ProcessorContext; @@ -35,9 +43,14 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.processor.internals.testutil.LogAndContinueOnInvalidProcessor; +import org.apache.kafka.streams.processor.internals.testutil.LogAndContinueOnInvalidPunctuate; +import org.apache.kafka.streams.processor.internals.testutil.LogAndFailOnInvalidProcessor; +import org.apache.kafka.streams.processor.internals.testutil.LogAndFailOnInvalidPunctuate; import org.apache.kafka.streams.query.Position; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; @@ -49,13 +62,17 @@ import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.internals.PositionSerde; import org.apache.kafka.streams.state.internals.ThreadCache; +import org.apache.kafka.test.MockProcessorSupplier; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; +import java.io.PrintWriter; +import java.io.StringWriter; import java.time.Duration; +import java.time.Instant; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -63,6 +80,7 @@ import java.util.List; import java.util.Map; import java.util.function.Consumer; +import java.util.Properties; import static java.util.Arrays.asList; import static org.apache.kafka.common.utils.Utils.mkEntry; @@ -634,7 +652,6 @@ public void configure(final Map configs) { ); final StreamTask task = mock(StreamTask.class); - when(task.streamTime()).thenReturn(STREAM_TIME); when(task.rawRecord()).thenReturn(new ConsumerRecord<>("topic", 0, 0, "key".getBytes(), "value".getBytes())); context.transitionToActive(task, null, null); @@ -691,7 +708,6 @@ public void configure(final Map configs) { ); final StreamTask task = mock(StreamTask.class); - when(task.streamTime()).thenReturn(STREAM_TIME); when(task.rawRecord()).thenReturn(new ConsumerRecord<>("topic", 0, 0, "key".getBytes(), "value".getBytes())); context.transitionToActive(task, null, null); @@ -721,6 +737,486 @@ public void configure(final Map configs) { + "please set the processing.exception.handler appropriately.", exception.getMessage()); } + @Test + public void shouldContinueInProcessorOnProcessingRecordAtBeginningExceptions() { + + final int[] expectedKeys = {0, 1, 2, 10}; + + final StreamsBuilder builder = new StreamsBuilder(); + + final ProcessorSupplier processorSupplier = + () -> new Processor() { + + org.apache.kafka.streams.processor.api.ProcessorContext context; + + @Override + public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { + this.context = context; + } + + @Override + public void process(Record record) { + int value = 100 / (10 * record.value().intValue()); + context.forward(new Record<>(record.key().toString(), Integer.toString(value), record.timestamp())); + } + + @Override + public void close() { + } + }; + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); + stream.process(processorSupplier).process(processor); + + Properties properties = new Properties(); + properties.put("processing.exception.handler", LogAndContinueOnInvalidProcessor.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver( + builder.build(), properties, + Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = + driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); + + for (final int expectedKey : expectedKeys) { + inputTopic.pipeInput(expectedKey, expectedKey, 0); + } + + final KeyValueTimestamp[] expected = { + new KeyValueTimestamp<>("1", "10", 0), + new KeyValueTimestamp<>("2", "5", 0), + new KeyValueTimestamp<>("10", "1", 0), + }; + assertEquals(expected.length, processor.theCapturedProcessor().processed().size()); + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.theCapturedProcessor().processed().get(i)); + } + } + } + + @Test + public void shouldContinueInProcessorOnProcessingRecordInMiddleExceptions() { + + final int[] expectedKeys = {1, 0, 2, 10}; + + final StreamsBuilder builder = new StreamsBuilder(); + + final ProcessorSupplier processorSupplier = + () -> new Processor() { + + org.apache.kafka.streams.processor.api.ProcessorContext context; + + @Override + public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { + this.context = context; + } + + @Override + public void process(Record record) { + int value = 100 / (10 * record.value().intValue()); + context.forward(new Record<>(record.key().toString(), Integer.toString(value), record.timestamp())); + } + + @Override + public void close() { + } + }; + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); + stream.process(processorSupplier).process(processor); + + Properties properties = new Properties(); + properties.put("processing.exception.handler", LogAndContinueOnInvalidProcessor.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver( + builder.build(), properties, + Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = + driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); + + for (final int expectedKey : expectedKeys) { + inputTopic.pipeInput(expectedKey, expectedKey, 0); + } + + final KeyValueTimestamp[] expected = { + new KeyValueTimestamp<>("1", "10", 0), + new KeyValueTimestamp<>("2", "5", 0), + new KeyValueTimestamp<>("10", "1", 0), + }; + assertEquals(expected.length, processor.theCapturedProcessor().processed().size()); + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.theCapturedProcessor().processed().get(i)); + } + } + } + + @Test + public void shouldContinueInProcessorOnProcessingRecordAtEndExceptions() { + + final int[] expectedKeys = {1, 2, 10, 0}; + + final StreamsBuilder builder = new StreamsBuilder(); + + final ProcessorSupplier processorSupplier = + () -> new Processor() { + + org.apache.kafka.streams.processor.api.ProcessorContext context; + + @Override + public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { + this.context = context; + } + + @Override + public void process(Record record) { + int value = 100 / (10 * record.value().intValue()); + context.forward(new Record<>(record.key().toString(), Integer.toString(value), record.timestamp())); + } + + @Override + public void close() { + } + }; + + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); + stream.process(processorSupplier).process(processor); + + Properties properties = new Properties(); + properties.put("processing.exception.handler", LogAndContinueOnInvalidProcessor.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver( + builder.build(), properties, + Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = + driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); + + for (final int expectedKey : expectedKeys) { + inputTopic.pipeInput(expectedKey, expectedKey, 0); + } + + final KeyValueTimestamp[] expected = { + new KeyValueTimestamp<>("1", "10", 0), + new KeyValueTimestamp<>("2", "5", 0), + new KeyValueTimestamp<>("10", "1", 0), + }; + assertEquals(expected.length, processor.theCapturedProcessor().processed().size()); + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.theCapturedProcessor().processed().get(i)); + } + } + } + + @Test + public void shouldFailInProcessorOnProcessingRecordAtBeginningExceptions() { + + final int[] expectedKeys = {0, 1, 2, 10}; + + final StreamsBuilder builder = new StreamsBuilder(); + + final ProcessorSupplier processorSupplier = + () -> new Processor() { + + org.apache.kafka.streams.processor.api.ProcessorContext context; + + @Override + public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { + this.context = context; + } + + @Override + public void process(Record record) { + int value = 100 / (10 * record.value().intValue()); + context.forward(new Record<>(record.key().toString(), Integer.toString(value), record.timestamp())); + } + + @Override + public void close() { + } + }; + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); + stream.process(processorSupplier).process(processor); + + Properties properties = new Properties(); + properties.put("processing.exception.handler", LogAndFailOnInvalidProcessor.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver( + builder.build(), properties, + Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = + driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); + + try { + for (final int expectedKey : expectedKeys) { + inputTopic.pipeInput(expectedKey, expectedKey, 0); + } + } catch (Exception exception) { + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + exception.printStackTrace(pw); + assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); + } + final KeyValueTimestamp[] expected = {}; + assertEquals(0, processor.theCapturedProcessor().processed().size()); + } + } + + @Test + public void shouldFailInProcessorOnProcessingRecordInMiddleExceptions() { + + final int[] expectedKeys = {1, 0, 2, 10}; + + final StreamsBuilder builder = new StreamsBuilder(); + + final ProcessorSupplier processorSupplier = + () -> new Processor() { + + org.apache.kafka.streams.processor.api.ProcessorContext context; + + @Override + public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { + this.context = context; + } + + @Override + public void process(Record record) { + int value = 100 / (10 * record.value().intValue()); + context.forward(new Record<>(record.key().toString(), Integer.toString(value), record.timestamp())); + } + + @Override + public void close() { + } + }; + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); + stream.process(processorSupplier).process(processor); + + Properties properties = new Properties(); + properties.put("processing.exception.handler", LogAndFailOnInvalidProcessor.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver( + builder.build(), properties, + Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = + driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); + + try { + for (final int expectedKey : expectedKeys) { + inputTopic.pipeInput(expectedKey, expectedKey, 0); + } + } catch (Exception exception) { + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + exception.printStackTrace(pw); + assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); + } + final KeyValueTimestamp[] expected = { + new KeyValueTimestamp<>("1", "10", 0), + }; + assertEquals(expected.length, processor.theCapturedProcessor().processed().size()); + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.theCapturedProcessor().processed().get(i)); + } + } + } + + @Test + public void shouldFailInProcessorOnProcessingRecordAtEndExceptions() { + + final int[] expectedKeys = {1, 2, 10, 0}; + + final StreamsBuilder builder = new StreamsBuilder(); + + final ProcessorSupplier processorSupplier = + () -> new Processor() { + + org.apache.kafka.streams.processor.api.ProcessorContext context; + + @Override + public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { + this.context = context; + } + + @Override + public void process(Record record) { + int value = 100 / (10 * record.value().intValue()); + context.forward(new Record<>(record.key().toString(), Integer.toString(value), record.timestamp())); + } + + @Override + public void close() { + } + }; + + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); + stream.process(processorSupplier).process(processor); + + Properties properties = new Properties(); + properties.put("processing.exception.handler", LogAndFailOnInvalidProcessor.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver( + builder.build(), properties, + Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = + driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); + + try { + for (final int expectedKey : expectedKeys) { + inputTopic.pipeInput(expectedKey, expectedKey, 0); + } + } catch (Exception exception) { + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + exception.printStackTrace(pw); + assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); + } + final KeyValueTimestamp[] expected = { + new KeyValueTimestamp<>("1", "10", 0), + new KeyValueTimestamp<>("2", "5", 0), + new KeyValueTimestamp<>("10", "1", 0), + }; + assertEquals(expected.length, processor.theCapturedProcessor().processed().size()); + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.theCapturedProcessor().processed().get(i)); + } + } + } + + @Test + public void shouldContinueOnPunctuateExceptions() { + + final int[] expectedKeys = {1, 2, 10}; + + final StreamsBuilder builder = new StreamsBuilder(); + + final ProcessorSupplier processorSupplier = + () -> new Processor() { + + org.apache.kafka.streams.processor.api.ProcessorContext context; + + @Override + public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { + this.context = context; + this.context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, ts -> { + String zero = "0"; + int i = 1 / Integer.parseInt(zero); + }); + } + + @Override + public void process(Record record) { + context.forward(record); + } + + @Override + public void close() { + } + }; + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); + stream.process(processorSupplier).process(processor); + + Properties properties = new Properties(); + properties.put("processing.exception.handler", LogAndContinueOnInvalidPunctuate.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver( + builder.build(), properties, + Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = + driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); + for (final int expectedKey : expectedKeys) { + driver.advanceWallClockTime(Duration.ofSeconds(2)); + inputTopic.pipeInput(expectedKey, expectedKey, 0); + } + + final KeyValueTimestamp[] expected = { + new KeyValueTimestamp<>(1, 1, 0), + new KeyValueTimestamp<>(2, 2, 0), + new KeyValueTimestamp<>(10, 10, 0), + }; + assertEquals(expected.length, processor.theCapturedProcessor().processed().size()); + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.theCapturedProcessor().processed().get(i)); + } + } + } + + @Test + public void shouldFailOnPunctuateExceptions() { + + final int[] expectedKeys = {0, 1, 2, 10}; + + final StreamsBuilder builder = new StreamsBuilder(); + + final ProcessorSupplier processorSupplier = + () -> new Processor() { + + org.apache.kafka.streams.processor.api.ProcessorContext context; + + @Override + public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { + this.context = context; + this.context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, ts -> { + String zero = "0"; + int i = 1 / Integer.parseInt(zero); + }); + } + + @Override + public void process(Record record) { + context.forward(record); + } + + @Override + public void close() { + } + }; + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); + stream.process(processorSupplier).process(processor); + + Properties properties = new Properties(); + properties.put("processing.exception.handler", LogAndFailOnInvalidPunctuate.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver( + builder.build(), properties, + Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = + driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); + + try { + inputTopic.pipeInput(expectedKeys[0], expectedKeys[0], 0); + inputTopic.pipeInput(expectedKeys[1], expectedKeys[1], 0); + driver.advanceWallClockTime(Duration.ofSeconds(2)); + inputTopic.pipeInput(expectedKeys[2], expectedKeys[2], 0); + + } catch (Exception exception) { + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + exception.printStackTrace(pw); + assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); + } + + final KeyValueTimestamp[] expected = { + new KeyValueTimestamp<>(0, 0, 0), + new KeyValueTimestamp<>(1, 1, 0), + }; + assertEquals(expected.length, processor.theCapturedProcessor().processed().size()); + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.theCapturedProcessor().processed().get(i)); + } + } + } @SuppressWarnings("unchecked") private KeyValueStore keyValueStoreMock() { final KeyValueStore keyValueStoreMock = mock(KeyValueStore.class); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidDsl.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidDsl.java new file mode 100644 index 0000000000000..f626e832fb87a --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidDsl.java @@ -0,0 +1,37 @@ +package org.apache.kafka.streams.processor.internals.testutil; + +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; +import org.apache.kafka.streams.processor.ErrorHandlerContext; +import org.apache.kafka.streams.processor.api.Record; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.Map; + +import static org.junit.Assert.*; + +public class LogAndContinueOnInvalidProcessor implements ProcessingExceptionHandler { + + @Override + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + + assertArrayEquals(new byte[] {0, 0, 0, 0}, context.sourceRawKey()); + assertArrayEquals(new byte[] {0, 0, 0, 0}, context.sourceRawValue()); + assertEquals(0, record.key()); + assertEquals(0, record.value()); + assertEquals("TOPIC_NAME", context.topic()); + assertEquals("KSTREAM-PROCESSOR-0000000001", context.processorNodeId()); + + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + exception.printStackTrace(pw); + assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); + + return ProcessingHandlerResponse.CONTINUE; + } + + @Override + public void configure(final Map configs) { + // No-op + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidProcessor.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidProcessor.java new file mode 100644 index 0000000000000..f626e832fb87a --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidProcessor.java @@ -0,0 +1,37 @@ +package org.apache.kafka.streams.processor.internals.testutil; + +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; +import org.apache.kafka.streams.processor.ErrorHandlerContext; +import org.apache.kafka.streams.processor.api.Record; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.Map; + +import static org.junit.Assert.*; + +public class LogAndContinueOnInvalidProcessor implements ProcessingExceptionHandler { + + @Override + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + + assertArrayEquals(new byte[] {0, 0, 0, 0}, context.sourceRawKey()); + assertArrayEquals(new byte[] {0, 0, 0, 0}, context.sourceRawValue()); + assertEquals(0, record.key()); + assertEquals(0, record.value()); + assertEquals("TOPIC_NAME", context.topic()); + assertEquals("KSTREAM-PROCESSOR-0000000001", context.processorNodeId()); + + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + exception.printStackTrace(pw); + assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); + + return ProcessingHandlerResponse.CONTINUE; + } + + @Override + public void configure(final Map configs) { + // No-op + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidPunctuate.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidPunctuate.java new file mode 100644 index 0000000000000..89aa09d844e14 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidPunctuate.java @@ -0,0 +1,36 @@ +package org.apache.kafka.streams.processor.internals.testutil; + +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; +import org.apache.kafka.streams.processor.ErrorHandlerContext; +import org.apache.kafka.streams.processor.api.Record; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.Map; + +import static org.junit.Assert.*; + +public class LogAndContinueOnInvalidPunctuate implements ProcessingExceptionHandler { + + @Override + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + + assertNull(context.sourceRawKey()); + assertNull(context.sourceRawValue()); + assertNull(record); + assertNull(context.topic()); + assertEquals("KSTREAM-PROCESSOR-0000000001", context.processorNodeId()); + + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + exception.printStackTrace(pw); + assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); + + return ProcessingHandlerResponse.CONTINUE; + } + + @Override + public void configure(final Map configs) { + // No-op + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidDsl.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidDsl.java new file mode 100644 index 0000000000000..2e1a9daf0ab7f --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidDsl.java @@ -0,0 +1,37 @@ +package org.apache.kafka.streams.processor.internals.testutil; + +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; +import org.apache.kafka.streams.processor.ErrorHandlerContext; +import org.apache.kafka.streams.processor.api.Record; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.Map; + +import static org.junit.Assert.*; + +public class LogAndContinueOnInvalidDsl implements ProcessingExceptionHandler { + + @Override + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + + assertArrayEquals(new byte[] {0, 0, 0, 1}, context.sourceRawKey()); + assertArrayEquals(new byte[] {0, 0, 0, 1}, context.sourceRawValue()); + assertEquals(1, record.key()); + assertEquals("mv1", record.value()); + assertEquals("TOPIC_NAME", context.topic()); + assertEquals("KSTREAM-MAP-0000000002", context.processorNodeId()); + + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + exception.printStackTrace(pw); + assertTrue(sw.toString().contains("java.lang.StringIndexOutOfBoundsException: begin 0, end 4, length 3")); + + return ProcessingHandlerResponse.CONTINUE; + } + + @Override + public void configure(final Map configs) { + // No-op + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidProcessor.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidProcessor.java new file mode 100644 index 0000000000000..b7bd8f2475a26 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidProcessor.java @@ -0,0 +1,38 @@ +package org.apache.kafka.streams.processor.internals.testutil; + +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; +import org.apache.kafka.streams.processor.ErrorHandlerContext; +import org.apache.kafka.streams.processor.api.Record; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.Map; + +import static org.junit.Assert.*; + +public class LogAndFailOnInvalidProcessor implements ProcessingExceptionHandler { + + @Override + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + + assertArrayEquals(new byte[] {0, 0, 0, 0}, context.sourceRawKey()); + assertArrayEquals(new byte[] {0, 0, 0, 0}, context.sourceRawValue()); + assertEquals(0, record.key()); + assertEquals(0, record.value()); + assertEquals("TOPIC_NAME", context.topic()); + assertEquals("KSTREAM-PROCESSOR-0000000001", context.processorNodeId()); + + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + exception.printStackTrace(pw); + assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); + + + return ProcessingHandlerResponse.FAIL; + } + + @Override + public void configure(final Map configs) { + // No-op + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidPunctuate.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidPunctuate.java new file mode 100644 index 0000000000000..33a8fdf241d72 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidPunctuate.java @@ -0,0 +1,37 @@ +package org.apache.kafka.streams.processor.internals.testutil; + +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; +import org.apache.kafka.streams.processor.ErrorHandlerContext; +import org.apache.kafka.streams.processor.api.Record; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.Map; + +import static org.junit.Assert.*; + +public class LogAndFailOnInvalidPunctuate implements ProcessingExceptionHandler { + + @Override + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + + assertNull(context.sourceRawKey()); + assertNull(context.sourceRawValue()); + assertNull(record); + assertNull(context.topic()); + assertEquals("KSTREAM-PROCESSOR-0000000001", context.processorNodeId()); + + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + exception.printStackTrace(pw); + assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); + + + return ProcessingHandlerResponse.FAIL; + } + + @Override + public void configure(final Map configs) { + // No-op + } +} From 9c84ceb1a54b2e7ab606f78ce88fed5d4745c337 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Sun, 26 May 2024 04:18:35 +0200 Subject: [PATCH 25/28] KAFKA-16448: Simplify unit tests and handle metrics --- .../internals/ErrorHandlerContextImpl.java | 2 - .../internals/ProcessorContextImpl.java | 19 + .../ProcessingExceptionHandlerTest.java | 461 +++++++++++++ .../internals/ProcessorContextImplTest.java | 634 +++--------------- .../testutil/LogAndContinueOnInvalidDsl.java | 37 - .../LogAndContinueOnInvalidProcessor.java | 37 - .../LogAndContinueOnInvalidPunctuate.java | 36 - .../testutil/LogAndFailOnInvalidDsl.java | 37 - .../LogAndFailOnInvalidProcessor.java | 38 -- .../LogAndFailOnInvalidPunctuate.java | 37 - 10 files changed, 579 insertions(+), 759 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/ProcessingExceptionHandlerTest.java delete mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidDsl.java delete mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidProcessor.java delete mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidPunctuate.java delete mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidDsl.java delete mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidProcessor.java delete mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidPunctuate.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java index e5debd5977fd7..0b4830cd13df6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ErrorHandlerContextImpl.java @@ -22,9 +22,7 @@ import org.apache.kafka.streams.processor.TaskId; public class ErrorHandlerContextImpl implements ErrorHandlerContext { - private InternalProcessorContext processorContext; - private final String topic; private final int partition; private final long offset; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index 264184da5d32d..7cbd643ae2f6f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.header.internals.RecordHeaders; +import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.ProcessingExceptionHandler; @@ -35,10 +36,13 @@ import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.query.Position; import org.apache.kafka.streams.state.internals.PositionSerde; import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.streams.state.internals.ThreadCache.DirtyEntryFlushListener; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.time.Duration; import java.util.HashMap; @@ -53,6 +57,8 @@ import static org.apache.kafka.streams.processor.internals.AbstractReadWriteDecorator.getReadWriteStore; public class ProcessorContextImpl extends AbstractProcessorContext implements RecordCollector.Supplier { + private static final Logger log = LoggerFactory.getLogger(ProcessorContextImpl.class); + // the below are null for standby tasks private StreamTask streamTask; private RecordCollector collector; @@ -64,6 +70,7 @@ public class ProcessorContextImpl extends AbstractProcessorContext cacheNameToFlushListener = new HashMap<>(); + private final Sensor droppedRecordsSensor; @SuppressWarnings("this-escape") public ProcessorContextImpl(final TaskId id, @@ -78,6 +85,9 @@ public ProcessorContextImpl(final TaskId id, IQ_CONSISTENCY_OFFSET_VECTOR_ENABLED, false); processingExceptionHandler = config.processingExceptionHandler(); + + final String threadId = Thread.currentThread().getName(); + droppedRecordsSensor = TaskMetrics.droppedRecordsSensor(threadId, taskId().toString(), metrics()); } @Override @@ -316,6 +326,15 @@ private void forwardInternal(final ProcessorNode child, " continue after a processing error, please set the " + PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.", e); + } else { + log.warn( + "Skipping record due to processing error. topic=[{}] partition=[{}] offset=[{}]", + topic(), + partition(), + offset(), + e + ); + droppedRecordsSensor.record(); } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/ProcessingExceptionHandlerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/ProcessingExceptionHandlerTest.java new file mode 100644 index 0000000000000..aa39fc1e7ffb8 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/ProcessingExceptionHandlerTest.java @@ -0,0 +1,461 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.processor; + +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.KeyValueTimestamp; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.TestInputTopic; +import org.apache.kafka.streams.TopologyTestDriver; +import org.apache.kafka.streams.errors.ProcessingExceptionHandler; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; + +import java.time.Duration; +import java.time.Instant; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertIterableEquals; + +@RunWith(MockitoJUnitRunner.StrictStubs.class) +public class ProcessingExceptionHandlerTest { + @Test + public void shouldContinueInProcessorOnProcessingRecordAtBeginningExceptions() { + final List> events = Arrays.asList( + new KeyValue<>("ID123-1", "ID123-A1"), + new KeyValue<>("ID123-2", "ID123-A2"), + new KeyValue<>("ID123-3", "ID123-A3"), + new KeyValue<>("ID123-4", "ID123-A4") + ); + + final List> expected = Arrays.asList( + new KeyValueTimestamp<>("ID123-2", "ID123-A2", 0), + new KeyValueTimestamp<>("ID123-3", "ID123-A3", 0), + new KeyValueTimestamp<>("ID123-4", "ID123-A4", 0) + ); + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final StreamsBuilder builder = new StreamsBuilder(); + builder + .stream("TOPIC_NAME", Consumed.with(Serdes.String(), Serdes.String())) + .process(runtimeErrorProcessorSupplierMock()) + .process(processor); + + final Properties properties = new Properties(); + properties.put("processing.exception.handler", ContinueProcessingExceptionHandlerMockTest.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); + inputTopic.pipeKeyValueList(events, Instant.EPOCH, Duration.ZERO); + + assertEquals(expected.size(), processor.theCapturedProcessor().processed().size()); + assertIterableEquals(expected, processor.theCapturedProcessor().processed()); + } + } + + @Test + public void shouldContinueInProcessorOnProcessingRecordInMiddleExceptions() { + final List> events = Arrays.asList( + new KeyValue<>("ID123-2", "ID123-A2"), + new KeyValue<>("ID123-1", "ID123-A1"), + new KeyValue<>("ID123-3", "ID123-A3"), + new KeyValue<>("ID123-4", "ID123-A4") + ); + + final List> expected = Arrays.asList( + new KeyValueTimestamp<>("ID123-2", "ID123-A2", 0), + new KeyValueTimestamp<>("ID123-3", "ID123-A3", 0), + new KeyValueTimestamp<>("ID123-4", "ID123-A4", 0) + ); + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final StreamsBuilder builder = new StreamsBuilder(); + builder + .stream("TOPIC_NAME", Consumed.with(Serdes.String(), Serdes.String())) + .process(runtimeErrorProcessorSupplierMock()) + .process(processor); + + final Properties properties = new Properties(); + properties.put("processing.exception.handler", ContinueProcessingExceptionHandlerMockTest.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); + inputTopic.pipeKeyValueList(events, Instant.EPOCH, Duration.ZERO); + + assertEquals(expected.size(), processor.theCapturedProcessor().processed().size()); + assertArrayEquals(expected.toArray(), processor.theCapturedProcessor().processed().toArray()); + } + } + + @Test + public void shouldContinueInProcessorOnProcessingRecordAtEndExceptions() { + final List> events = Arrays.asList( + new KeyValue<>("ID123-2", "ID123-A2"), + new KeyValue<>("ID123-3", "ID123-A3"), + new KeyValue<>("ID123-4", "ID123-A4"), + new KeyValue<>("ID123-1", "ID123-A1") + ); + + final List> expected = Arrays.asList( + new KeyValueTimestamp<>("ID123-2", "ID123-A2", 0), + new KeyValueTimestamp<>("ID123-3", "ID123-A3", 0), + new KeyValueTimestamp<>("ID123-4", "ID123-A4", 0) + ); + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final StreamsBuilder builder = new StreamsBuilder(); + builder + .stream("TOPIC_NAME", Consumed.with(Serdes.String(), Serdes.String())) + .process(runtimeErrorProcessorSupplierMock()) + .process(processor); + + final Properties properties = new Properties(); + properties.put("processing.exception.handler", ContinueProcessingExceptionHandlerMockTest.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); + inputTopic.pipeKeyValueList(events, Instant.EPOCH, Duration.ZERO); + + assertEquals(expected.size(), processor.theCapturedProcessor().processed().size()); + assertArrayEquals(expected.toArray(), processor.theCapturedProcessor().processed().toArray()); + } + } + + @Test + public void shouldFailInProcessorOnProcessingRecordAtBeginningExceptions() { + final List> events = Arrays.asList( + new KeyValue<>("ID123-1", "ID123-A1"), + new KeyValue<>("ID123-2", "ID123-A2"), + new KeyValue<>("ID123-3", "ID123-A3"), + new KeyValue<>("ID123-4", "ID123-A4") + ); + + final List> expected = Collections.emptyList(); + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final StreamsBuilder builder = new StreamsBuilder(); + builder + .stream("TOPIC_NAME", Consumed.with(Serdes.String(), Serdes.String())) + .process(runtimeErrorProcessorSupplierMock()) + .process(processor); + + final Properties properties = new Properties(); + properties.put("processing.exception.handler", FailProcessingExceptionHandlerMockTest.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); + + final RuntimeException exception = assertThrows(RuntimeException.class, + () -> inputTopic.pipeKeyValueList(events, Instant.EPOCH, Duration.ZERO)); + + assertEquals("Exception should be handled by processing exception handler", exception.getCause().getMessage()); + assertEquals(0, processor.theCapturedProcessor().processed().size()); + assertArrayEquals(expected.toArray(), processor.theCapturedProcessor().processed().toArray()); + } + } + + @Test + public void shouldFailInProcessorOnProcessingRecordInMiddleExceptions() { + final List> events = Arrays.asList( + new KeyValue<>("ID123-2", "ID123-A2"), + new KeyValue<>("ID123-1", "ID123-A1"), + new KeyValue<>("ID123-3", "ID123-A3"), + new KeyValue<>("ID123-4", "ID123-A4") + ); + + final List> expected = Collections.singletonList( + new KeyValueTimestamp<>("ID123-2", "ID123-A2", 0) + ); + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final StreamsBuilder builder = new StreamsBuilder(); + builder + .stream("TOPIC_NAME", Consumed.with(Serdes.String(), Serdes.String())) + .process(runtimeErrorProcessorSupplierMock()) + .process(processor); + + final Properties properties = new Properties(); + properties.put("processing.exception.handler", FailProcessingExceptionHandlerMockTest.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); + + final RuntimeException exception = assertThrows(RuntimeException.class, + () -> inputTopic.pipeKeyValueList(events, Instant.EPOCH, Duration.ZERO)); + + assertEquals("Exception should be handled by processing exception handler", exception.getCause().getMessage()); + assertEquals(expected.size(), processor.theCapturedProcessor().processed().size()); + assertArrayEquals(expected.toArray(), processor.theCapturedProcessor().processed().toArray()); + } + } + + @Test + public void shouldFailInProcessorOnProcessingRecordAtEndExceptions() { + final List> events = Arrays.asList( + new KeyValue<>("ID123-2", "ID123-A2"), + new KeyValue<>("ID123-3", "ID123-A3"), + new KeyValue<>("ID123-4", "ID123-A4"), + new KeyValue<>("ID123-1", "ID123-A1") + ); + + final List> expected = Arrays.asList( + new KeyValueTimestamp<>("ID123-2", "ID123-A2", 0), + new KeyValueTimestamp<>("ID123-3", "ID123-A3", 0), + new KeyValueTimestamp<>("ID123-4", "ID123-A4", 0) + ); + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final StreamsBuilder builder = new StreamsBuilder(); + builder + .stream("TOPIC_NAME", Consumed.with(Serdes.String(), Serdes.String())) + .process(runtimeErrorProcessorSupplierMock()) + .process(processor); + + final Properties properties = new Properties(); + properties.put("processing.exception.handler", FailProcessingExceptionHandlerMockTest.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); + + final RuntimeException exception = assertThrows(RuntimeException.class, + () -> inputTopic.pipeKeyValueList(events, Instant.EPOCH, Duration.ZERO)); + + assertEquals("Exception should be handled by processing exception handler", exception.getCause().getMessage()); + assertEquals(expected.size(), processor.theCapturedProcessor().processed().size()); + assertArrayEquals(expected.toArray(), processor.theCapturedProcessor().processed().toArray()); + } + } + + @Test + public void shouldContinueOnPunctuateExceptions() { + final List> events = Arrays.asList( + new KeyValue<>("ID123-1", "ID123-A1"), + new KeyValue<>("ID123-2", "ID123-A2"), + new KeyValue<>("ID123-3", "ID123-A3"), + new KeyValue<>("ID123-4", "ID123-A4") + ); + + final List> expected = Arrays.asList( + new KeyValueTimestamp<>("ID123-1", "ID123-A1", 0), + new KeyValueTimestamp<>("ID123-2", "ID123-A2", 0), + new KeyValueTimestamp<>("ID123-3", "ID123-A3", 0), + new KeyValueTimestamp<>("ID123-4", "ID123-A4", 0) + ); + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final StreamsBuilder builder = new StreamsBuilder(); + builder + .stream("TOPIC_NAME", Consumed.with(Serdes.String(), Serdes.String())) + .process(runtimeErrorPunctuateProcessorSupplierMock()) + .process(processor); + + final Properties properties = new Properties(); + properties.put("processing.exception.handler", ContinuePunctuateProcessingExceptionHandlerMockTest.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); + inputTopic.pipeKeyValueList(events, Instant.EPOCH, Duration.ZERO); + + assertEquals(expected.size(), processor.theCapturedProcessor().processed().size()); + assertArrayEquals(expected.toArray(), processor.theCapturedProcessor().processed().toArray()); + } + } + + @Test + public void shouldFailOnPunctuateExceptions() { + final List> events = Arrays.asList( + new KeyValue<>("ID123-1", "ID123-A1"), + new KeyValue<>("ID123-2", "ID123-A2"), + new KeyValue<>("ID123-3", "ID123-A3") + ); + + final List> expected = Arrays.asList( + new KeyValueTimestamp<>("ID123-1", "ID123-A1", 0), + new KeyValueTimestamp<>("ID123-2", "ID123-A2", 0), + new KeyValueTimestamp<>("ID123-3", "ID123-A3", 0) + ); + + final MockProcessorSupplier processor = new MockProcessorSupplier<>(); + final StreamsBuilder builder = new StreamsBuilder(); + builder + .stream("TOPIC_NAME", Consumed.with(Serdes.String(), Serdes.String())) + .process(runtimeErrorPunctuateProcessorSupplierMock()) + .process(processor); + + final Properties properties = new Properties(); + properties.put("processing.exception.handler", FailPunctuateProcessingExceptionHandlerMockTest.class); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); + + final RuntimeException exception = assertThrows(RuntimeException.class, + () -> { + inputTopic.pipeKeyValueList(events, Instant.EPOCH, Duration.ZERO); + driver.advanceWallClockTime(Duration.ofSeconds(2)); + inputTopic.pipeInput("ID123-4", "ID123-A4", 0); + }); + + assertEquals("Exception should be handled by processing exception handler", exception.getCause().getMessage()); + assertEquals(expected.size(), processor.theCapturedProcessor().processed().size()); + assertArrayEquals(expected.toArray(), processor.theCapturedProcessor().processed().toArray()); + } + } + + public static class ContinueProcessingExceptionHandlerMockTest implements ProcessingExceptionHandler { + @Override + public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + assertEquals("ID123-1", new String(context.sourceRawKey())); + assertEquals("ID123-A1", new String(context.sourceRawValue())); + assertEquals("ID123-1", record.key()); + assertEquals("ID123-A1", record.value()); + assertEquals("TOPIC_NAME", context.topic()); + assertEquals("KSTREAM-PROCESSOR-0000000001", context.processorNodeId()); + assertTrue(exception.getMessage().contains("Exception should be handled by processing exception handler")); + + return ProcessingExceptionHandler.ProcessingHandlerResponse.CONTINUE; + } + + @Override + public void configure(final Map configs) { + // No-op + } + } + + public static class FailProcessingExceptionHandlerMockTest implements ProcessingExceptionHandler { + @Override + public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + assertEquals("ID123-1", new String(context.sourceRawKey())); + assertEquals("ID123-A1", new String(context.sourceRawValue())); + assertEquals("ID123-1", record.key()); + assertEquals("ID123-A1", record.value()); + assertEquals("TOPIC_NAME", context.topic()); + assertEquals("KSTREAM-PROCESSOR-0000000001", context.processorNodeId()); + assertTrue(exception.getMessage().contains("Exception should be handled by processing exception handler")); + + return ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL; + } + + @Override + public void configure(final Map configs) { + // No-op + } + } + + public static class ContinuePunctuateProcessingExceptionHandlerMockTest implements ProcessingExceptionHandler { + @Override + public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + assertNull(context.sourceRawKey()); + assertNull(context.sourceRawValue()); + assertNull(record); + assertNull(context.topic()); + assertEquals("KSTREAM-PROCESSOR-0000000001", context.processorNodeId()); + assertTrue(exception.getMessage().contains("Exception should be handled by processing exception handler")); + + return ProcessingExceptionHandler.ProcessingHandlerResponse.CONTINUE; + } + + @Override + public void configure(final Map configs) { + // No-op + } + } + + public static class FailPunctuateProcessingExceptionHandlerMockTest implements ProcessingExceptionHandler { + @Override + public ProcessingExceptionHandler.ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + assertNull(context.sourceRawKey()); + assertNull(context.sourceRawValue()); + assertNull(record); + assertNull(context.topic()); + assertEquals("KSTREAM-PROCESSOR-0000000001", context.processorNodeId()); + assertTrue(exception.getMessage().contains("Exception should be handled by processing exception handler")); + + return ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL; + } + + @Override + public void configure(final Map configs) { + // No-op + } + } + + /** + * Processor supplier that throws a runtime exception on schedule. + * + * @return the processor supplier + */ + private org.apache.kafka.streams.processor.api.ProcessorSupplier runtimeErrorPunctuateProcessorSupplierMock() { + return () -> new org.apache.kafka.streams.processor.api.Processor() { + org.apache.kafka.streams.processor.api.ProcessorContext context; + + @Override + public void init(final org.apache.kafka.streams.processor.api.ProcessorContext context) { + this.context = context; + this.context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, ts -> { + throw new RuntimeException("Exception should be handled by processing exception handler"); + }); + } + + @Override + public void process(final Record record) { + context.forward(record); + } + }; + } + + /** + * Processor supplier that throws a runtime exception on process. + * + * @return the processor supplier + */ + private ProcessorSupplier runtimeErrorProcessorSupplierMock() { + return () -> new Processor() { + org.apache.kafka.streams.processor.api.ProcessorContext context; + + @Override + public void init(final org.apache.kafka.streams.processor.api.ProcessorContext context) { + this.context = context; + } + + @Override + public void process(final Record record) { + if (record.key().equals("ID123-1")) { + throw new RuntimeException("Exception should be handled by processing exception handler"); + } + + context.forward(new Record<>(record.key(), record.value(), record.timestamp())); + } + }; + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java index 1d5d7dfd8b205..47f331246e55c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java @@ -17,23 +17,18 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; 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.IntegerSerializer; -import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsConfig.InternalConfig; import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.KeyValueTimestamp; -import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.StreamsBuilder; -import org.apache.kafka.streams.TestInputTopic; -import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ErrorHandlerContext; import org.apache.kafka.streams.processor.ProcessorContext; @@ -43,14 +38,9 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.api.Processor; -import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; -import org.apache.kafka.streams.processor.internals.testutil.LogAndContinueOnInvalidProcessor; -import org.apache.kafka.streams.processor.internals.testutil.LogAndContinueOnInvalidPunctuate; -import org.apache.kafka.streams.processor.internals.testutil.LogAndFailOnInvalidProcessor; -import org.apache.kafka.streams.processor.internals.testutil.LogAndFailOnInvalidPunctuate; import org.apache.kafka.streams.query.Position; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; @@ -62,17 +52,13 @@ import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.internals.PositionSerde; import org.apache.kafka.streams.state.internals.ThreadCache; -import org.apache.kafka.test.MockProcessorSupplier; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; -import java.io.PrintWriter; -import java.io.StringWriter; import java.time.Duration; -import java.time.Instant; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -80,7 +66,6 @@ import java.util.List; import java.util.Map; import java.util.function.Consumer; -import java.util.Properties; import static java.util.Arrays.asList; import static org.apache.kafka.common.utils.Utils.mkEntry; @@ -107,6 +92,10 @@ public class ProcessorContextImplTest { private final StreamsConfig streamsConfig = streamsConfigMock(); + private final Metrics metrics = new Metrics(); + private final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test", StreamsConfig.METRICS_LATEST, new MockTime()); + private final String threadId = Thread.currentThread().getName(); + @Mock private RecordCollector recordCollector; @Mock @@ -180,7 +169,7 @@ public void setup() { mock(TaskId.class), streamsConfig, stateManager, - mock(StreamsMetricsImpl.class), + streamsMetrics, mock(ThreadCache.class) ); @@ -212,7 +201,7 @@ private ProcessorContextImpl getStandbyContext() { mock(TaskId.class), streamsConfig, stateManager, - mock(StreamsMetricsImpl.class), + streamsMetrics, mock(ThreadCache.class) ); } @@ -451,7 +440,7 @@ public void shouldSendRecordHeadersToChangelogTopicWhenConsistencyEnabled() { mock(TaskId.class), streamsConfigWithConsistencyMock(), stateManager, - mock(StreamsMetricsImpl.class), + streamsMetrics, mock(ThreadCache.class) ); @@ -624,35 +613,25 @@ public void shouldSetAndGetProcessorMetaData() { @Test public void shouldContinueOnProcessingExceptions() { - when(streamsConfig.processingExceptionHandler()).thenReturn(new ProcessingExceptionHandler() { - @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { - assertArrayEquals(new byte[] {107, 101, 121}, context.sourceRawKey()); - assertArrayEquals(new byte[] {118, 97, 108, 117, 101}, context.sourceRawValue()); - assertEquals("fakeChild", context.processorNodeId()); - assertEquals("key", record.key()); - assertEquals("value", record.value()); - assertEquals("Exception should be handled by processing exception handler", exception.getMessage()); + when(streamsConfig.processingExceptionHandler()) + .thenReturn(processingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingHandlerResponse.CONTINUE)); - return ProcessingHandlerResponse.CONTINUE; - } + final TaskId taskId = mock(TaskId.class); + when(taskId.toString()).thenReturn("0_0"); - @Override - public void configure(final Map configs) { - // No-op - } - }); + final StreamTask task = mock(StreamTask.class); + when(task.rawRecord()).thenReturn(new ConsumerRecord<>("topic", 0, 0, KEY_BYTES.get(), VALUE_BYTES)); - context = new ProcessorContextImpl( - mock(TaskId.class), + final Metrics metrics = new Metrics(); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test", StreamsConfig.METRICS_LATEST, new MockTime()); + final ProcessorContextImpl context = new ProcessorContextImpl( + taskId, streamsConfig, stateManager, - mock(StreamsMetricsImpl.class), + streamsMetrics, mock(ThreadCache.class) ); - final StreamTask task = mock(StreamTask.class); - when(task.rawRecord()).thenReturn(new ConsumerRecord<>("topic", 0, 0, "key".getBytes(), "value".getBytes())); context.transitionToActive(task, null, null); final ProcessorNode processorNode = new ProcessorNode<>( @@ -676,39 +655,53 @@ public void configure(final Map configs) { context.setCurrentNode(processorNode); context.forward("key", "value"); + + final MetricName dropTotal; + final MetricName dropRate; + dropTotal = new MetricName( + "dropped-records-total", + "stream-task-metrics", + "The total number of dropped records", + mkMap( + mkEntry("thread-id", threadId), + mkEntry("task-id", "0_0") + ) + ); + dropRate = new MetricName( + "dropped-records-rate", + "stream-task-metrics", + "The average number of dropped records per second", + mkMap( + mkEntry("thread-id", threadId), + mkEntry("task-id", "0_0") + ) + ); + + assertEquals(1.0, metrics.metrics().get(dropTotal).metricValue()); + assertTrue((Double) metrics.metrics().get(dropRate).metricValue() > 0.0); } @Test public void shouldFailOnProcessingExceptions() { - when(streamsConfig.processingExceptionHandler()).thenReturn(new ProcessingExceptionHandler() { - @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { - assertArrayEquals(new byte[] {107, 101, 121}, context.sourceRawKey()); - assertArrayEquals(new byte[] {118, 97, 108, 117, 101}, context.sourceRawValue()); - assertEquals("fakeChild", context.processorNodeId()); - assertEquals("key", record.key()); - assertEquals("value", record.value()); - assertEquals("Exception should be handled by processing exception handler", exception.getMessage()); + when(streamsConfig.processingExceptionHandler()) + .thenReturn(processingExceptionHandlerMock(ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL)); - return ProcessingHandlerResponse.FAIL; - } + final TaskId taskId = mock(TaskId.class); + when(taskId.toString()).thenReturn("0_0"); - @Override - public void configure(final Map configs) { - // No-op - } - }); + final StreamTask task = mock(StreamTask.class); + when(task.rawRecord()).thenReturn(new ConsumerRecord<>("topic", 0, 0, KEY_BYTES.get(), VALUE_BYTES)); - context = new ProcessorContextImpl( - mock(TaskId.class), + final Metrics metrics = new Metrics(); + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test", StreamsConfig.METRICS_LATEST, new MockTime()); + final ProcessorContextImpl context = new ProcessorContextImpl( + taskId, streamsConfig, stateManager, - mock(StreamsMetricsImpl.class), + streamsMetrics, mock(ThreadCache.class) ); - final StreamTask task = mock(StreamTask.class); - when(task.rawRecord()).thenReturn(new ConsumerRecord<>("topic", 0, 0, "key".getBytes(), "value".getBytes())); context.transitionToActive(task, null, null); final ProcessorNode processorNode = new ProcessorNode<>( @@ -735,488 +728,59 @@ public void configure(final Map configs) { assertEquals("Processing exception handler is set to fail upon a processing error. " + "If you would rather have the streaming pipeline continue after a processing error, " + "please set the processing.exception.handler appropriately.", exception.getMessage()); - } - - @Test - public void shouldContinueInProcessorOnProcessingRecordAtBeginningExceptions() { - - final int[] expectedKeys = {0, 1, 2, 10}; - - final StreamsBuilder builder = new StreamsBuilder(); - - final ProcessorSupplier processorSupplier = - () -> new Processor() { - - org.apache.kafka.streams.processor.api.ProcessorContext context; - - @Override - public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { - this.context = context; - } - - @Override - public void process(Record record) { - int value = 100 / (10 * record.value().intValue()); - context.forward(new Record<>(record.key().toString(), Integer.toString(value), record.timestamp())); - } - - @Override - public void close() { - } - }; - - final MockProcessorSupplier processor = new MockProcessorSupplier<>(); - final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); - stream.process(processorSupplier).process(processor); - - Properties properties = new Properties(); - properties.put("processing.exception.handler", LogAndContinueOnInvalidProcessor.class); - - try (final TopologyTestDriver driver = new TopologyTestDriver( - builder.build(), properties, - Instant.ofEpochMilli(0L))) { - final TestInputTopic inputTopic = - driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); - - for (final int expectedKey : expectedKeys) { - inputTopic.pipeInput(expectedKey, expectedKey, 0); - } - - final KeyValueTimestamp[] expected = { - new KeyValueTimestamp<>("1", "10", 0), - new KeyValueTimestamp<>("2", "5", 0), - new KeyValueTimestamp<>("10", "1", 0), - }; - assertEquals(expected.length, processor.theCapturedProcessor().processed().size()); - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.theCapturedProcessor().processed().get(i)); - } - } - } - - @Test - public void shouldContinueInProcessorOnProcessingRecordInMiddleExceptions() { - - final int[] expectedKeys = {1, 0, 2, 10}; - - final StreamsBuilder builder = new StreamsBuilder(); - - final ProcessorSupplier processorSupplier = - () -> new Processor() { - - org.apache.kafka.streams.processor.api.ProcessorContext context; - - @Override - public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { - this.context = context; - } - - @Override - public void process(Record record) { - int value = 100 / (10 * record.value().intValue()); - context.forward(new Record<>(record.key().toString(), Integer.toString(value), record.timestamp())); - } - - @Override - public void close() { - } - }; - - final MockProcessorSupplier processor = new MockProcessorSupplier<>(); - final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); - stream.process(processorSupplier).process(processor); - - Properties properties = new Properties(); - properties.put("processing.exception.handler", LogAndContinueOnInvalidProcessor.class); - - try (final TopologyTestDriver driver = new TopologyTestDriver( - builder.build(), properties, - Instant.ofEpochMilli(0L))) { - final TestInputTopic inputTopic = - driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); - - for (final int expectedKey : expectedKeys) { - inputTopic.pipeInput(expectedKey, expectedKey, 0); - } - - final KeyValueTimestamp[] expected = { - new KeyValueTimestamp<>("1", "10", 0), - new KeyValueTimestamp<>("2", "5", 0), - new KeyValueTimestamp<>("10", "1", 0), - }; - assertEquals(expected.length, processor.theCapturedProcessor().processed().size()); - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.theCapturedProcessor().processed().get(i)); - } - } - } - - @Test - public void shouldContinueInProcessorOnProcessingRecordAtEndExceptions() { - - final int[] expectedKeys = {1, 2, 10, 0}; - - final StreamsBuilder builder = new StreamsBuilder(); - - final ProcessorSupplier processorSupplier = - () -> new Processor() { - org.apache.kafka.streams.processor.api.ProcessorContext context; - - @Override - public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { - this.context = context; - } - - @Override - public void process(Record record) { - int value = 100 / (10 * record.value().intValue()); - context.forward(new Record<>(record.key().toString(), Integer.toString(value), record.timestamp())); - } - - @Override - public void close() { - } - }; - - - final MockProcessorSupplier processor = new MockProcessorSupplier<>(); - final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); - stream.process(processorSupplier).process(processor); - - Properties properties = new Properties(); - properties.put("processing.exception.handler", LogAndContinueOnInvalidProcessor.class); - - try (final TopologyTestDriver driver = new TopologyTestDriver( - builder.build(), properties, - Instant.ofEpochMilli(0L))) { - final TestInputTopic inputTopic = - driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); - - for (final int expectedKey : expectedKeys) { - inputTopic.pipeInput(expectedKey, expectedKey, 0); - } - - final KeyValueTimestamp[] expected = { - new KeyValueTimestamp<>("1", "10", 0), - new KeyValueTimestamp<>("2", "5", 0), - new KeyValueTimestamp<>("10", "1", 0), - }; - assertEquals(expected.length, processor.theCapturedProcessor().processed().size()); - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.theCapturedProcessor().processed().get(i)); - } - } - } + final MetricName dropTotal; + final MetricName dropRate; + dropTotal = new MetricName( + "dropped-records-total", + "stream-task-metrics", + "The total number of dropped records", + mkMap( + mkEntry("thread-id", threadId), + mkEntry("task-id", "0_0") + ) + ); + dropRate = new MetricName( + "dropped-records-rate", + "stream-task-metrics", + "The average number of dropped records per second", + mkMap( + mkEntry("thread-id", threadId), + mkEntry("task-id", "0_0") + ) + ); - @Test - public void shouldFailInProcessorOnProcessingRecordAtBeginningExceptions() { - - final int[] expectedKeys = {0, 1, 2, 10}; - - final StreamsBuilder builder = new StreamsBuilder(); - - final ProcessorSupplier processorSupplier = - () -> new Processor() { - - org.apache.kafka.streams.processor.api.ProcessorContext context; - - @Override - public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { - this.context = context; - } - - @Override - public void process(Record record) { - int value = 100 / (10 * record.value().intValue()); - context.forward(new Record<>(record.key().toString(), Integer.toString(value), record.timestamp())); - } - - @Override - public void close() { - } - }; - - final MockProcessorSupplier processor = new MockProcessorSupplier<>(); - final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); - stream.process(processorSupplier).process(processor); - - Properties properties = new Properties(); - properties.put("processing.exception.handler", LogAndFailOnInvalidProcessor.class); - - try (final TopologyTestDriver driver = new TopologyTestDriver( - builder.build(), properties, - Instant.ofEpochMilli(0L))) { - final TestInputTopic inputTopic = - driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); - - try { - for (final int expectedKey : expectedKeys) { - inputTopic.pipeInput(expectedKey, expectedKey, 0); - } - } catch (Exception exception) { - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - exception.printStackTrace(pw); - assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); - } - final KeyValueTimestamp[] expected = {}; - assertEquals(0, processor.theCapturedProcessor().processed().size()); - } + assertEquals(0.0, metrics.metrics().get(dropTotal).metricValue()); + assertEquals(0.0, metrics.metrics().get(dropRate).metricValue()); } - @Test - public void shouldFailInProcessorOnProcessingRecordInMiddleExceptions() { - - final int[] expectedKeys = {1, 0, 2, 10}; - - final StreamsBuilder builder = new StreamsBuilder(); - - final ProcessorSupplier processorSupplier = - () -> new Processor() { - - org.apache.kafka.streams.processor.api.ProcessorContext context; - - @Override - public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { - this.context = context; - } - - @Override - public void process(Record record) { - int value = 100 / (10 * record.value().intValue()); - context.forward(new Record<>(record.key().toString(), Integer.toString(value), record.timestamp())); - } - - @Override - public void close() { - } - }; - - final MockProcessorSupplier processor = new MockProcessorSupplier<>(); - final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); - stream.process(processorSupplier).process(processor); - - Properties properties = new Properties(); - properties.put("processing.exception.handler", LogAndFailOnInvalidProcessor.class); - - try (final TopologyTestDriver driver = new TopologyTestDriver( - builder.build(), properties, - Instant.ofEpochMilli(0L))) { - final TestInputTopic inputTopic = - driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); - - try { - for (final int expectedKey : expectedKeys) { - inputTopic.pipeInput(expectedKey, expectedKey, 0); - } - } catch (Exception exception) { - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - exception.printStackTrace(pw); - assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); - } - final KeyValueTimestamp[] expected = { - new KeyValueTimestamp<>("1", "10", 0), - }; - assertEquals(expected.length, processor.theCapturedProcessor().processed().size()); - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.theCapturedProcessor().processed().get(i)); - } - } - } - - @Test - public void shouldFailInProcessorOnProcessingRecordAtEndExceptions() { - - final int[] expectedKeys = {1, 2, 10, 0}; - - final StreamsBuilder builder = new StreamsBuilder(); - - final ProcessorSupplier processorSupplier = - () -> new Processor() { - - org.apache.kafka.streams.processor.api.ProcessorContext context; - - @Override - public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { - this.context = context; - } - - @Override - public void process(Record record) { - int value = 100 / (10 * record.value().intValue()); - context.forward(new Record<>(record.key().toString(), Integer.toString(value), record.timestamp())); - } - - @Override - public void close() { - } - }; - - - final MockProcessorSupplier processor = new MockProcessorSupplier<>(); - final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); - stream.process(processorSupplier).process(processor); - - Properties properties = new Properties(); - properties.put("processing.exception.handler", LogAndFailOnInvalidProcessor.class); - - try (final TopologyTestDriver driver = new TopologyTestDriver( - builder.build(), properties, - Instant.ofEpochMilli(0L))) { - final TestInputTopic inputTopic = - driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); - - try { - for (final int expectedKey : expectedKeys) { - inputTopic.pipeInput(expectedKey, expectedKey, 0); - } - } catch (Exception exception) { - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - exception.printStackTrace(pw); - assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); - } - final KeyValueTimestamp[] expected = { - new KeyValueTimestamp<>("1", "10", 0), - new KeyValueTimestamp<>("2", "5", 0), - new KeyValueTimestamp<>("10", "1", 0), - }; - assertEquals(expected.length, processor.theCapturedProcessor().processed().size()); - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.theCapturedProcessor().processed().get(i)); - } - } - } + /** + * Processing exception handler mock + * + * @param response the processing handler response + * @return the processing exception handler mock + */ + private ProcessingExceptionHandler processingExceptionHandlerMock(final ProcessingExceptionHandler.ProcessingHandlerResponse response) { + return new ProcessingExceptionHandler() { + @Override + public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { + assertArrayEquals(KEY_BYTES.get(), context.sourceRawKey()); + assertArrayEquals(VALUE_BYTES, context.sourceRawValue()); + assertEquals("fakeChild", context.processorNodeId()); + assertEquals("key", record.key()); + assertEquals("value", record.value()); + assertEquals("Exception should be handled by processing exception handler", exception.getMessage()); - @Test - public void shouldContinueOnPunctuateExceptions() { - - final int[] expectedKeys = {1, 2, 10}; - - final StreamsBuilder builder = new StreamsBuilder(); - - final ProcessorSupplier processorSupplier = - () -> new Processor() { - - org.apache.kafka.streams.processor.api.ProcessorContext context; - - @Override - public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { - this.context = context; - this.context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, ts -> { - String zero = "0"; - int i = 1 / Integer.parseInt(zero); - }); - } - - @Override - public void process(Record record) { - context.forward(record); - } - - @Override - public void close() { - } - }; - - final MockProcessorSupplier processor = new MockProcessorSupplier<>(); - final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); - stream.process(processorSupplier).process(processor); - - Properties properties = new Properties(); - properties.put("processing.exception.handler", LogAndContinueOnInvalidPunctuate.class); - - try (final TopologyTestDriver driver = new TopologyTestDriver( - builder.build(), properties, - Instant.ofEpochMilli(0L))) { - final TestInputTopic inputTopic = - driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); - for (final int expectedKey : expectedKeys) { - driver.advanceWallClockTime(Duration.ofSeconds(2)); - inputTopic.pipeInput(expectedKey, expectedKey, 0); + return response; } - final KeyValueTimestamp[] expected = { - new KeyValueTimestamp<>(1, 1, 0), - new KeyValueTimestamp<>(2, 2, 0), - new KeyValueTimestamp<>(10, 10, 0), - }; - assertEquals(expected.length, processor.theCapturedProcessor().processed().size()); - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.theCapturedProcessor().processed().get(i)); + @Override + public void configure(final Map configs) { + // No-op } - } + }; } - @Test - public void shouldFailOnPunctuateExceptions() { - - final int[] expectedKeys = {0, 1, 2, 10}; - - final StreamsBuilder builder = new StreamsBuilder(); - - final ProcessorSupplier processorSupplier = - () -> new Processor() { - - org.apache.kafka.streams.processor.api.ProcessorContext context; - - @Override - public void init(org.apache.kafka.streams.processor.api.ProcessorContext context) { - this.context = context; - this.context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, ts -> { - String zero = "0"; - int i = 1 / Integer.parseInt(zero); - }); - } - - @Override - public void process(Record record) { - context.forward(record); - } - - @Override - public void close() { - } - }; - - final MockProcessorSupplier processor = new MockProcessorSupplier<>(); - final KStream stream = builder.stream("TOPIC_NAME", Consumed.with(Serdes.Integer(), Serdes.Integer())); - stream.process(processorSupplier).process(processor); - - Properties properties = new Properties(); - properties.put("processing.exception.handler", LogAndFailOnInvalidPunctuate.class); - - try (final TopologyTestDriver driver = new TopologyTestDriver( - builder.build(), properties, - Instant.ofEpochMilli(0L))) { - final TestInputTopic inputTopic = - driver.createInputTopic("TOPIC_NAME", new IntegerSerializer(), new IntegerSerializer()); - - try { - inputTopic.pipeInput(expectedKeys[0], expectedKeys[0], 0); - inputTopic.pipeInput(expectedKeys[1], expectedKeys[1], 0); - driver.advanceWallClockTime(Duration.ofSeconds(2)); - inputTopic.pipeInput(expectedKeys[2], expectedKeys[2], 0); - - } catch (Exception exception) { - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - exception.printStackTrace(pw); - assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); - } - - final KeyValueTimestamp[] expected = { - new KeyValueTimestamp<>(0, 0, 0), - new KeyValueTimestamp<>(1, 1, 0), - }; - assertEquals(expected.length, processor.theCapturedProcessor().processed().size()); - for (int i = 0; i < expected.length; i++) { - assertEquals(expected[i], processor.theCapturedProcessor().processed().get(i)); - } - } - } @SuppressWarnings("unchecked") private KeyValueStore keyValueStoreMock() { final KeyValueStore keyValueStoreMock = mock(KeyValueStore.class); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidDsl.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidDsl.java deleted file mode 100644 index f626e832fb87a..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidDsl.java +++ /dev/null @@ -1,37 +0,0 @@ -package org.apache.kafka.streams.processor.internals.testutil; - -import org.apache.kafka.streams.errors.ProcessingExceptionHandler; -import org.apache.kafka.streams.processor.ErrorHandlerContext; -import org.apache.kafka.streams.processor.api.Record; - -import java.io.PrintWriter; -import java.io.StringWriter; -import java.util.Map; - -import static org.junit.Assert.*; - -public class LogAndContinueOnInvalidProcessor implements ProcessingExceptionHandler { - - @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { - - assertArrayEquals(new byte[] {0, 0, 0, 0}, context.sourceRawKey()); - assertArrayEquals(new byte[] {0, 0, 0, 0}, context.sourceRawValue()); - assertEquals(0, record.key()); - assertEquals(0, record.value()); - assertEquals("TOPIC_NAME", context.topic()); - assertEquals("KSTREAM-PROCESSOR-0000000001", context.processorNodeId()); - - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - exception.printStackTrace(pw); - assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); - - return ProcessingHandlerResponse.CONTINUE; - } - - @Override - public void configure(final Map configs) { - // No-op - } -} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidProcessor.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidProcessor.java deleted file mode 100644 index f626e832fb87a..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidProcessor.java +++ /dev/null @@ -1,37 +0,0 @@ -package org.apache.kafka.streams.processor.internals.testutil; - -import org.apache.kafka.streams.errors.ProcessingExceptionHandler; -import org.apache.kafka.streams.processor.ErrorHandlerContext; -import org.apache.kafka.streams.processor.api.Record; - -import java.io.PrintWriter; -import java.io.StringWriter; -import java.util.Map; - -import static org.junit.Assert.*; - -public class LogAndContinueOnInvalidProcessor implements ProcessingExceptionHandler { - - @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { - - assertArrayEquals(new byte[] {0, 0, 0, 0}, context.sourceRawKey()); - assertArrayEquals(new byte[] {0, 0, 0, 0}, context.sourceRawValue()); - assertEquals(0, record.key()); - assertEquals(0, record.value()); - assertEquals("TOPIC_NAME", context.topic()); - assertEquals("KSTREAM-PROCESSOR-0000000001", context.processorNodeId()); - - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - exception.printStackTrace(pw); - assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); - - return ProcessingHandlerResponse.CONTINUE; - } - - @Override - public void configure(final Map configs) { - // No-op - } -} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidPunctuate.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidPunctuate.java deleted file mode 100644 index 89aa09d844e14..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndContinueOnInvalidPunctuate.java +++ /dev/null @@ -1,36 +0,0 @@ -package org.apache.kafka.streams.processor.internals.testutil; - -import org.apache.kafka.streams.errors.ProcessingExceptionHandler; -import org.apache.kafka.streams.processor.ErrorHandlerContext; -import org.apache.kafka.streams.processor.api.Record; - -import java.io.PrintWriter; -import java.io.StringWriter; -import java.util.Map; - -import static org.junit.Assert.*; - -public class LogAndContinueOnInvalidPunctuate implements ProcessingExceptionHandler { - - @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { - - assertNull(context.sourceRawKey()); - assertNull(context.sourceRawValue()); - assertNull(record); - assertNull(context.topic()); - assertEquals("KSTREAM-PROCESSOR-0000000001", context.processorNodeId()); - - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - exception.printStackTrace(pw); - assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); - - return ProcessingHandlerResponse.CONTINUE; - } - - @Override - public void configure(final Map configs) { - // No-op - } -} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidDsl.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidDsl.java deleted file mode 100644 index 2e1a9daf0ab7f..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidDsl.java +++ /dev/null @@ -1,37 +0,0 @@ -package org.apache.kafka.streams.processor.internals.testutil; - -import org.apache.kafka.streams.errors.ProcessingExceptionHandler; -import org.apache.kafka.streams.processor.ErrorHandlerContext; -import org.apache.kafka.streams.processor.api.Record; - -import java.io.PrintWriter; -import java.io.StringWriter; -import java.util.Map; - -import static org.junit.Assert.*; - -public class LogAndContinueOnInvalidDsl implements ProcessingExceptionHandler { - - @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { - - assertArrayEquals(new byte[] {0, 0, 0, 1}, context.sourceRawKey()); - assertArrayEquals(new byte[] {0, 0, 0, 1}, context.sourceRawValue()); - assertEquals(1, record.key()); - assertEquals("mv1", record.value()); - assertEquals("TOPIC_NAME", context.topic()); - assertEquals("KSTREAM-MAP-0000000002", context.processorNodeId()); - - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - exception.printStackTrace(pw); - assertTrue(sw.toString().contains("java.lang.StringIndexOutOfBoundsException: begin 0, end 4, length 3")); - - return ProcessingHandlerResponse.CONTINUE; - } - - @Override - public void configure(final Map configs) { - // No-op - } -} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidProcessor.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidProcessor.java deleted file mode 100644 index b7bd8f2475a26..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidProcessor.java +++ /dev/null @@ -1,38 +0,0 @@ -package org.apache.kafka.streams.processor.internals.testutil; - -import org.apache.kafka.streams.errors.ProcessingExceptionHandler; -import org.apache.kafka.streams.processor.ErrorHandlerContext; -import org.apache.kafka.streams.processor.api.Record; - -import java.io.PrintWriter; -import java.io.StringWriter; -import java.util.Map; - -import static org.junit.Assert.*; - -public class LogAndFailOnInvalidProcessor implements ProcessingExceptionHandler { - - @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { - - assertArrayEquals(new byte[] {0, 0, 0, 0}, context.sourceRawKey()); - assertArrayEquals(new byte[] {0, 0, 0, 0}, context.sourceRawValue()); - assertEquals(0, record.key()); - assertEquals(0, record.value()); - assertEquals("TOPIC_NAME", context.topic()); - assertEquals("KSTREAM-PROCESSOR-0000000001", context.processorNodeId()); - - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - exception.printStackTrace(pw); - assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); - - - return ProcessingHandlerResponse.FAIL; - } - - @Override - public void configure(final Map configs) { - // No-op - } -} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidPunctuate.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidPunctuate.java deleted file mode 100644 index 33a8fdf241d72..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/LogAndFailOnInvalidPunctuate.java +++ /dev/null @@ -1,37 +0,0 @@ -package org.apache.kafka.streams.processor.internals.testutil; - -import org.apache.kafka.streams.errors.ProcessingExceptionHandler; -import org.apache.kafka.streams.processor.ErrorHandlerContext; -import org.apache.kafka.streams.processor.api.Record; - -import java.io.PrintWriter; -import java.io.StringWriter; -import java.util.Map; - -import static org.junit.Assert.*; - -public class LogAndFailOnInvalidPunctuate implements ProcessingExceptionHandler { - - @Override - public ProcessingHandlerResponse handle(final ErrorHandlerContext context, final Record record, final Exception exception) { - - assertNull(context.sourceRawKey()); - assertNull(context.sourceRawValue()); - assertNull(record); - assertNull(context.topic()); - assertEquals("KSTREAM-PROCESSOR-0000000001", context.processorNodeId()); - - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - exception.printStackTrace(pw); - assertTrue(sw.toString().contains("java.lang.ArithmeticException: / by zero")); - - - return ProcessingHandlerResponse.FAIL; - } - - @Override - public void configure(final Map configs) { - // No-op - } -} From d3e2c58caf99ddbb2754b3e72df07b89b6673527 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Sun, 26 May 2024 14:25:52 +0200 Subject: [PATCH 26/28] KAFKA-16448: Add metrics in tests --- .../processor/internals/StreamTask.java | 11 ++ .../ProcessingExceptionHandlerTest.java | 103 ++++++++++++++++-- .../internals/ProcessorContextImplTest.java | 78 +++++++------ 3 files changed, 144 insertions(+), 48 deletions(-) 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 e7ad7416f2770..1d0a9967600ed 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 @@ -102,6 +102,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator, private final Sensor restoreRemainingSensor; private final Sensor punctuateLatencySensor; private final Sensor bufferedRecordsSensor; + private final Sensor droppedRecordsSensor; private final Map e2eLatencySensors = new HashMap<>(); private final RecordQueueCreator recordQueueCreator; @@ -162,6 +163,7 @@ public StreamTask(final TaskId id, processLatencySensor = TaskMetrics.processLatencySensor(threadId, taskId, streamsMetrics); punctuateLatencySensor = TaskMetrics.punctuateSensor(threadId, taskId, streamsMetrics); bufferedRecordsSensor = TaskMetrics.activeBufferedRecordsSensor(threadId, taskId, streamsMetrics); + droppedRecordsSensor = TaskMetrics.droppedRecordsSensor(threadId, taskId, streamsMetrics); for (final String terminalNodeName : topology.terminalNodes()) { e2eLatencySensors.put( @@ -925,6 +927,15 @@ public void punctuate(final ProcessorNode node, " continue after a processing error, please set the " + PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " appropriately.", logPrefix, node.name()), e); + } else { + log.warn( + "Skipping punctuation due to processing error. topic=[{}] partition=[{}] offset=[{}]", + recordContext.topic(), + recordContext.partition(), + recordContext.offset(), + e + ); + droppedRecordsSensor.record(); } } finally { processorContext.setCurrentNode(null); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/ProcessingExceptionHandlerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/ProcessingExceptionHandlerTest.java index aa39fc1e7ffb8..fb438ed2f3442 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/ProcessingExceptionHandlerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/ProcessingExceptionHandlerTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.KeyValue; @@ -41,7 +42,8 @@ import java.util.Map; import java.util.Properties; -import static org.junit.Assert.assertArrayEquals; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; @@ -50,6 +52,8 @@ @RunWith(MockitoJUnitRunner.StrictStubs.class) public class ProcessingExceptionHandlerTest { + private final String threadId = Thread.currentThread().getName(); + @Test public void shouldContinueInProcessorOnProcessingRecordAtBeginningExceptions() { final List> events = Arrays.asList( @@ -81,6 +85,12 @@ public void shouldContinueInProcessorOnProcessingRecordAtBeginningExceptions() { assertEquals(expected.size(), processor.theCapturedProcessor().processed().size()); assertIterableEquals(expected, processor.theCapturedProcessor().processed()); + + final MetricName dropTotal = droppedRecordsTotalMetric(); + final MetricName dropRate = droppedRecordsRateMetric(); + + assertEquals(1.0, driver.metrics().get(dropTotal).metricValue()); + assertTrue((Double) driver.metrics().get(dropRate).metricValue() > 0.0); } } @@ -114,7 +124,13 @@ public void shouldContinueInProcessorOnProcessingRecordInMiddleExceptions() { inputTopic.pipeKeyValueList(events, Instant.EPOCH, Duration.ZERO); assertEquals(expected.size(), processor.theCapturedProcessor().processed().size()); - assertArrayEquals(expected.toArray(), processor.theCapturedProcessor().processed().toArray()); + assertIterableEquals(expected, processor.theCapturedProcessor().processed()); + + final MetricName dropTotal = droppedRecordsTotalMetric(); + final MetricName dropRate = droppedRecordsRateMetric(); + + assertEquals(1.0, driver.metrics().get(dropTotal).metricValue()); + assertTrue((Double) driver.metrics().get(dropRate).metricValue() > 0.0); } } @@ -148,7 +164,13 @@ public void shouldContinueInProcessorOnProcessingRecordAtEndExceptions() { inputTopic.pipeKeyValueList(events, Instant.EPOCH, Duration.ZERO); assertEquals(expected.size(), processor.theCapturedProcessor().processed().size()); - assertArrayEquals(expected.toArray(), processor.theCapturedProcessor().processed().toArray()); + assertIterableEquals(expected, processor.theCapturedProcessor().processed()); + + final MetricName dropTotal = droppedRecordsTotalMetric(); + final MetricName dropRate = droppedRecordsRateMetric(); + + assertEquals(1.0, driver.metrics().get(dropTotal).metricValue()); + assertTrue((Double) driver.metrics().get(dropRate).metricValue() > 0.0); } } @@ -181,7 +203,13 @@ public void shouldFailInProcessorOnProcessingRecordAtBeginningExceptions() { assertEquals("Exception should be handled by processing exception handler", exception.getCause().getMessage()); assertEquals(0, processor.theCapturedProcessor().processed().size()); - assertArrayEquals(expected.toArray(), processor.theCapturedProcessor().processed().toArray()); + assertIterableEquals(expected, processor.theCapturedProcessor().processed()); + + final MetricName dropTotal = droppedRecordsTotalMetric(); + final MetricName dropRate = droppedRecordsRateMetric(); + + assertEquals(0.0, driver.metrics().get(dropTotal).metricValue()); + assertEquals(0.0, driver.metrics().get(dropRate).metricValue()); } } @@ -216,7 +244,13 @@ public void shouldFailInProcessorOnProcessingRecordInMiddleExceptions() { assertEquals("Exception should be handled by processing exception handler", exception.getCause().getMessage()); assertEquals(expected.size(), processor.theCapturedProcessor().processed().size()); - assertArrayEquals(expected.toArray(), processor.theCapturedProcessor().processed().toArray()); + assertIterableEquals(expected, processor.theCapturedProcessor().processed()); + + final MetricName dropTotal = droppedRecordsTotalMetric(); + final MetricName dropRate = droppedRecordsRateMetric(); + + assertEquals(0.0, driver.metrics().get(dropTotal).metricValue()); + assertEquals(0.0, driver.metrics().get(dropRate).metricValue()); } } @@ -253,7 +287,13 @@ public void shouldFailInProcessorOnProcessingRecordAtEndExceptions() { assertEquals("Exception should be handled by processing exception handler", exception.getCause().getMessage()); assertEquals(expected.size(), processor.theCapturedProcessor().processed().size()); - assertArrayEquals(expected.toArray(), processor.theCapturedProcessor().processed().toArray()); + assertIterableEquals(expected, processor.theCapturedProcessor().processed()); + + final MetricName dropTotal = droppedRecordsTotalMetric(); + final MetricName dropRate = droppedRecordsRateMetric(); + + assertEquals(0.0, driver.metrics().get(dropTotal).metricValue()); + assertEquals(0.0, driver.metrics().get(dropRate).metricValue()); } } @@ -286,9 +326,16 @@ public void shouldContinueOnPunctuateExceptions() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); inputTopic.pipeKeyValueList(events, Instant.EPOCH, Duration.ZERO); + driver.advanceWallClockTime(Duration.ofSeconds(2)); assertEquals(expected.size(), processor.theCapturedProcessor().processed().size()); - assertArrayEquals(expected.toArray(), processor.theCapturedProcessor().processed().toArray()); + assertIterableEquals(expected, processor.theCapturedProcessor().processed()); + + final MetricName dropTotal = droppedRecordsTotalMetric(); + final MetricName dropRate = droppedRecordsRateMetric(); + + assertEquals(1.0, driver.metrics().get(dropTotal).metricValue()); + assertTrue((Double) driver.metrics().get(dropRate).metricValue() > 0.0); } } @@ -328,7 +375,13 @@ public void shouldFailOnPunctuateExceptions() { assertEquals("Exception should be handled by processing exception handler", exception.getCause().getMessage()); assertEquals(expected.size(), processor.theCapturedProcessor().processed().size()); - assertArrayEquals(expected.toArray(), processor.theCapturedProcessor().processed().toArray()); + assertIterableEquals(expected, processor.theCapturedProcessor().processed()); + + final MetricName dropTotal = droppedRecordsTotalMetric(); + final MetricName dropRate = droppedRecordsRateMetric(); + + assertEquals(0.0, driver.metrics().get(dropTotal).metricValue()); + assertEquals(0.0, driver.metrics().get(dropRate).metricValue()); } } @@ -410,6 +463,40 @@ public void configure(final Map configs) { } } + /** + * Metric name for dropped records total. + * + * @return the metric name + */ + private MetricName droppedRecordsTotalMetric() { + return new MetricName( + "dropped-records-total", + "stream-task-metrics", + "The total number of dropped records", + mkMap( + mkEntry("thread-id", threadId), + mkEntry("task-id", "0_0") + ) + ); + } + + /** + * Metric name for dropped records rate. + * + * @return the metric name + */ + private MetricName droppedRecordsRateMetric() { + return new MetricName( + "dropped-records-rate", + "stream-task-metrics", + "The average number of dropped records per second", + mkMap( + mkEntry("thread-id", threadId), + mkEntry("task-id", "0_0") + ) + ); + } + /** * Processor supplier that throws a runtime exception on schedule. * diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java index 47f331246e55c..4abd5f240dae9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java @@ -656,26 +656,8 @@ public void shouldContinueOnProcessingExceptions() { context.forward("key", "value"); - final MetricName dropTotal; - final MetricName dropRate; - dropTotal = new MetricName( - "dropped-records-total", - "stream-task-metrics", - "The total number of dropped records", - mkMap( - mkEntry("thread-id", threadId), - mkEntry("task-id", "0_0") - ) - ); - dropRate = new MetricName( - "dropped-records-rate", - "stream-task-metrics", - "The average number of dropped records per second", - mkMap( - mkEntry("thread-id", threadId), - mkEntry("task-id", "0_0") - ) - ); + final MetricName dropTotal = droppedRecordsTotalMetric(); + final MetricName dropRate = droppedRecordsRateMetric(); assertEquals(1.0, metrics.metrics().get(dropTotal).metricValue()); assertTrue((Double) metrics.metrics().get(dropRate).metricValue() > 0.0); @@ -729,26 +711,8 @@ public void shouldFailOnProcessingExceptions() { + "If you would rather have the streaming pipeline continue after a processing error, " + "please set the processing.exception.handler appropriately.", exception.getMessage()); - final MetricName dropTotal; - final MetricName dropRate; - dropTotal = new MetricName( - "dropped-records-total", - "stream-task-metrics", - "The total number of dropped records", - mkMap( - mkEntry("thread-id", threadId), - mkEntry("task-id", "0_0") - ) - ); - dropRate = new MetricName( - "dropped-records-rate", - "stream-task-metrics", - "The average number of dropped records per second", - mkMap( - mkEntry("thread-id", threadId), - mkEntry("task-id", "0_0") - ) - ); + final MetricName dropTotal = droppedRecordsTotalMetric(); + final MetricName dropRate = droppedRecordsRateMetric(); assertEquals(0.0, metrics.metrics().get(dropTotal).metricValue()); assertEquals(0.0, metrics.metrics().get(dropRate).metricValue()); @@ -781,6 +745,40 @@ public void configure(final Map configs) { }; } + /** + * Metric name for dropped records total. + * + * @return the metric name + */ + private MetricName droppedRecordsTotalMetric() { + return new MetricName( + "dropped-records-total", + "stream-task-metrics", + "The total number of dropped records", + mkMap( + mkEntry("thread-id", threadId), + mkEntry("task-id", "0_0") + ) + ); + } + + /** + * Metric name for dropped records rate. + * + * @return the metric name + */ + private MetricName droppedRecordsRateMetric() { + return new MetricName( + "dropped-records-rate", + "stream-task-metrics", + "The average number of dropped records per second", + mkMap( + mkEntry("thread-id", threadId), + mkEntry("task-id", "0_0") + ) + ); + } + @SuppressWarnings("unchecked") private KeyValueStore keyValueStoreMock() { final KeyValueStore keyValueStoreMock = mock(KeyValueStore.class); From c388a067af83bc10c80fca2c0cceeb8f2fb1c8a6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Sun, 26 May 2024 15:42:42 +0200 Subject: [PATCH 27/28] KAFKA-16448: Move topology test driver to integration tests --- .../apache/kafka/streams/TopologyConfig.java | 8 ++- ...ssingExceptionHandlerIntegrationTest.java} | 56 +++++++++---------- 2 files changed, 33 insertions(+), 31 deletions(-) rename streams/src/test/java/org/apache/kafka/streams/{processor/ProcessingExceptionHandlerTest.java => integration/ProcessingExceptionHandlerIntegrationTest.java} (93%) diff --git a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java index 95b6ddd717727..b333de60bc7ad 100644 --- a/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/TopologyConfig.java @@ -153,7 +153,6 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo this.applicationConfigs = globalAppConfigs; this.topologyOverrides = topologyOverrides; - this.processingExceptionHandler = () -> globalAppConfigs.getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class); if (isTopologyOverride(BUFFERED_RECORDS_PER_PARTITION_CONFIG, topologyOverrides)) { maxBufferedSize = getInt(BUFFERED_RECORDS_PER_PARTITION_CONFIG); @@ -229,6 +228,13 @@ public TopologyConfig(final String topologyName, final StreamsConfig globalAppCo deserializationExceptionHandlerSupplier = () -> globalAppConfigs.getConfiguredInstance(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG, DeserializationExceptionHandler.class); } + if (isTopologyOverride(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, topologyOverrides)) { + processingExceptionHandler = () -> getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class); + log.info("Topology {} is overriding {} to {}", topologyName, PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, getClass(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG)); + } else { + processingExceptionHandler = () -> globalAppConfigs.getConfiguredInstance(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ProcessingExceptionHandler.class); + } + if (isTopologyOverride(DEFAULT_DSL_STORE_CONFIG, topologyOverrides)) { storeType = getString(DEFAULT_DSL_STORE_CONFIG); log.info("Topology {} is overriding {} to {}", topologyName, DEFAULT_DSL_STORE_CONFIG, storeType); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/ProcessingExceptionHandlerTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java similarity index 93% rename from streams/src/test/java/org/apache/kafka/streams/processor/ProcessingExceptionHandlerTest.java rename to streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java index fb438ed2f3442..32b5512b1fa47 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/ProcessingExceptionHandlerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.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; +package org.apache.kafka.streams.integration; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.serialization.Serdes; @@ -22,17 +22,20 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.KeyValueTimestamp; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.errors.ProcessingExceptionHandler; import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.ErrorHandlerContext; +import org.apache.kafka.streams.processor.PunctuationType; +import org.apache.kafka.streams.processor.api.ContextualProcessor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.test.IntegrationTest; import org.apache.kafka.test.MockProcessorSupplier; import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.junit.MockitoJUnitRunner; +import org.junit.experimental.categories.Category; import java.time.Duration; import java.time.Instant; @@ -50,11 +53,11 @@ import static org.junit.Assert.assertTrue; import static org.junit.jupiter.api.Assertions.assertIterableEquals; -@RunWith(MockitoJUnitRunner.StrictStubs.class) -public class ProcessingExceptionHandlerTest { +@Category(IntegrationTest.class) +public class ProcessingExceptionHandlerIntegrationTest { private final String threadId = Thread.currentThread().getName(); - @Test + @org.junit.Test public void shouldContinueInProcessorOnProcessingRecordAtBeginningExceptions() { final List> events = Arrays.asList( new KeyValue<>("ID123-1", "ID123-A1"), @@ -77,7 +80,7 @@ public void shouldContinueInProcessorOnProcessingRecordAtBeginningExceptions() { .process(processor); final Properties properties = new Properties(); - properties.put("processing.exception.handler", ContinueProcessingExceptionHandlerMockTest.class); + properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ContinueProcessingExceptionHandlerMockTest.class); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); @@ -94,7 +97,7 @@ public void shouldContinueInProcessorOnProcessingRecordAtBeginningExceptions() { } } - @Test + @org.junit.Test public void shouldContinueInProcessorOnProcessingRecordInMiddleExceptions() { final List> events = Arrays.asList( new KeyValue<>("ID123-2", "ID123-A2"), @@ -117,7 +120,7 @@ public void shouldContinueInProcessorOnProcessingRecordInMiddleExceptions() { .process(processor); final Properties properties = new Properties(); - properties.put("processing.exception.handler", ContinueProcessingExceptionHandlerMockTest.class); + properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ContinueProcessingExceptionHandlerMockTest.class); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); @@ -134,7 +137,7 @@ public void shouldContinueInProcessorOnProcessingRecordInMiddleExceptions() { } } - @Test + @org.junit.Test public void shouldContinueInProcessorOnProcessingRecordAtEndExceptions() { final List> events = Arrays.asList( new KeyValue<>("ID123-2", "ID123-A2"), @@ -157,7 +160,7 @@ public void shouldContinueInProcessorOnProcessingRecordAtEndExceptions() { .process(processor); final Properties properties = new Properties(); - properties.put("processing.exception.handler", ContinueProcessingExceptionHandlerMockTest.class); + properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ContinueProcessingExceptionHandlerMockTest.class); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); @@ -174,7 +177,7 @@ public void shouldContinueInProcessorOnProcessingRecordAtEndExceptions() { } } - @Test + @org.junit.Test public void shouldFailInProcessorOnProcessingRecordAtBeginningExceptions() { final List> events = Arrays.asList( new KeyValue<>("ID123-1", "ID123-A1"), @@ -193,7 +196,7 @@ public void shouldFailInProcessorOnProcessingRecordAtBeginningExceptions() { .process(processor); final Properties properties = new Properties(); - properties.put("processing.exception.handler", FailProcessingExceptionHandlerMockTest.class); + properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, FailProcessingExceptionHandlerMockTest.class); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); @@ -213,7 +216,7 @@ public void shouldFailInProcessorOnProcessingRecordAtBeginningExceptions() { } } - @Test + @org.junit.Test public void shouldFailInProcessorOnProcessingRecordInMiddleExceptions() { final List> events = Arrays.asList( new KeyValue<>("ID123-2", "ID123-A2"), @@ -234,7 +237,7 @@ public void shouldFailInProcessorOnProcessingRecordInMiddleExceptions() { .process(processor); final Properties properties = new Properties(); - properties.put("processing.exception.handler", FailProcessingExceptionHandlerMockTest.class); + properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, FailProcessingExceptionHandlerMockTest.class); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); @@ -254,7 +257,7 @@ public void shouldFailInProcessorOnProcessingRecordInMiddleExceptions() { } } - @Test + @org.junit.Test public void shouldFailInProcessorOnProcessingRecordAtEndExceptions() { final List> events = Arrays.asList( new KeyValue<>("ID123-2", "ID123-A2"), @@ -277,7 +280,7 @@ public void shouldFailInProcessorOnProcessingRecordAtEndExceptions() { .process(processor); final Properties properties = new Properties(); - properties.put("processing.exception.handler", FailProcessingExceptionHandlerMockTest.class); + properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, FailProcessingExceptionHandlerMockTest.class); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); @@ -297,7 +300,7 @@ public void shouldFailInProcessorOnProcessingRecordAtEndExceptions() { } } - @Test + @org.junit.Test public void shouldContinueOnPunctuateExceptions() { final List> events = Arrays.asList( new KeyValue<>("ID123-1", "ID123-A1"), @@ -321,7 +324,7 @@ public void shouldContinueOnPunctuateExceptions() { .process(processor); final Properties properties = new Properties(); - properties.put("processing.exception.handler", ContinuePunctuateProcessingExceptionHandlerMockTest.class); + properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ContinuePunctuateProcessingExceptionHandlerMockTest.class); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); @@ -361,7 +364,7 @@ public void shouldFailOnPunctuateExceptions() { .process(processor); final Properties properties = new Properties(); - properties.put("processing.exception.handler", FailPunctuateProcessingExceptionHandlerMockTest.class); + properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, FailPunctuateProcessingExceptionHandlerMockTest.class); try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); @@ -527,21 +530,14 @@ public void process(final Record record) { * @return the processor supplier */ private ProcessorSupplier runtimeErrorProcessorSupplierMock() { - return () -> new Processor() { - org.apache.kafka.streams.processor.api.ProcessorContext context; - - @Override - public void init(final org.apache.kafka.streams.processor.api.ProcessorContext context) { - this.context = context; - } - + return () -> new ContextualProcessor() { @Override public void process(final Record record) { if (record.key().equals("ID123-1")) { throw new RuntimeException("Exception should be handled by processing exception handler"); } - context.forward(new Record<>(record.key(), record.value(), record.timestamp())); + context().forward(new Record<>(record.key(), record.value(), record.timestamp())); } }; } From f011b32faadf1d6e9b7b7815f61510a3ddb10ecc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lo=C3=AFc=20Greffier?= Date: Mon, 27 May 2024 14:14:43 +0200 Subject: [PATCH 28/28] KAFKA-16448: Pass source node id in deserialization handler --- .../processor/internals/GlobalStateManagerImpl.java | 3 ++- .../streams/processor/internals/RecordDeserializer.java | 7 ++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java index d42e90bf33f4c..76a09985218af 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java @@ -329,7 +329,8 @@ private void reprocessState(final List topicPartitions, Thread.currentThread().getName(), globalProcessorContext.taskId().toString(), globalProcessorContext.metrics() - ) + ), + null ); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 9068d8ac4afde..3cd46ca143820 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 @@ -68,7 +68,7 @@ ConsumerRecord deserialize(final ProcessorContext processo Optional.empty() ); } catch (final Exception deserializationException) { - handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor); + handleDeserializationFailure(deserializationExceptionHandler, processorContext, deserializationException, rawRecord, log, droppedRecordsSensor, sourceNode.name()); return null; // 'handleDeserializationFailure' would either throw or swallow -- if we swallow we need to skip the record by returning 'null' } } @@ -78,7 +78,8 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa final Exception deserializationException, final ConsumerRecord rawRecord, final Logger log, - final Sensor droppedRecordsSensor) { + final Sensor droppedRecordsSensor, + final String sourceNodeName) { final DeserializationExceptionHandler.DeserializationHandlerResponse response; try { final ErrorHandlerContextImpl errorHandlerContext = new ErrorHandlerContextImpl( @@ -89,7 +90,7 @@ public static void handleDeserializationFailure(final DeserializationExceptionHa rawRecord.headers(), rawRecord.key(), rawRecord.value(), - null, + sourceNodeName, processorContext.taskId()); response = deserializationExceptionHandler.handle(errorHandlerContext, rawRecord, deserializationException); } catch (final Exception fatalUserException) {