From 69e99e192ee6e2a1811dc375e169a23a97f51c84 Mon Sep 17 00:00:00 2001 From: Philipp Schirmer Date: Wed, 5 Oct 2022 09:02:10 +0200 Subject: [PATCH 01/10] Fix badge for Maven Central (#17) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 3fc9acb..5f92877 100644 --- a/README.md +++ b/README.md @@ -1,7 +1,7 @@ [![Build Status](https://dev.azure.com/bakdata/public/_apis/build/status/bakdata.kafka-error-handling?branchName=master)](https://dev.azure.com/bakdata/public/_build/latest?definitionId=23&branchName=master) [![Sonarcloud status](https://sonarcloud.io/api/project_badges/measure?project=com.bakdata.kafka%3Aerror-handling&metric=alert_status)](https://sonarcloud.io/dashboard?id=com.bakdata.kafka%3Aerror-handling) [![Code coverage](https://sonarcloud.io/api/project_badges/measure?project=com.bakdata.kafka%3Aerror-handling&metric=coverage)](https://sonarcloud.io/dashboard?id=com.bakdata.kafka%3Aerror-handling) -[![Maven](https://img.shields.io/maven-central/v/com.bakdata.kafka/error-handling.svg)](https://search.maven.org/search?q=g:com.bakdata.kafka%20AND%20a:error-handling&core=gav) +[![Maven](https://img.shields.io/maven-central/v/com.bakdata.kafka/error-handling-core.svg)](https://search.maven.org/search?q=g:com.bakdata.kafka%20AND%20a:error-handling-core&core=gav) # Kafka error handling Libraries for error handling in [Kafka Streams](https://kafka.apache.org/documentation/streams/). From da0b4db539b3d16a9dc8a277dce6f873efea9e96 Mon Sep 17 00:00:00 2001 From: Philipp Schirmer Date: Wed, 28 Dec 2022 12:14:40 +0100 Subject: [PATCH 02/10] Implement new Kafka 3.3 APIs (#18) --- error-handling-avro/lombok.config | 1 + .../kafka/AvroDeadLetterConverter.java | 29 ++ .../kafka/AvroDeadLetterConverterTest.java | 4 +- .../kafka/AvroDeadLetterProcessorTest.java | 136 ++++++ error-handling-core/lombok.config | 1 + .../bakdata/kafka/DeadLetterProcessor.java | 110 +++++ .../bakdata/kafka/DeadLetterTransformer.java | 5 +- .../kafka/DecoratorProcessingContext.java | 39 ++ .../com/bakdata/kafka/DecoratorProcessor.java | 59 +++ .../kafka/DecoratorValueProcessor.java | 58 +++ .../ErrorCapturingApiProcessorContext.java | 60 +++ ...rrorCapturingFixedKeyProcessorContext.java | 60 +++ .../kafka/ErrorCapturingFlatTransformer.java | 2 + .../ErrorCapturingFlatValueTransformer.java | 4 +- ...rCapturingFlatValueTransformerWithKey.java | 4 +- .../kafka/ErrorCapturingProcessor.java | 176 +++++++ .../kafka/ErrorCapturingTransformer.java | 4 +- .../kafka/ErrorCapturingValueProcessor.java | 168 +++++++ .../kafka/ErrorCapturingValueTransformer.java | 4 +- ...ErrorCapturingValueTransformerWithKey.java | 4 +- .../kafka/ErrorDescribingProcessor.java | 109 +++++ .../kafka/ErrorDescribingTransformer.java | 4 +- .../kafka/ErrorDescribingValueProcessor.java | 106 +++++ .../ErrorDescribingValueTransformer.java | 4 +- ...rrorDescribingValueTransformerWithKey.java | 4 +- .../bakdata/kafka/ErrorHeaderProcessor.java | 147 ++++++ .../bakdata/kafka/ErrorHeaderTransformer.java | 26 +- .../kafka/ErrorLoggingFlatKeyValueMapper.java | 4 +- .../kafka/ErrorLoggingFlatTransformer.java | 2 + .../kafka/ErrorLoggingFlatValueMapper.java | 4 +- .../ErrorLoggingFlatValueMapperWithKey.java | 4 +- .../ErrorLoggingFlatValueTransformer.java | 4 +- ...rorLoggingFlatValueTransformerWithKey.java | 4 +- .../kafka/ErrorLoggingKeyValueMapper.java | 4 +- .../bakdata/kafka/ErrorLoggingProcessor.java | 169 +++++++ .../kafka/ErrorLoggingTransformer.java | 6 +- .../kafka/ErrorLoggingValueMapper.java | 4 +- .../kafka/ErrorLoggingValueMapperWithKey.java | 4 +- .../kafka/ErrorLoggingValueProcessor.java | 164 +++++++ .../kafka/ErrorLoggingValueTransformer.java | 6 +- .../ErrorLoggingValueTransformerWithKey.java | 6 +- .../java/com/bakdata/kafka/ErrorUtil.java | 18 +- .../com/bakdata/kafka/SuccessKeyValue.java | 6 +- .../java/com/bakdata/kafka/SuccessValue.java | 6 +- ...pturingFlatKeyValueMapperTopologyTest.java | 8 +- ...rCapturingFlatTransformerTopologyTest.java | 4 +- ...rCapturingFlatValueMapperTopologyTest.java | 5 +- ...ingFlatValueMapperWithKeyTopologyTest.java | 5 +- ...uringFlatValueTransformerTopologyTest.java | 5 +- ...atValueTransformerWithKeyTopologyTest.java | 5 +- ...orCapturingKeyValueMapperTopologyTest.java | 5 +- .../ErrorCapturingProcessorTopologyTest.java | 450 ++++++++++++++++++ ...ErrorCapturingTransformerTopologyTest.java | 5 +- ...ErrorCapturingValueMapperTopologyTest.java | 5 +- ...pturingValueMapperWithKeyTopologyTest.java | 5 +- ...orCapturingValueProcessorTopologyTest.java | 387 +++++++++++++++ ...CapturingValueTransformerTopologyTest.java | 5 +- ...ngValueTransformerWithKeyTopologyTest.java | 5 +- .../ErrorDescribingProcessorTopologyTest.java | 302 ++++++++++++ ...rDescribingValueProcessorTopologyTest.java | 253 ++++++++++ .../ErrorHeaderProcessorTopologyTest.java | 185 +++++++ .../ErrorLoggingProcessorTopologyTest.java | 373 +++++++++++++++ ...rrorLoggingValueProcessorTopologyTest.java | 318 +++++++++++++ .../kafka/ErrorCaptureTopologyTest.java | 8 +- error-handling-proto/lombok.config | 1 + .../kafka/ProtoDeadLetterConverter.java | 29 ++ .../kafka/ProtoDeadLetterProcessorTest.java | 170 +++++++ gradle.properties | 6 +- 68 files changed, 4193 insertions(+), 94 deletions(-) create mode 100644 error-handling-avro/src/test/java/com/bakdata/kafka/AvroDeadLetterProcessorTest.java create mode 100644 error-handling-core/src/main/java/com/bakdata/kafka/DeadLetterProcessor.java create mode 100644 error-handling-core/src/main/java/com/bakdata/kafka/DecoratorProcessingContext.java create mode 100644 error-handling-core/src/main/java/com/bakdata/kafka/DecoratorProcessor.java create mode 100644 error-handling-core/src/main/java/com/bakdata/kafka/DecoratorValueProcessor.java create mode 100644 error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingApiProcessorContext.java create mode 100644 error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFixedKeyProcessorContext.java create mode 100644 error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingProcessor.java create mode 100644 error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingValueProcessor.java create mode 100644 error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingProcessor.java create mode 100644 error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingValueProcessor.java create mode 100644 error-handling-core/src/main/java/com/bakdata/kafka/ErrorHeaderProcessor.java create mode 100644 error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingProcessor.java create mode 100644 error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueProcessor.java create mode 100644 error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingProcessorTopologyTest.java create mode 100644 error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueProcessorTopologyTest.java create mode 100644 error-handling-core/src/test/java/com/bakdata/kafka/ErrorDescribingProcessorTopologyTest.java create mode 100644 error-handling-core/src/test/java/com/bakdata/kafka/ErrorDescribingValueProcessorTopologyTest.java create mode 100644 error-handling-core/src/test/java/com/bakdata/kafka/ErrorHeaderProcessorTopologyTest.java create mode 100644 error-handling-core/src/test/java/com/bakdata/kafka/ErrorLoggingProcessorTopologyTest.java create mode 100644 error-handling-core/src/test/java/com/bakdata/kafka/ErrorLoggingValueProcessorTopologyTest.java create mode 100644 error-handling-proto/src/test/java/com/bakdata/kafka/ProtoDeadLetterProcessorTest.java diff --git a/error-handling-avro/lombok.config b/error-handling-avro/lombok.config index 6aa51d7..189c0be 100644 --- a/error-handling-avro/lombok.config +++ b/error-handling-avro/lombok.config @@ -1,2 +1,3 @@ # This file is generated by the 'io.freefair.lombok' Gradle plugin config.stopBubbling = true +lombok.addLombokGeneratedAnnotation = true diff --git a/error-handling-avro/src/main/java/com/bakdata/kafka/AvroDeadLetterConverter.java b/error-handling-avro/src/main/java/com/bakdata/kafka/AvroDeadLetterConverter.java index 20e6c63..6103040 100644 --- a/error-handling-avro/src/main/java/com/bakdata/kafka/AvroDeadLetterConverter.java +++ b/error-handling-avro/src/main/java/com/bakdata/kafka/AvroDeadLetterConverter.java @@ -25,6 +25,7 @@ package com.bakdata.kafka; import org.apache.kafka.streams.kstream.ValueTransformerSupplier; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier; /** * Convert a {@code DeadLetterDescription} to an Avro {@code DeadLetter} @@ -66,10 +67,38 @@ public DeadLetter convert(final DeadLetterDescription deadLetterDescription) { * @param description shared description for all errors * @param type of the input value * @return a transformer supplier + * @deprecated Use {@link #asProcessor(String)} */ + @Deprecated(since = "1.4.0") public static ValueTransformerSupplier, DeadLetter> asTransformer( final String description) { return DeadLetterTransformer.create(description, new AvroDeadLetterConverter()); } + /** + * Creates a processor that uses the AvroDeadLetterConverter + * + *
{@code
+     * // Example, this works for all error capturing topologies
+     * final KeyValueMapper> mapper = ...;
+     * final KStream input = ...;
+     * final KStream> processed = input.map(captureErrors(mapper));
+     * final KStream output = processed.flatMapValues(ProcessedKeyValue::getValues);
+     * final KStream> errors = processed.flatMap(ProcessedKeyValue::getErrors);
+     * final KStream deadLetters = errors.processValues(
+     *                      AvroDeadLetterConverter.asProcessor("Description"));
+     * deadLetters.to(ERROR_TOPIC);
+     * }
+     * 
+ * + * @param description shared description for all errors + * @param type of the input key + * @param type of the input value + * @return a processor supplier + */ + public static FixedKeyProcessorSupplier, DeadLetter> asProcessor( + final String description) { + return DeadLetterProcessor.create(description, new AvroDeadLetterConverter()); + } + } diff --git a/error-handling-avro/src/test/java/com/bakdata/kafka/AvroDeadLetterConverterTest.java b/error-handling-avro/src/test/java/com/bakdata/kafka/AvroDeadLetterConverterTest.java index 60c932c..e64fb6b 100644 --- a/error-handling-avro/src/test/java/com/bakdata/kafka/AvroDeadLetterConverterTest.java +++ b/error-handling-avro/src/test/java/com/bakdata/kafka/AvroDeadLetterConverterTest.java @@ -37,7 +37,7 @@ class AvroDeadLetterConverterTest { private SoftAssertions softly; @Test - void shouldConvertDeadletterDescriptionWithOptionalFields() { + void shouldConvertDeadLetterDescriptionWithOptionalFields() { final AvroDeadLetterConverter converter = new AvroDeadLetterConverter(); final DeadLetterDescription deadLetterDescription = DeadLetterDescription.builder() .inputValue("inputValue") @@ -63,7 +63,7 @@ void shouldConvertDeadletterDescriptionWithOptionalFields() { } @Test - void shouldConvertDeadletterDescriptionWithoutOptionalFields() { + void shouldConvertDeadLetterDescriptionWithoutOptionalFields() { final AvroDeadLetterConverter converter = new AvroDeadLetterConverter(); final DeadLetterDescription onlyRequiredFieldsDeadLetterDescription = DeadLetterDescription.builder() .description("description") diff --git a/error-handling-avro/src/test/java/com/bakdata/kafka/AvroDeadLetterProcessorTest.java b/error-handling-avro/src/test/java/com/bakdata/kafka/AvroDeadLetterProcessorTest.java new file mode 100644 index 0000000..65275b0 --- /dev/null +++ b/error-handling-avro/src/test/java/com/bakdata/kafka/AvroDeadLetterProcessorTest.java @@ -0,0 +1,136 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.when; + +import io.confluent.kafka.streams.serdes.avro.SpecificAvroSerde; +import java.util.Arrays; +import java.util.List; +import java.util.Properties; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Produced; +import org.assertj.core.api.SoftAssertions; +import org.assertj.core.api.junit.jupiter.InjectSoftAssertions; +import org.assertj.core.api.junit.jupiter.SoftAssertionsExtension; +import org.jooq.lambda.Seq; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; + +@ExtendWith(MockitoExtension.class) +@ExtendWith(SoftAssertionsExtension.class) +@MockitoSettings(strictness = Strictness.STRICT_STUBS) +class AvroDeadLetterProcessorTest extends ErrorCaptureTopologyTest { + private static final String ERROR_TOPIC = "errors"; + private static final String OUTPUT_TOPIC = "output"; + private static final String INPUT_TOPIC = "input"; + private static final Serde STRING_SERDE = Serdes.String(); + private static final String DEAD_LETTER_DESCRIPTION = "Description"; + private static final String ERROR_MESSAGE = "ERROR!"; + @InjectSoftAssertions + private SoftAssertions softly; + @Mock + private KeyValueMapper> mapper; + + @Override + protected void buildTopology(final StreamsBuilder builder) { + final KStream input = builder.stream(INPUT_TOPIC, Consumed.with(null, STRING_SERDE)); + final KStream> mapped = + input.map(ErrorCapturingKeyValueMapper.captureErrors(this.mapper)); + mapped.flatMapValues(ProcessedKeyValue::getValues) + .to(OUTPUT_TOPIC, Produced.valueSerde(STRING_SERDE)); + mapped.flatMap(ProcessedKeyValue::getErrors) + .processValues(AvroDeadLetterConverter.asProcessor(DEAD_LETTER_DESCRIPTION)) + .to(ERROR_TOPIC); + } + + @Override + protected Properties getKafkaProperties() { + final Properties kafkaProperties = super.getKafkaProperties(); + kafkaProperties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, SpecificAvroSerde.class); + return kafkaProperties; + } + + @Test + void shouldConvertAndSerializeAvroDeadLetter() { + when(this.mapper.apply(any(), any())).thenThrow(new RuntimeException(ERROR_MESSAGE)); + this.createTopology(); + this.topology.input(INPUT_TOPIC).withValueSerde(STRING_SERDE) + .add(1, "foo") + .add(2, "bar"); + + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withValueSerde(STRING_SERDE)) + .toList(); + this.softly.assertThat(records) + .isEmpty(); + + final List> errors = Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueType(DeadLetter.class)) + .toList(); + + this.softly.assertThat(errors) + .hasSize(2) + .extracting(ProducerRecord::value).allSatisfy( + deadLetter -> { + this.softly.assertThat(deadLetter.getDescription()).isEqualTo(DEAD_LETTER_DESCRIPTION); + this.softly.assertThat(deadLetter.getCause().getMessage()).hasValue(ERROR_MESSAGE); + this.softly.assertThat(deadLetter.getCause().getErrorClass()) + .hasValue(RuntimeException.class.getCanonicalName()); + // We don't check the exact stack trace, but only that it consists of multiple lines + this.softly.assertThat(deadLetter.getCause().getStackTrace()).map(s -> Arrays.asList(s.split("\n"))) + .get().asList().hasSizeGreaterThan(1); + this.softly.assertThat(deadLetter.getTopic()).hasValue(INPUT_TOPIC); + this.softly.assertThat(deadLetter.getPartition()).hasValue(0); + } + ); + this.softly.assertThat(errors).extracting(ProducerRecord::value).element(0).satisfies( + deadLetter -> { + this.softly.assertThat(deadLetter.getInputValue()).hasValue("foo"); + this.softly.assertThat(deadLetter.getOffset()).hasValue(0L); + } + ); + this.softly.assertThat(errors).extracting(ProducerRecord::value).element(1).satisfies( + deadLetter -> { + this.softly.assertThat(deadLetter.getInputValue()).hasValue("bar"); + this.softly.assertThat(deadLetter.getOffset()).hasValue(1L); + } + ); + + } +} diff --git a/error-handling-core/lombok.config b/error-handling-core/lombok.config index 6aa51d7..189c0be 100644 --- a/error-handling-core/lombok.config +++ b/error-handling-core/lombok.config @@ -1,2 +1,3 @@ # This file is generated by the 'io.freefair.lombok' Gradle plugin config.stopBubbling = true +lombok.addLombokGeneratedAnnotation = true diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/DeadLetterProcessor.java b/error-handling-core/src/main/java/com/bakdata/kafka/DeadLetterProcessor.java new file mode 100644 index 0000000..90b90e0 --- /dev/null +++ b/error-handling-core/src/main/java/com/bakdata/kafka/DeadLetterProcessor.java @@ -0,0 +1,110 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import java.util.Optional; +import lombok.Getter; +import lombok.NonNull; +import lombok.RequiredArgsConstructor; +import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.kafka.streams.processor.api.FixedKeyProcessor; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier; +import org.apache.kafka.streams.processor.api.FixedKeyRecord; +import org.apache.kafka.streams.processor.api.RecordMetadata; + +/** + * {@link FixedKeyProcessor} that creates a {@code DeadLetter} from a processing error. + * + * @param type of key + * @param type of value + * @param the DeadLetter type + */ +@Getter +@RequiredArgsConstructor +public class DeadLetterProcessor implements FixedKeyProcessor, T> { + private final @NonNull String description; + private final @NonNull DeadLetterConverter deadLetterConverter; + private FixedKeyProcessorContext context; + + /** + * Transforms captured errors for serialization + * + *
{@code
+     * // Example, this works for all error capturing topologies
+     * final KeyValueMapper> mapper = ...;
+     * final KStream input = ...;
+     * final KStream> processed = input.map(captureErrors(mapper));
+     * final KStream output = processed.flatMapValues(ProcessedKeyValue::getValues);
+     * final KStream> errors = processed.flatMap(ProcessedKeyValue::getErrors);
+     * final DeadLetterConverter deadLetterConverter = ...
+     * final KStream deadLetters = errors.processValues(
+     *                      DeadLetterProcessor.create("Description", deadLetterConverter));
+     * deadLetters.to(ERROR_TOPIC);
+     * }
+     * 
+ * + * @param description shared description for all errors + * @param deadLetterConverter converter from DeadLetterDescriptions to VR + * @param type of the input key + * @param type of the input value + * @param type of the output value + * @return a processor supplier + */ + public static FixedKeyProcessorSupplier, VR> create(final String description, + final DeadLetterConverter deadLetterConverter) { + return () -> new DeadLetterProcessor<>(description, deadLetterConverter); + } + + @Override + public void init(final FixedKeyProcessorContext context) { + this.context = context; + } + + @Override + public void process(final FixedKeyRecord> inputRecord) { + final ProcessingError error = inputRecord.value(); + final Throwable throwable = error.getThrowable(); + final Optional metadata = this.context.recordMetadata(); + final DeadLetterDescription deadLetterDescription = DeadLetterDescription.builder() + .inputValue(Optional.ofNullable(error.getValue()).map(ErrorUtil::toString).orElse(null)) + .cause(DeadLetterDescription.Cause.builder() + .message(throwable.getMessage()) + .stackTrace(ExceptionUtils.getStackTrace(throwable)) + .errorClass(throwable.getClass().getName()) + .build()) + .description(this.description) + .topic(metadata.map(RecordMetadata::topic).orElse(null)) + .partition(metadata.map(RecordMetadata::partition).orElse(null)) + .offset(metadata.map(RecordMetadata::offset).orElse(null)) + .build(); + this.context.forward(inputRecord.withValue(this.deadLetterConverter.convert(deadLetterDescription))); + } + + @Override + public void close() { + // do nothing + } +} diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/DeadLetterTransformer.java b/error-handling-core/src/main/java/com/bakdata/kafka/DeadLetterTransformer.java index c243ba8..c47b1ad 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/DeadLetterTransformer.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/DeadLetterTransformer.java @@ -38,13 +38,15 @@ * * @param type of value * @param the DeadLetter type + * @deprecated Use {@link DeadLetterProcessor} */ @Getter @RequiredArgsConstructor +@Deprecated(since = "1.4.0") public class DeadLetterTransformer implements ValueTransformer, T> { private final @NonNull String description; private final @NonNull DeadLetterConverter deadLetterConverter; - private ProcessorContext context = null; + private ProcessorContext context; /** * Transforms captured errors for serialization @@ -66,6 +68,7 @@ public class DeadLetterTransformer implements ValueTransformer type of the input value + * @param type of the output value * @return a transformer supplier */ public static ValueTransformerSupplier, VR> create(final String description, diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/DecoratorProcessingContext.java b/error-handling-core/src/main/java/com/bakdata/kafka/DecoratorProcessingContext.java new file mode 100644 index 0000000..2fbb216 --- /dev/null +++ b/error-handling-core/src/main/java/com/bakdata/kafka/DecoratorProcessingContext.java @@ -0,0 +1,39 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import lombok.NonNull; +import lombok.RequiredArgsConstructor; +import lombok.experimental.Delegate; +import org.apache.kafka.streams.processor.api.ProcessingContext; + +/** + * Base class for decorating a {@code ProcessingContext} + */ +@RequiredArgsConstructor +public abstract class DecoratorProcessingContext implements ProcessingContext { + @Delegate + private final @NonNull ProcessingContext wrapped; +} diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/DecoratorProcessor.java b/error-handling-core/src/main/java/com/bakdata/kafka/DecoratorProcessor.java new file mode 100644 index 0000000..eff9c92 --- /dev/null +++ b/error-handling-core/src/main/java/com/bakdata/kafka/DecoratorProcessor.java @@ -0,0 +1,59 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import lombok.NonNull; +import lombok.RequiredArgsConstructor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; + +/** + * Base class for decorating a {@code Processor} + * + * @param type of input keys + * @param type of input values + * @param type of output keys + * @param type of output values + */ +@RequiredArgsConstructor +public abstract class DecoratorProcessor implements Processor { + private final @NonNull Processor wrapped; + + @Override + public void close() { + this.wrapped.close(); + } + + @Override + public void process(final Record inputRecord) { + this.wrapped.process(inputRecord); + } + + @Override + public void init(final ProcessorContext context) { + this.wrapped.init(context); + } +} diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/DecoratorValueProcessor.java b/error-handling-core/src/main/java/com/bakdata/kafka/DecoratorValueProcessor.java new file mode 100644 index 0000000..6800e36 --- /dev/null +++ b/error-handling-core/src/main/java/com/bakdata/kafka/DecoratorValueProcessor.java @@ -0,0 +1,58 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import lombok.NonNull; +import lombok.RequiredArgsConstructor; +import org.apache.kafka.streams.processor.api.FixedKeyProcessor; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext; +import org.apache.kafka.streams.processor.api.FixedKeyRecord; + +/** + * Base class for decorating a {@code ValueTransformerWithKey} + * + * @param type of input keys + * @param type of input values + * @param type of output values + */ +@RequiredArgsConstructor +public abstract class DecoratorValueProcessor implements FixedKeyProcessor { + private final @NonNull FixedKeyProcessor wrapped; + + @Override + public void close() { + this.wrapped.close(); + } + + @Override + public void process(final FixedKeyRecord inputRecord) { + this.wrapped.process(inputRecord); + } + + @Override + public void init(final FixedKeyProcessorContext context) { + this.wrapped.init(context); + } +} diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingApiProcessorContext.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingApiProcessorContext.java new file mode 100644 index 0000000..a542c63 --- /dev/null +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingApiProcessorContext.java @@ -0,0 +1,60 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import lombok.NonNull; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; + +final class ErrorCapturingApiProcessorContext extends DecoratorProcessingContext + implements ProcessorContext { + private final @NonNull ProcessorContext> wrapped; + + ErrorCapturingApiProcessorContext( + final @NonNull ProcessorContext> wrapped) { + super(wrapped); + this.wrapped = wrapped; + } + + private static Record> getValue(final Record outputRecord) { + final VR value = outputRecord.value(); + final ProcessedKeyValue recordWithOldKey = SuccessKeyValue.of(value); + return outputRecord.withValue(recordWithOldKey); + } + + @Override + public void forward(final Record outputRecord) { + final Record> recordWithOldKey = getValue(outputRecord); + this.wrapped.forward(recordWithOldKey); + } + + @Override + public void forward(final Record outputRecord, + final String childName) { + final Record> recordWithOldKey = getValue(outputRecord); + this.wrapped.forward(recordWithOldKey, childName); + } +} diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFixedKeyProcessorContext.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFixedKeyProcessorContext.java new file mode 100644 index 0000000..81ea31d --- /dev/null +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFixedKeyProcessorContext.java @@ -0,0 +1,60 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import lombok.NonNull; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext; +import org.apache.kafka.streams.processor.api.FixedKeyRecord; + +final class ErrorCapturingFixedKeyProcessorContext extends DecoratorProcessingContext + implements FixedKeyProcessorContext { + private final @NonNull FixedKeyProcessorContext> wrapped; + + ErrorCapturingFixedKeyProcessorContext( + final @NonNull FixedKeyProcessorContext> wrapped) { + super(wrapped); + this.wrapped = wrapped; + } + + private static FixedKeyRecord> getValue( + final FixedKeyRecord outputRecord) { + final VR value = outputRecord.value(); + final ProcessedValue recordWithOldKey = SuccessValue.of(value); + return outputRecord.withValue(recordWithOldKey); + } + + @Override + public void forward(final FixedKeyRecord outputRecord) { + final FixedKeyRecord> recordWithOldKey = getValue(outputRecord); + this.wrapped.forward(recordWithOldKey); + } + + @Override + public void forward(final FixedKeyRecord outputRecord, + final String childName) { + final FixedKeyRecord> recordWithOldKey = getValue(outputRecord); + this.wrapped.forward(recordWithOldKey, childName); + } +} diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFlatTransformer.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFlatTransformer.java index cb31e87..84d2f8c 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFlatTransformer.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFlatTransformer.java @@ -46,8 +46,10 @@ * @param type of output values * @see #captureErrors(Transformer) * @see #captureErrors(Transformer, Predicate) + * @deprecated Use {@link ErrorCapturingProcessor} */ @RequiredArgsConstructor(access = AccessLevel.PRIVATE) +@Deprecated(since = "1.4.0") public final class ErrorCapturingFlatTransformer implements Transformer>>> { private final @NonNull Transformer>> wrapped; diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformer.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformer.java index 43880e5..23b5a6c 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformer.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformer.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -45,9 +45,11 @@ * @param type of output values * @see #captureErrors(ValueTransformer) * @see #captureErrors(ValueTransformer, Predicate) + * @deprecated Use {@link ErrorCapturingValueProcessor} */ @Slf4j @RequiredArgsConstructor(access = AccessLevel.PRIVATE) +@Deprecated(since = "1.4.0") public final class ErrorCapturingFlatValueTransformer implements ValueTransformer>> { private final @NonNull ValueTransformer> wrapped; diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformerWithKey.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformerWithKey.java index d43a150..16c35d3 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformerWithKey.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformerWithKey.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -44,8 +44,10 @@ * @param type of output values * @see #captureErrors(ValueTransformerWithKey) * @see #captureErrors(ValueTransformerWithKey, Predicate) + * @deprecated Use {@link ErrorCapturingValueProcessor} */ @RequiredArgsConstructor(access = AccessLevel.PRIVATE) +@Deprecated(since = "1.4.0") public final class ErrorCapturingFlatValueTransformerWithKey implements ValueTransformerWithKey>> { private final @NonNull ValueTransformerWithKey> wrapped; diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingProcessor.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingProcessor.java new file mode 100644 index 0000000..178cfd8 --- /dev/null +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingProcessor.java @@ -0,0 +1,176 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import java.util.Set; +import java.util.function.Predicate; +import lombok.AccessLevel; +import lombok.NonNull; +import lombok.RequiredArgsConstructor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.state.StoreBuilder; + +/** + * Wrap a {@code Processor} and capture thrown exceptions. + * + * @param type of input keys + * @param type of input values + * @param type of output keys + * @param type of output values + * @see #captureErrors(Processor) + * @see #captureErrors(Processor, Predicate) + */ +@RequiredArgsConstructor(access = AccessLevel.PRIVATE) +public final class ErrorCapturingProcessor + implements Processor> { + private final @NonNull Processor wrapped; + private final @NonNull Predicate errorFilter; + private ProcessorContext> context; + + /** + * Wrap a {@code Processor} and capture thrown exceptions. Recoverable Kafka exceptions such as a schema registry + * timeout are forwarded and not captured. + * + * @param processor {@code Processor} whose exceptions should be captured + * @param type of input keys + * @param type of input values + * @param type of output keys + * @param type of output values + * @return {@code Processor} + * @see #captureErrors(Processor, Predicate) + * @see ErrorUtil#isRecoverable(Exception) + */ + public static Processor> captureErrors( + final @NonNull Processor processor) { + return captureErrors(processor, ErrorUtil::isRecoverable); + } + + /** + * Wrap a {@code Processor} and capture thrown exceptions. + *
{@code
+     * final KStream input = ...;
+     * final KStream> processed = input.process(() -> captureErrors(new Processor() {...}));
+     * final KStream output = processed.flatMapValues(ProcessedKeyValue::getValues);
+     * final KStream> errors = processed.flatMap(ProcessedKeyValue::getErrors);
+     * }
+     * 
+ * + * @param processor {@code Processor} whose exceptions should be captured + * @param errorFilter expression that filters errors which should be thrown and not captured + * @param type of input keys + * @param type of input values + * @param type of output keys + * @param type of output values + * @return {@code Processor} + */ + public static Processor> captureErrors( + final @NonNull Processor processor, + final @NonNull Predicate errorFilter) { + return new ErrorCapturingProcessor<>(processor, errorFilter); + } + + /** + * Wrap a {@code ProcessorSupplier} and capture thrown exceptions. Recoverable Kafka exceptions such as a schema + * registry timeout are forwarded and not captured. + * + * @param supplier {@code ProcessorSupplier} whose exceptions should be captured + * @param type of input keys + * @param type of input values + * @param type of output keys + * @param type of output values + * @return {@code ProcessorSupplier} + * @see #captureErrors(ProcessorSupplier, Predicate) + * @see ErrorUtil#isRecoverable(Exception) + */ + public static ProcessorSupplier> captureErrors( + final @NonNull ProcessorSupplier supplier) { + return captureErrors(supplier, ErrorUtil::isRecoverable); + } + + /** + * Wrap a {@code ProcessorSupplier} and capture thrown exceptions. + *
{@code
+     * final ProcessorSupplier processor = ...;
+     * final KStream input = ...;
+     * final KStream> processed = input.process(captureErrors(processor));
+     * final KStream output = processed.flatMapValues(ProcessedKeyValue::getValues);
+     * final KStream> errors = processed.flatMap(ProcessedKeyValue::getErrors);
+     * }
+     * 
+ * + * @param supplier {@code ProcessorSupplier} whose exceptions should be captured + * @param errorFilter expression that filters errors which should be thrown and not captured + * @param type of input keys + * @param type of input values + * @param type of output keys + * @param type of output values + * @return {@code ProcessorSupplier} + */ + public static ProcessorSupplier> captureErrors( + final @NonNull ProcessorSupplier supplier, + final @NonNull Predicate errorFilter) { + return new ProcessorSupplier<>() { + @Override + public Set> stores() { + return supplier.stores(); + } + + @Override + public Processor> get() { + return captureErrors(supplier.get(), errorFilter); + } + }; + } + + @Override + public void close() { + this.wrapped.close(); + } + + @Override + public void init(final ProcessorContext> context) { + this.wrapped.init(new ErrorCapturingApiProcessorContext<>(context)); + this.context = context; + } + + @Override + public void process(final Record inputRecord) { + try { + this.wrapped.process(inputRecord); + } catch (final Exception e) { + if (this.errorFilter.test(e)) { + throw e; + } + final ProcessedKeyValue errorWithOldKey = + ErrorKeyValue.of(inputRecord.key(), inputRecord.value(), e); + // new key is only relevant if no error occurs + this.context.forward(inputRecord.withKey(null).withValue(errorWithOldKey)); + } + } + +} diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingTransformer.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingTransformer.java index a0b576b..909cf55 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingTransformer.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingTransformer.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -44,8 +44,10 @@ * @param type of output values * @see #captureErrors(Transformer) * @see #captureErrors(Transformer, Predicate) + * @deprecated Use {@link ErrorCapturingProcessor} */ @RequiredArgsConstructor(access = AccessLevel.PRIVATE) +@Deprecated(since = "1.4.0") public final class ErrorCapturingTransformer implements Transformer>> { private final @NonNull Transformer> wrapped; diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingValueProcessor.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingValueProcessor.java new file mode 100644 index 0000000..8e1349b --- /dev/null +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingValueProcessor.java @@ -0,0 +1,168 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import java.util.Set; +import java.util.function.Predicate; +import lombok.AccessLevel; +import lombok.NonNull; +import lombok.RequiredArgsConstructor; +import org.apache.kafka.streams.processor.api.FixedKeyProcessor; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier; +import org.apache.kafka.streams.processor.api.FixedKeyRecord; +import org.apache.kafka.streams.state.StoreBuilder; + +/** + * Wrap a {@code FixedKeyProcessor} and capture thrown exceptions. + * + * @param type of input keys + * @param type of input values + * @param type of output values + * @see #captureErrors(FixedKeyProcessor) + * @see #captureErrors(FixedKeyProcessor, Predicate) + */ +@RequiredArgsConstructor(access = AccessLevel.PRIVATE) +public final class ErrorCapturingValueProcessor + implements FixedKeyProcessor> { + private final @NonNull FixedKeyProcessor wrapped; + private final @NonNull Predicate errorFilter; + private FixedKeyProcessorContext> context; + + /** + * Wrap a {@code FixedKeyProcessor} and capture thrown exceptions. Recoverable Kafka exceptions such as a schema + * registry timeout are forwarded and not captured. + * + * @param processor {@code FixedKeyProcessor} whose exceptions should be captured + * @param type of input keys + * @param type of input values + * @param type of output values + * @return {@code FixedKeyProcessor} + * @see #captureErrors(FixedKeyProcessor, Predicate) + * @see ErrorUtil#isRecoverable(Exception) + */ + public static FixedKeyProcessor> captureErrors( + final @NonNull FixedKeyProcessor processor) { + return captureErrors(processor, ErrorUtil::isRecoverable); + } + + /** + * Wrap a {@code FixedKeyProcessor} and capture thrown exceptions. + *
{@code
+     * final KStream input = ...;
+     * final KStream> processed = input.processValues(() -> captureErrors(new FixedKeyProcessor() {...}));
+     * final KStream output = processed.flatMapValues(ProcessedValue::getValues);
+     * final KStream> errors = processed.flatMapValues(ProcessedValue::getErrors);
+     * }
+     * 
+ * + * @param processor {@code FixedKeyProcessor} whose exceptions should be captured + * @param errorFilter expression that filters errors which should be thrown and not captured + * @param type of input keys + * @param type of input values + * @param type of output values + * @return {@code FixedKeyProcessor} + */ + public static FixedKeyProcessor> captureErrors( + final @NonNull FixedKeyProcessor processor, + final @NonNull Predicate errorFilter) { + return new ErrorCapturingValueProcessor<>(processor, errorFilter); + } + + /** + * Wrap a {@code FixedKeyProcessorSupplier} and capture thrown exceptions. Recoverable Kafka exceptions such as a + * schema registry timeout are forwarded and not captured. + * + * @param supplier {@code FixedKeyProcessorSupplier} whose exceptions should be captured + * @param type of input keys + * @param type of input values + * @param type of output values + * @return {@code FixedKeyProcessorSupplier} + * @see #captureErrors(FixedKeyProcessorSupplier, Predicate) + * @see ErrorUtil#isRecoverable(Exception) + */ + public static FixedKeyProcessorSupplier> captureErrors( + final @NonNull FixedKeyProcessorSupplier supplier) { + return captureErrors(supplier, ErrorUtil::isRecoverable); + } + + /** + * Wrap a {@code FixedKeyProcessorSupplier} and capture thrown exceptions. + *
{@code
+     * final FixedKeyProcessorSupplier processor = ...;
+     * final KStream input = ...;
+     * final KStream> processed = input.processValues(() -> captureErrors(processor.get()));
+     * final KStream output = processed.flatMapValues(ProcessedValue::getValues);
+     * final KStream> errors = processed.flatMapValues(ProcessedValue::getErrors);
+     * }
+     * 
+ * + * @param supplier {@code FixedKeyProcessorSupplier} whose exceptions should be captured + * @param errorFilter expression that filters errors which should be thrown and not captured + * @param type of input keys + * @param type of input values + * @param type of output values + * @return {@code FixedKeyProcessorSupplier} + */ + public static FixedKeyProcessorSupplier> captureErrors( + final @NonNull FixedKeyProcessorSupplier supplier, + final @NonNull Predicate errorFilter) { + return new FixedKeyProcessorSupplier<>() { + @Override + public Set> stores() { + return supplier.stores(); + } + + @Override + public FixedKeyProcessor> get() { + return captureErrors(supplier.get(), errorFilter); + } + }; + } + + @Override + public void close() { + this.wrapped.close(); + } + + @Override + public void init(final FixedKeyProcessorContext> context) { + this.wrapped.init(new ErrorCapturingFixedKeyProcessorContext<>(context)); + this.context = context; + } + + @Override + public void process(final FixedKeyRecord inputRecord) { + try { + this.wrapped.process(inputRecord); + } catch (final Exception e) { + if (this.errorFilter.test(e)) { + throw e; + } + this.context.forward(inputRecord.withValue(ErrorValue.of(inputRecord.value(), e))); + } + } + +} diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingValueTransformer.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingValueTransformer.java index d10a737..76a6b8d 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingValueTransformer.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingValueTransformer.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -42,9 +42,11 @@ * @param type of output values * @see #captureErrors(ValueTransformer) * @see #captureErrors(ValueTransformer, Predicate) + * @deprecated Use {@link ErrorCapturingValueProcessor} */ @Slf4j @RequiredArgsConstructor(access = AccessLevel.PRIVATE) +@Deprecated(since = "1.4.0") public final class ErrorCapturingValueTransformer implements ValueTransformer> { private final @NonNull ValueTransformer wrapped; private final @NonNull Predicate errorFilter; diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingValueTransformerWithKey.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingValueTransformerWithKey.java index fcc2398..b30c833 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingValueTransformerWithKey.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorCapturingValueTransformerWithKey.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -42,8 +42,10 @@ * @param type of output values * @see #captureErrors(ValueTransformerWithKey) * @see #captureErrors(ValueTransformerWithKey, Predicate) + * @deprecated Use {@link ErrorCapturingValueProcessor} */ @RequiredArgsConstructor(access = AccessLevel.PRIVATE) +@Deprecated(since = "1.4.0") public final class ErrorCapturingValueTransformerWithKey implements ValueTransformerWithKey> { private final @NonNull ValueTransformerWithKey wrapped; diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingProcessor.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingProcessor.java new file mode 100644 index 0000000..e40b7b5 --- /dev/null +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingProcessor.java @@ -0,0 +1,109 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import java.util.Set; +import lombok.NonNull; +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.state.StoreBuilder; + +/** + * Wrap a {@code Processor} and describe thrown exceptions with input key and value. + * + * @param type of input keys + * @param type of input values + * @param type of output keys + * @param type of output values + * @see #describeErrors(Processor) + */ +public final class ErrorDescribingProcessor extends DecoratorProcessor { + + private ErrorDescribingProcessor(final @NonNull Processor wrapped) { + super(wrapped); + } + + /** + * Wrap a {@code Processor} and describe thrown exceptions with input key and value. + *
{@code
+     * final KStream input = ...;
+     * final KStream output = input.process(() -> describeErrors(new Processor() {...}));
+     * }
+     * 
+ * + * @param processor {@code Processor} whose exceptions should be described + * @param type of input keys + * @param type of input values + * @param type of output keys + * @param type of output values + * @return {@code Processor} + */ + public static Processor describeErrors( + final @NonNull Processor processor) { + return new ErrorDescribingProcessor<>(processor); + } + + /** + * Wrap a {@code ProcessorSupplier} and describe thrown exceptions with input key and value. + *
{@code
+     * final ProcessorSupplier processor = ...;
+     * final KStream input = ...;
+     * final KStream output = input.process(describeErrors(processor));
+     * }
+     * 
+ * + * @param supplier {@code ProcessorSupplier} whose exceptions should be described + * @param type of input keys + * @param type of input values + * @param type of output keys + * @param type of output values + * @return {@code ProcessorSupplier} + */ + public static ProcessorSupplier describeErrors( + final @NonNull ProcessorSupplier supplier) { + return new ProcessorSupplier<>() { + @Override + public Set> stores() { + return supplier.stores(); + } + + @Override + public Processor get() { + return describeErrors(supplier.get()); + } + }; + } + + @Override + public void process(final Record inputRecord) { + try { + super.process(inputRecord); + } catch (final Exception e) { + throw new ProcessingException(inputRecord.key(), inputRecord.value(), e); + } + } + +} diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingTransformer.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingTransformer.java index 1805edf..9387e00 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingTransformer.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingTransformer.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -37,7 +37,9 @@ * @param type of input values * @param type of transformation result * @see #describeErrors(Transformer) + * @deprecated Use {@link ErrorDescribingProcessor} */ +@Deprecated(since = "1.4.0") public final class ErrorDescribingTransformer extends DecoratorTransformer { private ErrorDescribingTransformer(final @NonNull Transformer wrapped) { diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingValueProcessor.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingValueProcessor.java new file mode 100644 index 0000000..24c98db --- /dev/null +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingValueProcessor.java @@ -0,0 +1,106 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import java.util.Set; +import lombok.NonNull; +import org.apache.kafka.streams.processor.api.FixedKeyProcessor; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier; +import org.apache.kafka.streams.processor.api.FixedKeyRecord; +import org.apache.kafka.streams.state.StoreBuilder; + +/** + * Wrap a {@code FixedKeyProcessor} and describe thrown exceptions with input key and value. + * + * @param type of input keys + * @param type of input values + * @param type of output values + * @see #describeErrors(FixedKeyProcessor) + */ +public final class ErrorDescribingValueProcessor extends DecoratorValueProcessor { + + private ErrorDescribingValueProcessor(final @NonNull FixedKeyProcessor wrapped) { + super(wrapped); + } + + /** + * Wrap a {@code FixedKeyProcessor} and describe thrown exceptions with input key and value. + *
{@code
+     * final KStream input = ...;
+     * final KStream output = input.processValues(() -> describeErrors(new FixedKeyProcessor() {...}));
+     * }
+     * 
+ * + * @param processor {@code FixedKeyProcessor} whose exceptions should be described + * @param type of input keys + * @param type of input values + * @param type of output values + * @return {@code FixedKeyProcessor} + */ + public static FixedKeyProcessor describeErrors( + final @NonNull FixedKeyProcessor processor) { + return new ErrorDescribingValueProcessor<>(processor); + } + + /** + * Wrap a {@code FixedKeyProcessorSupplier} and describe thrown exceptions with input key and value. + *
{@code
+     * final FixedKeyProcessorSupplier processor = ...;
+     * final KStream input = ...;
+     * final KStream output = input.processValues(describeErrors(processor));
+     * }
+     * 
+ * + * @param supplier {@code FixedKeyProcessorSupplier} whose exceptions should be described + * @param type of input keys + * @param type of input values + * @param type of output values + * @return {@code FixedKeyProcessorSupplier} + */ + public static FixedKeyProcessorSupplier describeErrors( + final @NonNull FixedKeyProcessorSupplier supplier) { + return new FixedKeyProcessorSupplier<>() { + @Override + public Set> stores() { + return supplier.stores(); + } + + @Override + public FixedKeyProcessor get() { + return describeErrors(supplier.get()); + } + }; + } + + @Override + public void process(final FixedKeyRecord inputRecord) { + try { + super.process(inputRecord); + } catch (final Exception e) { + throw new ProcessingException(inputRecord.key(), inputRecord.value(), e); + } + } + +} diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingValueTransformer.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingValueTransformer.java index 38d7c3d..9987424 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingValueTransformer.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingValueTransformer.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -36,7 +36,9 @@ * @param type of input values * @param type of output values * @see #describeErrors(ValueTransformer) + * @deprecated Use {@link ErrorDescribingValueProcessor} */ +@Deprecated(since = "1.4.0") public final class ErrorDescribingValueTransformer extends DecoratorValueTransformer { private ErrorDescribingValueTransformer(final @NonNull ValueTransformer wrapped) { diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingValueTransformerWithKey.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingValueTransformerWithKey.java index 8cba128..6b0ceff 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingValueTransformerWithKey.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorDescribingValueTransformerWithKey.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -37,7 +37,9 @@ * @param type of input values * @param type of output values * @see #describeErrors(ValueTransformerWithKey) + * @deprecated Use {@link ErrorDescribingValueProcessor} */ +@Deprecated(since = "1.4.0") public final class ErrorDescribingValueTransformerWithKey extends DecoratorValueTransformerWithKey { private ErrorDescribingValueTransformerWithKey(final @NonNull ValueTransformerWithKey wrapped) { diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorHeaderProcessor.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorHeaderProcessor.java new file mode 100644 index 0000000..34403b1 --- /dev/null +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorHeaderProcessor.java @@ -0,0 +1,147 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import java.nio.charset.StandardCharsets; +import java.util.Optional; +import lombok.NonNull; +import lombok.RequiredArgsConstructor; +import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.streams.processor.api.FixedKeyProcessor; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier; +import org.apache.kafka.streams.processor.api.FixedKeyRecord; +import org.apache.kafka.streams.processor.api.RecordMetadata; + +/** + * {@link FixedKeyProcessor} that produces a message with the original value and headers detailing the error that has + * been captured. + * + * Headers added by this FixedKeyProcessor: + *
    + *
  • {@code __streams.errors.topic}: original input topic of the erroneous record
  • + *
  • {@code __streams.errors.partition}: original partition of the input topic of the erroneous record
  • + *
  • {@code __streams.errors.offset}: original offset in the partition of the input topic of the erroneous + * record
  • + *
  • {@code __streams.errors.description}: description of the context in which an exception has been thrown
  • + *
  • {@code __streams.errors.exception.class.name}: class of the exception that was captured
  • + *
  • {@code __streams.errors.exception.message}: message of the exception that was captured
  • + *
  • {@code __streams.errors.exception.stack_trace}: stack trace of the exception that was captured
  • + *
+ * + * @param type of key + * @param type of value + */ +@RequiredArgsConstructor +public class ErrorHeaderProcessor implements FixedKeyProcessor, V> { + /** + * Prefix of all headers added by this FixedKeyProcessor + */ + public static final String HEADER_PREFIX = "__streams.errors."; + /** + * Header indicating the original input topic of the erroneous record + */ + public static final String TOPIC = HEADER_PREFIX + "topic"; + /** + * Header indicating the original partition in the input topic of the erroneous record + */ + public static final String PARTITION = HEADER_PREFIX + "partition"; + /** + * Header indicating the original offset in the partition of the input topic of the erroneous record + */ + public static final String OFFSET = HEADER_PREFIX + "offset"; + /** + * Header indicating the description of the context in which an exception has been thrown + */ + public static final String DESCRIPTION = HEADER_PREFIX + "description"; + /** + * Prefix of all headers detailing the error message added by this FixedKeyProcessor + */ + public static final String EXCEPTION_PREFIX = HEADER_PREFIX + "exception."; + /** + * Header indicating the class of the exception that was captured + */ + public static final String EXCEPTION_CLASS_NAME = EXCEPTION_PREFIX + "class.name"; + /** + * Header indicating the message of the exception that was captured + */ + public static final String EXCEPTION_MESSAGE = EXCEPTION_PREFIX + "message"; + /** + * Header indicating the stack trace of the exception that was captured + */ + public static final String EXCEPTION_STACK_TRACE = EXCEPTION_PREFIX + "stack_trace"; + private final @NonNull String description; + private FixedKeyProcessorContext context; + + /** + * Create a new {@code ErrorHeaderProcessor} with the provided description + * + * @param description description of the context in which an exception has been thrown + * @param type of key + * @param type of value + * @return {@code FixedKeyProcessorSupplier} that produces a message with the original value and headers detailing + * the error that has been captured. + */ + public static FixedKeyProcessorSupplier, V> withErrorHeaders( + final String description) { + return () -> new ErrorHeaderProcessor<>(description); + } + + static void addHeader(final String key, final String value, final Headers headers) { + headers.remove(key); + headers.add(key, value == null ? null : value.getBytes(StandardCharsets.UTF_8)); + } + + @Override + public void init(final FixedKeyProcessorContext context) { + this.context = context; + } + + @Override + public void process(final FixedKeyRecord> inputRecord) { + final Headers headers = inputRecord.headers(); + final Optional metadata = this.context.recordMetadata(); + addHeader(TOPIC, metadata.map(RecordMetadata::topic).orElse(null), headers); + addHeader(PARTITION, metadata.map(RecordMetadata::partition) + .map(p -> Integer.toString(p)) + .orElse(null), headers); + addHeader(OFFSET, metadata.map(RecordMetadata::offset) + .map(p -> Long.toString(p)) + .orElse(null), headers); + final ProcessingError value = inputRecord.value(); + addHeader(EXCEPTION_CLASS_NAME, value.getThrowable().getClass().getName(), headers); + addHeader(EXCEPTION_MESSAGE, value.getThrowable().getMessage(), headers); + addHeader(EXCEPTION_STACK_TRACE, ExceptionUtils.getStackTrace(value.getThrowable()), headers); + addHeader(DESCRIPTION, this.description, headers); + this.context.forward(inputRecord.withValue(value.getValue())); + } + + @Override + public void close() { + // do nothing + } + +} diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorHeaderTransformer.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorHeaderTransformer.java index 7317379..cbf44e9 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorHeaderTransformer.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorHeaderTransformer.java @@ -24,7 +24,6 @@ package com.bakdata.kafka; -import java.nio.charset.StandardCharsets; import lombok.NonNull; import lombok.RequiredArgsConstructor; import org.apache.commons.lang3.exception.ExceptionUtils; @@ -50,47 +49,49 @@ * * * @param type of value + * @deprecated Use {@link ErrorHeaderProcessor} */ @RequiredArgsConstructor +@Deprecated(since = "1.4.0") public class ErrorHeaderTransformer implements ValueTransformer, V> { /** * Prefix of all headers added by this ValueTransformer */ - public static final String HEADER_PREFIX = "__streams.errors."; + public static final String HEADER_PREFIX = ErrorHeaderProcessor.HEADER_PREFIX; /** * Header indicating the original input topic of the erroneous record */ - public static final String TOPIC = HEADER_PREFIX + "topic"; + public static final String TOPIC = ErrorHeaderProcessor.TOPIC; /** * Header indicating the original partition in the input topic of the erroneous record */ - public static final String PARTITION = HEADER_PREFIX + "partition"; + public static final String PARTITION = ErrorHeaderProcessor.PARTITION; /** * Header indicating the original offset in the partition of the input topic of the erroneous record */ - public static final String OFFSET = HEADER_PREFIX + "offset"; + public static final String OFFSET = ErrorHeaderProcessor.OFFSET; /** * Header indicating the description of the context in which an exception has been thrown */ - public static final String DESCRIPTION = HEADER_PREFIX + "description"; + public static final String DESCRIPTION = ErrorHeaderProcessor.DESCRIPTION; /** * Prefix of all headers detailing the error message added by this ValueTransformer */ - public static final String EXCEPTION_PREFIX = HEADER_PREFIX + "exception."; + public static final String EXCEPTION_PREFIX = ErrorHeaderProcessor.EXCEPTION_PREFIX; /** * Header indicating the class of the exception that was captured */ - public static final String EXCEPTION_CLASS_NAME = EXCEPTION_PREFIX + "class.name"; + public static final String EXCEPTION_CLASS_NAME = ErrorHeaderProcessor.EXCEPTION_CLASS_NAME; /** * Header indicating the message of the exception that was captured */ - public static final String EXCEPTION_MESSAGE = EXCEPTION_PREFIX + "message"; + public static final String EXCEPTION_MESSAGE = ErrorHeaderProcessor.EXCEPTION_MESSAGE; /** * Header indicating the stack trace of the exception that was captured */ - public static final String EXCEPTION_STACK_TRACE = EXCEPTION_PREFIX + "stack_trace"; + public static final String EXCEPTION_STACK_TRACE = ErrorHeaderProcessor.EXCEPTION_STACK_TRACE; private final @NonNull String description; - private ProcessorContext context = null; + private ProcessorContext context; /** * Create a new {@code ErrorHeaderTransformer} with the provided description @@ -128,7 +129,6 @@ public void close() { private void addHeader(final String key, final String value) { final Headers headers = this.context.headers(); - headers.remove(key); - headers.add(key, value == null ? null : value.getBytes(StandardCharsets.UTF_8)); + ErrorHeaderProcessor.addHeader(key, value, headers); } } diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatKeyValueMapper.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatKeyValueMapper.java index ba3cfb6..3f58f7b 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatKeyValueMapper.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatKeyValueMapper.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -95,7 +95,7 @@ public Iterable apply(final K key, final V value) { if (this.errorFilter.test(e)) { throw e; } - log.error("Cannot process ('" + ErrorUtil.toString(key) + "', '" + ErrorUtil.toString(value) + "')", e); + log.error("Cannot process ('{}', '{}')", ErrorUtil.toString(key), ErrorUtil.toString(value), e); return emptyList(); } } diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatTransformer.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatTransformer.java index 0ec31be..deb3b60 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatTransformer.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatTransformer.java @@ -45,9 +45,11 @@ * @param type of map result * @see #logErrors(Transformer) * @see #logErrors(Transformer, Predicate) + * @deprecated Use {@link ErrorLoggingProcessor} */ @Slf4j @RequiredArgsConstructor(access = AccessLevel.PRIVATE) +@Deprecated(since = "1.4.0") public final class ErrorLoggingFlatTransformer implements Transformer> { private final @NonNull Transformer> wrapped; private final @NonNull Predicate errorFilter; diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueMapper.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueMapper.java index 29bb2c5..894c8d6 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueMapper.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueMapper.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -92,7 +92,7 @@ public Iterable apply(final V value) { if (this.errorFilter.test(e)) { throw e; } - log.error("Cannot process " + ErrorUtil.toString(value), e); + log.error("Cannot process {}", ErrorUtil.toString(value), e); return emptyList(); } } diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueMapperWithKey.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueMapperWithKey.java index 539c470..5e0de0e 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueMapperWithKey.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueMapperWithKey.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -95,7 +95,7 @@ public Iterable apply(final K key, final V value) { if (this.errorFilter.test(e)) { throw e; } - log.error("Cannot process ('" + ErrorUtil.toString(key) + "', '" + ErrorUtil.toString(value) + "')", e); + log.error("Cannot process ('{}', '{}')", ErrorUtil.toString(key), ErrorUtil.toString(value), e); return emptyList(); } } diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueTransformer.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueTransformer.java index e5cba4a..d4a1ccf 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueTransformer.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueTransformer.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -152,7 +152,7 @@ public Iterable transform(final V value) { if (this.errorFilter.test(e)) { throw e; } - log.error("Cannot process " + ErrorUtil.toString(value), e); + log.error("Cannot process {}", ErrorUtil.toString(value), e); return emptyList(); } } diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueTransformerWithKey.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueTransformerWithKey.java index 48d7052..404f5ef 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueTransformerWithKey.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingFlatValueTransformerWithKey.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -158,7 +158,7 @@ public Iterable transform(final K key, final V value) { if (this.errorFilter.test(e)) { throw e; } - log.error("Cannot process ('" + ErrorUtil.toString(key) + "', '" + ErrorUtil.toString(value) + "')", e); + log.error("Cannot process ('{}', '{}')", ErrorUtil.toString(key), ErrorUtil.toString(value), e); return emptyList(); } } diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingKeyValueMapper.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingKeyValueMapper.java index 89dd92b..88aeb38 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingKeyValueMapper.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingKeyValueMapper.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -97,7 +97,7 @@ public Iterable apply(final K key, final V value) { if (this.errorFilter.test(e)) { throw e; } - log.error("Cannot process ('" + ErrorUtil.toString(key) + "', '" + ErrorUtil.toString(value) + "')", e); + log.error("Cannot process ('{}', '{}')", ErrorUtil.toString(key), ErrorUtil.toString(value), e); return emptyList(); } } diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingProcessor.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingProcessor.java new file mode 100644 index 0000000..0f56eac --- /dev/null +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingProcessor.java @@ -0,0 +1,169 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import java.util.Set; +import java.util.function.Predicate; +import lombok.AccessLevel; +import lombok.NonNull; +import lombok.RequiredArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; +import org.apache.kafka.streams.state.StoreBuilder; + +/** + * Wrap a {@code Processor} and log thrown exceptions with input key and value. + * + * @param type of input keys + * @param type of input values + * @param type of output keys + * @param type of output values + * @see #logErrors(Processor) + * @see #logErrors(Processor, Predicate) + */ +@Slf4j +@RequiredArgsConstructor(access = AccessLevel.PRIVATE) +public final class ErrorLoggingProcessor implements Processor { + private final @NonNull Processor wrapped; + private final @NonNull Predicate errorFilter; + + /** + * Wrap a {@code Processor} and log thrown exceptions with input key and value. Recoverable Kafka exceptions such as + * a schema registry timeout are forwarded and not captured. + * + * @param processor {@code Processor} whose exceptions should be logged + * @param type of input keys + * @param type of input values + * @param type of output keys + * @param type of output values + * @return {@code Processor} + * @see #logErrors(Processor, Predicate) + * @see ErrorUtil#isRecoverable(Exception) + */ + public static Processor logErrors( + final @NonNull Processor processor) { + return logErrors(processor, ErrorUtil::isRecoverable); + } + + /** + * Wrap a {@code Processor} and log thrown exceptions with input key and value. + *
{@code
+     * final KStream input = ...;
+     * final KStream output = input.process(() -> logErrors(new Processor() {...}));
+     * }
+     * 
+ * + * @param processor {@code Processor} whose exceptions should be logged + * @param errorFilter expression that filters errors which should be thrown and not logged + * @param type of input keys + * @param type of input values + * @param type of output keys + * @param type of output values + * @return {@code Processor} + */ + public static Processor logErrors( + final @NonNull Processor processor, + final @NonNull Predicate errorFilter) { + return new ErrorLoggingProcessor<>(processor, errorFilter); + } + + /** + * Wrap a {@code ProcessorSupplier} and log thrown exceptions with input key and value. Recoverable Kafka exceptions + * such as a schema registry timeout are forwarded and not captured. + * + * @param supplier {@code ProcessorSupplier} whose exceptions should be logged + * @param type of input keys + * @param type of input values + * @param type of output keys + * @param type of output values + * @return {@code ProcessorSupplier} + * @see #logErrors(ProcessorSupplier, Predicate) + * @see ErrorUtil#isRecoverable(Exception) + */ + public static ProcessorSupplier logErrors( + final @NonNull ProcessorSupplier supplier) { + return logErrors(supplier, ErrorUtil::isRecoverable); + } + + /** + * Wrap a {@code ProcessorSupplier} and log thrown exceptions with input key and value. + *
{@code
+     * final ProcessorSupplier processor = ...;
+     * final KStream input = ...;
+     * final KStream output = input.process(logErrors(processor));
+     * }
+     * 
+ * + * @param supplier {@code ProcessorSupplier} whose exceptions should be logged + * @param errorFilter expression that filters errors which should be thrown and not logged + * @param type of input keys + * @param type of input values + * @param type of output keys + * @param type of output values + * @return {@code ProcessorSupplier} + */ + public static ProcessorSupplier logErrors( + final @NonNull ProcessorSupplier supplier, + final @NonNull Predicate errorFilter) { + return new ProcessorSupplier<>() { + @Override + public Set> stores() { + return supplier.stores(); + } + + @Override + public Processor get() { + return logErrors(supplier.get(), errorFilter); + } + }; + } + + @Override + public void close() { + this.wrapped.close(); + } + + @Override + public void init(final ProcessorContext context) { + this.wrapped.init(context); + } + + @Override + public void process(final Record inputRecord) { + try { + this.wrapped.process(inputRecord); + } catch (final Exception e) { + if (this.errorFilter.test(e)) { + throw e; + } + log.error("Cannot process ('{}', '{}')", ErrorUtil.toString(inputRecord.key()), + ErrorUtil.toString(inputRecord.value()), e); + } + } + +} diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingTransformer.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingTransformer.java index 5794bfb..819e7f2 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingTransformer.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingTransformer.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -43,9 +43,11 @@ * @param type of transformation result * @see #logErrors(Transformer) * @see #logErrors(Transformer, Predicate) + * @deprecated Use {@link ErrorLoggingProcessor} */ @Slf4j @RequiredArgsConstructor(access = AccessLevel.PRIVATE) +@Deprecated(since = "1.4.0") public final class ErrorLoggingTransformer implements Transformer { private final @NonNull Transformer wrapped; private final @NonNull Predicate errorFilter; @@ -155,7 +157,7 @@ public R transform(final K key, final V value) { if (this.errorFilter.test(e)) { throw e; } - log.error("Cannot process ('" + ErrorUtil.toString(key) + "', '" + ErrorUtil.toString(value) + "')", e); + log.error("Cannot process ('{}', '{}')", ErrorUtil.toString(key), ErrorUtil.toString(value), e); return null; } } diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueMapper.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueMapper.java index 070d70e..034cda2 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueMapper.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueMapper.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -95,7 +95,7 @@ public Iterable apply(final V value) { if (this.errorFilter.test(e)) { throw e; } - log.error("Cannot process " + ErrorUtil.toString(value), e); + log.error("Cannot process {}", ErrorUtil.toString(value), e); return emptyList(); } } diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueMapperWithKey.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueMapperWithKey.java index 8168474..9f476a5 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueMapperWithKey.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueMapperWithKey.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -98,7 +98,7 @@ public Iterable apply(final K key, final V value) { if (this.errorFilter.test(e)) { throw e; } - log.error("Cannot process ('" + ErrorUtil.toString(key) + "', '" + ErrorUtil.toString(value) + "')", e); + log.error("Cannot process ('{}', '{}')", ErrorUtil.toString(key), ErrorUtil.toString(value), e); return emptyList(); } } diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueProcessor.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueProcessor.java new file mode 100644 index 0000000..714750c --- /dev/null +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueProcessor.java @@ -0,0 +1,164 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import java.util.Set; +import java.util.function.Predicate; +import lombok.AccessLevel; +import lombok.NonNull; +import lombok.RequiredArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.streams.processor.api.FixedKeyProcessor; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier; +import org.apache.kafka.streams.processor.api.FixedKeyRecord; +import org.apache.kafka.streams.state.StoreBuilder; + +/** + * Wrap a {@code FixedKeyProcessor} and log thrown exceptions with input key and value. + * + * @param type of input keys + * @param type of input values + * @param type of output values + * @see #logErrors(FixedKeyProcessor) + * @see #logErrors(FixedKeyProcessor, Predicate) + */ +@Slf4j +@RequiredArgsConstructor(access = AccessLevel.PRIVATE) +public final class ErrorLoggingValueProcessor implements FixedKeyProcessor { + private final @NonNull FixedKeyProcessor wrapped; + private final @NonNull Predicate errorFilter; + + /** + * Wrap a {@code FixedKeyProcessor} and log thrown exceptions with input key and value. Recoverable Kafka exceptions + * such as a schema registry timeout are forwarded and not captured. + * + * @param processor {@code FixedKeyProcessor} whose exceptions should be logged + * @param type of input keys + * @param type of input values + * @param type of output values + * @return {@code FixedKeyProcessor} + * @see #logErrors(FixedKeyProcessor, Predicate) + * @see ErrorUtil#isRecoverable(Exception) + */ + public static FixedKeyProcessor logErrors( + final @NonNull FixedKeyProcessor processor) { + return logErrors(processor, ErrorUtil::isRecoverable); + } + + /** + * Wrap a {@code FixedKeyProcessor} and log thrown exceptions with input key and value. + *
{@code
+     * final KStream input = ...;
+     * final KStream output = input.processValues(() -> logErrors(new FixedKeyProcessor() {...}));
+     * }
+     * 
+ * + * @param processor {@code FixedKeyProcessor} whose exceptions should be logged + * @param errorFilter expression that filters errors which should be thrown and not logged + * @param type of input keys + * @param type of input values + * @param type of output values + * @return {@code FixedKeyProcessor} + */ + public static FixedKeyProcessor logErrors( + final @NonNull FixedKeyProcessor processor, + final @NonNull Predicate errorFilter) { + return new ErrorLoggingValueProcessor<>(processor, errorFilter); + } + + /** + * Wrap a {@code FixedKeyProcessorSupplier} and log thrown exceptions with input key and value. Recoverable Kafka + * exceptions such as a schema registry timeout are forwarded and not captured. + * + * @param supplier {@code FixedKeyProcessorSupplier} whose exceptions should be logged + * @param type of input keys + * @param type of input values + * @param type of output values + * @return {@code FixedKeyProcessorSupplier} + * @see #logErrors(FixedKeyProcessorSupplier, Predicate) + * @see ErrorUtil#isRecoverable(Exception) + */ + public static FixedKeyProcessorSupplier logErrors( + final @NonNull FixedKeyProcessorSupplier supplier) { + return logErrors(supplier, ErrorUtil::isRecoverable); + } + + /** + * Wrap a {@code FixedKeyProcessorSupplier} and log thrown exceptions with input key and value. + *
{@code
+     * final FixedKeyProcessorSupplier processor = ...;
+     * final KStream input = ...;
+     * final KStream output = input.processValues(logErrors(processor));
+     * }
+     * 
+ * + * @param supplier {@code FixedKeyProcessorSupplier} whose exceptions should be logged + * @param errorFilter expression that filters errors which should be thrown and not logged + * @param type of input keys + * @param type of input values + * @param type of output values + * @return {@code FixedKeyProcessorSupplier} + */ + public static FixedKeyProcessorSupplier logErrors( + final @NonNull FixedKeyProcessorSupplier supplier, + final @NonNull Predicate errorFilter) { + return new FixedKeyProcessorSupplier<>() { + @Override + public Set> stores() { + return supplier.stores(); + } + + @Override + public FixedKeyProcessor get() { + return logErrors(supplier.get(), errorFilter); + } + }; + } + + @Override + public void close() { + this.wrapped.close(); + } + + @Override + public void init(final FixedKeyProcessorContext context) { + this.wrapped.init(context); + } + + @Override + public void process(final FixedKeyRecord inputRecord) { + try { + this.wrapped.process(inputRecord); + } catch (final Exception e) { + if (this.errorFilter.test(e)) { + throw e; + } + log.error("Cannot process ('{}', '{}')", ErrorUtil.toString(inputRecord.key()), + ErrorUtil.toString(inputRecord.value()), e); + } + } + +} diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueTransformer.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueTransformer.java index 9f44e46..4c9fa2f 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueTransformer.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueTransformer.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -45,9 +45,11 @@ * @param type of output values * @see #logErrors(ValueTransformer) * @see #logErrors(ValueTransformer, Predicate) + * @deprecated Use {@link ErrorLoggingValueProcessor} */ @Slf4j @RequiredArgsConstructor(access = AccessLevel.PRIVATE) +@Deprecated(since = "1.4.0") public final class ErrorLoggingValueTransformer implements ValueTransformer> { private final @NonNull ValueTransformer wrapped; private final @NonNull Predicate errorFilter; @@ -155,7 +157,7 @@ public Iterable transform(final V value) { if (this.errorFilter.test(e)) { throw e; } - log.error("Cannot process " + ErrorUtil.toString(value), e); + log.error("Cannot process {}", ErrorUtil.toString(value), e); return emptyList(); } } diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueTransformerWithKey.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueTransformerWithKey.java index 037170c..2c3e877 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueTransformerWithKey.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorLoggingValueTransformerWithKey.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -46,9 +46,11 @@ * @param type of output values * @see #logErrors(ValueTransformerWithKey) * @see #logErrors(ValueTransformerWithKey, Predicate) + * @deprecated Use {@link ErrorLoggingValueProcessor} */ @Slf4j @RequiredArgsConstructor(access = AccessLevel.PRIVATE) +@Deprecated(since = "1.4.0") public final class ErrorLoggingValueTransformerWithKey implements ValueTransformerWithKey> { private final @NonNull ValueTransformerWithKey wrapped; @@ -161,7 +163,7 @@ public Iterable transform(final K key, final V value) { if (this.errorFilter.test(e)) { throw e; } - log.error("Cannot process ('" + ErrorUtil.toString(key) + "', '" + ErrorUtil.toString(value) + "')", e); + log.error("Cannot process ('{}', '{}')", ErrorUtil.toString(key), ErrorUtil.toString(value), e); return emptyList(); } } diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorUtil.java b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorUtil.java index 1902b71..6a0432d 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/ErrorUtil.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/ErrorUtil.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -118,15 +118,15 @@ public static String toString(final Object o) { /** * Convert a {@code SpecificRecord} to {@code String} using JSON serialization. * - * @param record record to be serialized + * @param specificRecord record to be serialized * @return JSON representation of record or record if an error occurred */ - private static Object toString(final SpecificRecord record) { + private static Object toString(final SpecificRecord specificRecord) { try { - return writeAsJson(record); + return writeAsJson(specificRecord); } catch (final IOException ex) { log.warn("Failed to write to json", ex); - return record; + return specificRecord; } } @@ -139,15 +139,15 @@ private static String writeAsJson(final SpecificRecord itemRecord) throws IOExce /** * Convert a {@code GenericRecord} to {@code String} using JSON serialization. * - * @param record record to be serialized + * @param genericRecord record to be serialized * @return JSON representation of record or record if an error occurred */ - private static Object toString(final GenericRecord record) { + private static Object toString(final GenericRecord genericRecord) { try { - return writeAsJson(record); + return writeAsJson(genericRecord); } catch (final IOException ex) { log.warn("Failed to write to json", ex); - return record; + return genericRecord; } } diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/SuccessKeyValue.java b/error-handling-core/src/main/java/com/bakdata/kafka/SuccessKeyValue.java index 2695a7b..00fdb63 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/SuccessKeyValue.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/SuccessKeyValue.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -34,7 +34,7 @@ @RequiredArgsConstructor(access = AccessLevel.PRIVATE) final class SuccessKeyValue implements ProcessedKeyValue { - private final VR record; + private final VR value; static ProcessedKeyValue of(final VR vr) { return new SuccessKeyValue<>(vr); @@ -48,6 +48,6 @@ public Iterable>> getErrors() { @Override public Iterable getValues() { // allow null values - return Collections.singletonList(this.record); + return Collections.singletonList(this.value); } } diff --git a/error-handling-core/src/main/java/com/bakdata/kafka/SuccessValue.java b/error-handling-core/src/main/java/com/bakdata/kafka/SuccessValue.java index c7c9811..330a451 100644 --- a/error-handling-core/src/main/java/com/bakdata/kafka/SuccessValue.java +++ b/error-handling-core/src/main/java/com/bakdata/kafka/SuccessValue.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -33,7 +33,7 @@ @RequiredArgsConstructor(access = AccessLevel.PRIVATE) final class SuccessValue implements ProcessedValue { - private final VR record; + private final VR value; static ProcessedValue of(final VR vr) { return new SuccessValue<>(vr); @@ -47,7 +47,7 @@ public Iterable> getErrors() { @Override public Iterable getValues() { // allow null values - return Collections.singletonList(this.record); + return Collections.singletonList(this.value); } } diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatKeyValueMapperTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatKeyValueMapperTopologyTest.java index a81e0fd..1121d37 100644 --- a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatKeyValueMapperTopologyTest.java +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatKeyValueMapperTopologyTest.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -73,8 +73,8 @@ protected void buildTopology(final StreamsBuilder builder) { mapped.flatMapValues(ProcessedKeyValue::getValues) .to(OUTPUT_TOPIC, Produced.with(DOUBLE_SERDE, LONG_SERDE)); mapped.flatMap(ProcessedKeyValue::getErrors) - .transformValues( - DeadLetterTransformer.create("Description", deadLetterDescription -> deadLetterDescription)) + .processValues( + DeadLetterProcessor.create("Description", deadLetterDescription -> deadLetterDescription)) .to(ERROR_TOPIC); } @@ -188,7 +188,7 @@ void shouldHandleNullInput(final SoftAssertions softly) { .containsExactlyInAnyOrder(2.0, 3.0); softly.assertThat(records) .extracting(ProducerRecord::value) - .containsExactlyInAnyOrder(2L, 3l); + .containsExactlyInAnyOrder(2L, 3L); final List> errors = Seq.seq(this.topology.streamOutput(ERROR_TOPIC) .withValueType(DeadLetterDescription.class)) diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatTransformerTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatTransformerTopologyTest.java index ac5d413..c39f298 100644 --- a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatTransformerTopologyTest.java +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatTransformerTopologyTest.java @@ -25,7 +25,6 @@ package com.bakdata.kafka; import static com.bakdata.kafka.FilterHelper.filterAll; -import static org.mockito.Mockito.description; import static org.mockito.Mockito.mock; import java.util.List; @@ -66,7 +65,8 @@ protected void buildTopology(final StreamsBuilder builder) { mapped.flatMapValues(ProcessedKeyValue::getValues) .to(OUTPUT_TOPIC, Produced.with(DOUBLE_SERDE, LONG_SERDE)); mapped.flatMap(ProcessedKeyValue::getErrors) - .transformValues(DeadLetterTransformer.create("Description", deadLetterDescription -> deadLetterDescription)) + .processValues( + DeadLetterProcessor.create("Description", deadLetterDescription -> deadLetterDescription)) .to(ERROR_TOPIC); } diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueMapperTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueMapperTopologyTest.java index 4d60bb6..eec5c22 100644 --- a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueMapperTopologyTest.java +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueMapperTopologyTest.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -74,7 +74,8 @@ protected void buildTopology(final StreamsBuilder builder) { .to(OUTPUT_TOPIC, Produced.valueSerde(LONG_SERDE)); mapped.flatMapValues(ProcessedValue::getErrors) - .transformValues(DeadLetterTransformer.create("Description", deadLetterDescription -> deadLetterDescription)) + .processValues( + DeadLetterProcessor.create("Description", deadLetterDescription -> deadLetterDescription)) .to(ERROR_TOPIC); } diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueMapperWithKeyTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueMapperWithKeyTopologyTest.java index 0031a79..b8ca720 100644 --- a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueMapperWithKeyTopologyTest.java +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueMapperWithKeyTopologyTest.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -72,7 +72,8 @@ protected void buildTopology(final StreamsBuilder builder) { mapped.flatMapValues(ProcessedValue::getValues) .to(OUTPUT_TOPIC, Produced.valueSerde(LONG_SERDE)); mapped.flatMapValues(ProcessedValue::getErrors) - .transformValues(DeadLetterTransformer.create("Description", deadLetterDescription -> deadLetterDescription)) + .processValues( + DeadLetterProcessor.create("Description", deadLetterDescription -> deadLetterDescription)) .to(ERROR_TOPIC); } diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformerTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformerTopologyTest.java index 3adb91d..262b9be 100644 --- a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformerTopologyTest.java +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformerTopologyTest.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -64,7 +64,8 @@ protected void buildTopology(final StreamsBuilder builder) { mapped.flatMapValues(ProcessedValue::getValues) .to(OUTPUT_TOPIC, Produced.with(INTEGER_SERDE, LONG_SERDE)); mapped.flatMapValues(ProcessedValue::getErrors) - .transformValues(DeadLetterTransformer.create("Description", deadLetterDescription -> deadLetterDescription)) + .processValues( + DeadLetterProcessor.create("Description", deadLetterDescription -> deadLetterDescription)) .to(ERROR_TOPIC); } diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformerWithKeyTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformerWithKeyTopologyTest.java index 81542c5..0c4fb59 100644 --- a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformerWithKeyTopologyTest.java +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingFlatValueTransformerWithKeyTopologyTest.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -64,7 +64,8 @@ protected void buildTopology(final StreamsBuilder builder) { mapped.flatMapValues(ProcessedValue::getValues) .to(OUTPUT_TOPIC, Produced.with(INTEGER_SERDE, LONG_SERDE)); mapped.flatMapValues(ProcessedValue::getErrors) - .transformValues(DeadLetterTransformer.create("Description", deadLetterDescription -> deadLetterDescription)) + .processValues( + DeadLetterProcessor.create("Description", deadLetterDescription -> deadLetterDescription)) .to(ERROR_TOPIC); } diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingKeyValueMapperTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingKeyValueMapperTopologyTest.java index f32cb2d..57a3910 100644 --- a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingKeyValueMapperTopologyTest.java +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingKeyValueMapperTopologyTest.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -73,7 +73,8 @@ protected void buildTopology(final StreamsBuilder builder) { mapped.flatMapValues(ProcessedKeyValue::getValues) .to(OUTPUT_TOPIC, Produced.with(DOUBLE_SERDE, LONG_SERDE)); mapped.flatMap(ProcessedKeyValue::getErrors) - .transformValues(DeadLetterTransformer.create("Description", deadLetterDescription -> deadLetterDescription)) + .processValues( + DeadLetterProcessor.create("Description", deadLetterDescription -> deadLetterDescription)) .to(ERROR_TOPIC); } diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingProcessorTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingProcessorTopologyTest.java new file mode 100644 index 0000000..30b49d4 --- /dev/null +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingProcessorTopologyTest.java @@ -0,0 +1,450 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import static com.bakdata.kafka.FilterHelper.filterAll; +import static org.mockito.Mockito.mock; + +import java.util.List; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; +import org.assertj.core.api.SoftAssertions; +import org.assertj.core.api.junit.jupiter.SoftAssertionsExtension; +import org.jooq.lambda.Seq; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(SoftAssertionsExtension.class) +class ErrorCapturingProcessorTopologyTest extends ErrorCaptureTopologyTest { + + private static final String ERROR_TOPIC = "errors"; + private static final String OUTPUT_TOPIC = "output"; + private static final String INPUT_TOPIC = "input"; + private static final Serde STRING_SERDE = Serdes.String(); + private static final Serde LONG_SERDE = Serdes.Long(); + private static final Serde DOUBLE_SERDE = Serdes.Double(); + private Processor mapper = null; + + @Override + protected void buildTopology(final StreamsBuilder builder) { + final KStream input = builder.stream(INPUT_TOPIC, Consumed.with(null, STRING_SERDE)); + final KStream> mapped = + input.process(ErrorCapturingProcessor.captureErrors(() -> this.mapper)); + mapped.flatMapValues(ProcessedKeyValue::getValues) + .to(OUTPUT_TOPIC, Produced.with(DOUBLE_SERDE, LONG_SERDE)); + mapped.flatMap(ProcessedKeyValue::getErrors) + .processValues( + DeadLetterProcessor.create("Description", deadLetterDescription -> deadLetterDescription)) + .to(ERROR_TOPIC); + } + + @Test + void shouldNotAllowNullProcessor(final SoftAssertions softly) { + softly.assertThatThrownBy(() -> ErrorCapturingProcessor.captureErrors( + (Processor) null)) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorCapturingProcessor.captureErrors( + (Processor) null, filterAll())) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorCapturingProcessor.captureErrors( + (ProcessorSupplier) null)) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorCapturingProcessor.captureErrors( + (ProcessorSupplier) null, filterAll())) + .isInstanceOf(NullPointerException.class); + } + + @Test + void shouldNotAllowNullFilter(final SoftAssertions softly) { + softly.assertThatThrownBy(() -> ErrorCapturingProcessor.captureErrors(this.mapper, null)) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorCapturingProcessor.captureErrors(() -> this.mapper, null)) + .isInstanceOf(NullPointerException.class); + } + + @Test + void shouldForwardSchemaRegistryTimeout(final SoftAssertions softly) { + final RuntimeException throwable = createSchemaRegistryTimeoutException(); + this.mapper = new Processor<>() { + + @Override + public void init(final ProcessorContext context) { + } + + @Override + public void process(final Record record) { + if (1 == record.key() && "foo".equals(record.value())) { + throw throwable; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + softly.assertThatThrownBy(() -> this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo")) + .hasCauseInstanceOf(SerializationException.class); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .isEmpty(); + + final List> errors = + Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueType(DeadLetterDescription.class)) + .toList(); + softly.assertThat(errors) + .isEmpty(); + } + + @Test + void shouldNotCaptureThrowable(final SoftAssertions softly) { + final Error throwable = mock(Error.class); + this.mapper = new Processor<>() { + + @Override + public void init(final ProcessorContext context) { + } + + @Override + public void process(final Record record) { + if (1 == record.key() && "foo".equals(record.value())) { + throw throwable; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + softly.assertThatThrownBy(() -> this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo")) + .isEqualTo(throwable); + } + + @Test + void shouldCaptureProcessorError(final SoftAssertions softly) { + this.mapper = new Processor<>() { + private ProcessorContext context = null; + + @Override + public void init(final ProcessorContext context) { + this.context = context; + } + + @Override + public void process(final Record record) { + if (1 == record.key() && "foo".equals(record.value())) { + throw new RuntimeException("Cannot process"); + } + if (2 == record.key() && "bar".equals(record.value())) { + this.context.forward(record.withKey(2.0).withValue(2L)); + return; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo") + .add(2, "bar"); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2.0)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ); + final List> errors = + Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueType(DeadLetterDescription.class)) + .toList(); + softly.assertThat(errors) + .hasSize(1) + .first() + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(1)) + .extracting(ProducerRecord::value) + .isInstanceOf(DeadLetterDescription.class) + .satisfies(deadLetter -> { + softly.assertThat(deadLetter.getInputValue()).isEqualTo("foo"); + softly.assertThat(deadLetter.getDescription()).isEqualTo("Description"); + final DeadLetterDescription.Cause cause = deadLetter.getCause(); + softly.assertThat(cause.getMessage()).isEqualTo("Cannot process"); + softly.assertThat(cause.getStackTrace()).isNotNull(); + softly.assertThat(cause.getErrorClass()).isEqualTo("java.lang.RuntimeException"); + softly.assertThat(deadLetter.getTopic()).isEqualTo(INPUT_TOPIC); + softly.assertThat(deadLetter.getPartition()).isEqualTo(0); + softly.assertThat(deadLetter.getOffset()).isEqualTo(0L); + }); + } + + @Test + void shouldReturnOnNullInput(final SoftAssertions softly) { + this.mapper = new Processor<>() { + private ProcessorContext context = null; + + @Override + public void init(final ProcessorContext context) { + this.context = context; + } + + @Override + public void process(final Record record) { + if (record.key() == null && record.value() == null) { + this.context.forward(record.withKey(2.0).withValue(2L)); + return; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2.0)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ); + final List> errors = + Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueType(DeadLetterDescription.class)) + .toList(); + softly.assertThat(errors) + .isEmpty(); + } + + @Test + void shouldHandleErrorOnNullInput(final SoftAssertions softly) { + this.mapper = new Processor<>() { + + @Override + public void init(final ProcessorContext context) { + } + + @Override + public void process(final Record record) { + if (record.key() == null && record.value() == null) { + throw new RuntimeException("Cannot process"); + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .isEmpty(); + final List> errors = + Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueType(DeadLetterDescription.class)) + .toList(); + softly.assertThat(errors) + .hasSize(1) + .first() + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isNull()) + .extracting(ProducerRecord::value) + .isInstanceOf(DeadLetterDescription.class) + .satisfies(deadLetter -> { + softly.assertThat(deadLetter.getInputValue()).isNull(); + softly.assertThat(deadLetter.getDescription()).isEqualTo("Description"); + final DeadLetterDescription.Cause cause = deadLetter.getCause(); + softly.assertThat(cause.getMessage()).isEqualTo("Cannot process"); + softly.assertThat(cause.getStackTrace()).isNotNull(); + softly.assertThat(cause.getErrorClass()).isEqualTo("java.lang.RuntimeException"); + softly.assertThat(deadLetter.getTopic()).isEqualTo(INPUT_TOPIC); + softly.assertThat(deadLetter.getPartition()).isEqualTo(0); + softly.assertThat(deadLetter.getOffset()).isEqualTo(0L); + }); + } + + @Test + void shouldForwardOnNullInput(final SoftAssertions softly) { + this.mapper = new Processor<>() { + private ProcessorContext context = null; + + @Override + public void init(final ProcessorContext context) { + this.context = context; + } + + @Override + public void process(final Record record) { + if (record.key() == null && record.value() == null) { + this.context.forward(record.withKey(3.0).withValue(3L)); + return; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(3.0)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(3L)) + ); + final List> errors = + Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueType(DeadLetterDescription.class)) + .toList(); + softly.assertThat(errors) + .isEmpty(); + } + + @Test + void shouldHandleForwardedNullKeyValue(final SoftAssertions softly) { + this.mapper = new Processor<>() { + private ProcessorContext context = null; + + @Override + public void init(final ProcessorContext context) { + this.context = context; + } + + @Override + public void process(final Record record) { + if (2 == record.key() && "bar".equals(record.value())) { + this.context.forward(record.withKey(2.0).withValue(2L)); + return; + } + if (3 == record.key() && "baz".equals(record.value())) { + this.context.forward(record.withKey(null).withValue(null)); + return; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(2, "bar") + .add(3, "baz"); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(2) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2.0)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isNull()) + .extracting(ProducerRecord::value) + .satisfies(value -> softly.assertThat(value).isNull()) + ); + final List> errors = + Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueType(DeadLetterDescription.class)) + .toList(); + softly.assertThat(errors) + .isEmpty(); + } + +} diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingTransformerTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingTransformerTopologyTest.java index 45aefd6..327b110 100644 --- a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingTransformerTopologyTest.java +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingTransformerTopologyTest.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -65,7 +65,8 @@ protected void buildTopology(final StreamsBuilder builder) { mapped.flatMapValues(ProcessedKeyValue::getValues) .to(OUTPUT_TOPIC, Produced.with(DOUBLE_SERDE, LONG_SERDE)); mapped.flatMap(ProcessedKeyValue::getErrors) - .transformValues(DeadLetterTransformer.create("Description", deadLetterDescription -> deadLetterDescription)) + .processValues( + DeadLetterProcessor.create("Description", deadLetterDescription -> deadLetterDescription)) .to(ERROR_TOPIC); } diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueMapperTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueMapperTopologyTest.java index 90d4730..8580c24 100644 --- a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueMapperTopologyTest.java +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueMapperTopologyTest.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -71,7 +71,8 @@ protected void buildTopology(final StreamsBuilder builder) { mapped.flatMapValues(ProcessedValue::getValues) .to(OUTPUT_TOPIC, Produced.valueSerde(LONG_SERDE)); mapped.flatMapValues(ProcessedValue::getErrors) - .transformValues(DeadLetterTransformer.create("Description", deadLetterDescription -> deadLetterDescription)) + .processValues( + DeadLetterProcessor.create("Description", deadLetterDescription -> deadLetterDescription)) .to(ERROR_TOPIC); } diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueMapperWithKeyTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueMapperWithKeyTopologyTest.java index 710687c..1bc7426 100644 --- a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueMapperWithKeyTopologyTest.java +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueMapperWithKeyTopologyTest.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -71,7 +71,8 @@ protected void buildTopology(final StreamsBuilder builder) { mapped.flatMapValues(ProcessedValue::getValues) .to(OUTPUT_TOPIC, Produced.valueSerde(LONG_SERDE)); mapped.flatMapValues(ProcessedValue::getErrors) - .transformValues(DeadLetterTransformer.create("Description", deadLetterDescription -> deadLetterDescription)) + .processValues( + DeadLetterProcessor.create("Description", deadLetterDescription -> deadLetterDescription)) .to(ERROR_TOPIC); } diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueProcessorTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueProcessorTopologyTest.java new file mode 100644 index 0000000..f633388 --- /dev/null +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueProcessorTopologyTest.java @@ -0,0 +1,387 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import static com.bakdata.kafka.FilterHelper.filterAll; +import static org.mockito.Mockito.mock; + +import java.util.List; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.processor.api.FixedKeyProcessor; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier; +import org.apache.kafka.streams.processor.api.FixedKeyRecord; +import org.assertj.core.api.SoftAssertions; +import org.assertj.core.api.junit.jupiter.SoftAssertionsExtension; +import org.jooq.lambda.Seq; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(SoftAssertionsExtension.class) +class ErrorCapturingValueProcessorTopologyTest extends ErrorCaptureTopologyTest { + + private static final String ERROR_TOPIC = "errors"; + private static final String OUTPUT_TOPIC = "output"; + private static final String INPUT_TOPIC = "input"; + private static final Serde STRING_SERDE = Serdes.String(); + private static final Serde LONG_SERDE = Serdes.Long(); + private static final Serde INTEGER_SERDE = Serdes.Integer(); + private FixedKeyProcessor mapper = null; + + @Override + protected void buildTopology(final StreamsBuilder builder) { + final KStream input = builder.stream(INPUT_TOPIC, Consumed.with(null, STRING_SERDE)); + final KStream> mapped = + input.processValues(ErrorCapturingValueProcessor.captureErrors(() -> this.mapper)); + mapped.flatMapValues(ProcessedValue::getValues) + .to(OUTPUT_TOPIC, Produced.with(INTEGER_SERDE, LONG_SERDE)); + mapped.flatMapValues(ProcessedValue::getErrors) + .processValues( + DeadLetterProcessor.create("Description", deadLetterDescription -> deadLetterDescription)) + .to(ERROR_TOPIC); + } + + @Test + void shouldNotAllowNullProcessor(final SoftAssertions softly) { + softly.assertThatThrownBy(() -> ErrorCapturingValueProcessor.captureErrors( + (FixedKeyProcessor) null)) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorCapturingValueProcessor.captureErrors( + (FixedKeyProcessor) null, filterAll())) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorCapturingValueProcessor.captureErrors( + (FixedKeyProcessorSupplier) null)) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorCapturingValueProcessor.captureErrors( + (FixedKeyProcessorSupplier) null, filterAll())) + .isInstanceOf(NullPointerException.class); + } + + @Test + void shouldNotAllowNullFilter(final SoftAssertions softly) { + softly.assertThatThrownBy(() -> ErrorCapturingValueProcessor.captureErrors(this.mapper, null)) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorCapturingValueProcessor.captureErrors(() -> this.mapper, null)) + .isInstanceOf(NullPointerException.class); + } + + @Test + void shouldForwardSchemaRegistryTimeout(final SoftAssertions softly) { + final RuntimeException throwable = createSchemaRegistryTimeoutException(); + this.mapper = new FixedKeyProcessor<>() { + + @Override + public void init(final FixedKeyProcessorContext context) { + } + + @Override + public void process(final FixedKeyRecord record) { + if ("foo".equals(record.value())) { + throw throwable; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + softly.assertThatThrownBy(() -> this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo")) + .hasCauseInstanceOf(SerializationException.class); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .isEmpty(); + + final List> errors = + Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueType(DeadLetterDescription.class)) + .toList(); + softly.assertThat(errors) + .isEmpty(); + } + + @Test + void shouldNotCaptureThrowable(final SoftAssertions softly) { + final Error throwable = mock(Error.class); + this.mapper = new FixedKeyProcessor<>() { + + @Override + public void init(final FixedKeyProcessorContext context) { + } + + @Override + public void process(final FixedKeyRecord record) { + if ("foo".equals(record.value())) { + throw throwable; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + softly.assertThatThrownBy(() -> this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo")) + .isEqualTo(throwable); + } + + @Test + void shouldCaptureProcessorError(final SoftAssertions softly) { + this.mapper = new FixedKeyProcessor<>() { + private FixedKeyProcessorContext context = null; + + @Override + public void init(final FixedKeyProcessorContext context) { + this.context = context; + } + + @Override + public void process(final FixedKeyRecord record) { + if ("foo".equals(record.value())) { + throw new RuntimeException("Cannot process"); + } + if ("bar".equals(record.value())) { + this.context.forward(record.withValue(2L)); + return; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo") + .add(2, "bar"); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(INTEGER_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ); + final List> errors = + Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueType(DeadLetterDescription.class)) + .toList(); + softly.assertThat(errors) + .hasSize(1) + .first() + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(1)) + .extracting(ProducerRecord::value) + .isInstanceOf(DeadLetterDescription.class) + .satisfies(deadLetter -> { + softly.assertThat(deadLetter.getInputValue()).isEqualTo("foo"); + softly.assertThat(deadLetter.getDescription()).isEqualTo("Description"); + final DeadLetterDescription.Cause cause = deadLetter.getCause(); + softly.assertThat(cause.getMessage()).isEqualTo("Cannot process"); + softly.assertThat(cause.getStackTrace()).isNotNull(); + softly.assertThat(cause.getErrorClass()).isEqualTo("java.lang.RuntimeException"); + softly.assertThat(deadLetter.getTopic()).isEqualTo(INPUT_TOPIC); + softly.assertThat(deadLetter.getPartition()).isEqualTo(0); + softly.assertThat(deadLetter.getOffset()).isEqualTo(0L); + }); + } + + @Test + void shouldReturnOnNullInput(final SoftAssertions softly) { + this.mapper = new FixedKeyProcessor<>() { + private FixedKeyProcessorContext context = null; + + @Override + public void init(final FixedKeyProcessorContext context) { + this.context = context; + } + + @Override + public void process(final FixedKeyRecord record) { + if (record.value() == null) { + this.context.forward(record.withValue(2L)); + return; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(INTEGER_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isNull()) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ); + final List> errors = + Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueType(DeadLetterDescription.class)) + .toList(); + softly.assertThat(errors) + .isEmpty(); + } + + @Test + void shouldHandleErrorOnNullInput(final SoftAssertions softly) { + this.mapper = new FixedKeyProcessor<>() { + + @Override + public void init(final FixedKeyProcessorContext context) { + } + + @Override + public void process(final FixedKeyRecord record) { + if (record.value() == null) { + throw new RuntimeException("Cannot process"); + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(INTEGER_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .isEmpty(); + final List> errors = + Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueType(DeadLetterDescription.class)) + .toList(); + softly.assertThat(errors) + .hasSize(1) + .first() + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isNull()) + .extracting(ProducerRecord::value) + .isInstanceOf(DeadLetterDescription.class) + .satisfies(deadLetter -> { + softly.assertThat(deadLetter.getInputValue()).isNull(); + softly.assertThat(deadLetter.getDescription()).isEqualTo("Description"); + final DeadLetterDescription.Cause cause = deadLetter.getCause(); + softly.assertThat(cause.getMessage()).isEqualTo("Cannot process"); + softly.assertThat(cause.getStackTrace()).isNotNull(); + softly.assertThat(cause.getErrorClass()).isEqualTo("java.lang.RuntimeException"); + softly.assertThat(deadLetter.getTopic()).isEqualTo(INPUT_TOPIC); + softly.assertThat(deadLetter.getPartition()).isEqualTo(0); + softly.assertThat(deadLetter.getOffset()).isEqualTo(0L); + }); + } + + @Test + void shouldHandleForwardedNullValue(final SoftAssertions softly) { + this.mapper = new FixedKeyProcessor<>() { + private FixedKeyProcessorContext context = null; + + @Override + public void init(final FixedKeyProcessorContext context) { + this.context = context; + } + + @Override + public void process(final FixedKeyRecord record) { + if ("bar".equals(record.value())) { + this.context.forward(record.withValue(null)); + return; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(2, "bar"); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(INTEGER_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2)) + .extracting(ProducerRecord::value) + .satisfies(value -> softly.assertThat(value).isNull()) + ); + final List> errors = + Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueType(DeadLetterDescription.class)) + .toList(); + softly.assertThat(errors) + .isEmpty(); + } +} diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueTransformerTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueTransformerTopologyTest.java index 97dc466..7342cb0 100644 --- a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueTransformerTopologyTest.java +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueTransformerTopologyTest.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -64,7 +64,8 @@ protected void buildTopology(final StreamsBuilder builder) { mapped.flatMapValues(ProcessedValue::getValues) .to(OUTPUT_TOPIC, Produced.with(INTEGER_SERDE, LONG_SERDE)); mapped.flatMapValues(ProcessedValue::getErrors) - .transformValues(DeadLetterTransformer.create("Description", deadLetterDescription -> deadLetterDescription)) + .processValues( + DeadLetterProcessor.create("Description", deadLetterDescription -> deadLetterDescription)) .to(ERROR_TOPIC); } diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueTransformerWithKeyTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueTransformerWithKeyTopologyTest.java index bff2f1d..4344263 100644 --- a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueTransformerWithKeyTopologyTest.java +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorCapturingValueTransformerWithKeyTopologyTest.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -64,7 +64,8 @@ protected void buildTopology(final StreamsBuilder builder) { mapped.flatMapValues(ProcessedValue::getValues) .to(OUTPUT_TOPIC, Produced.with(INTEGER_SERDE, LONG_SERDE)); mapped.flatMapValues(ProcessedValue::getErrors) - .transformValues(DeadLetterTransformer.create("Description", deadLetterDescription -> deadLetterDescription)) + .processValues( + DeadLetterProcessor.create("Description", deadLetterDescription -> deadLetterDescription)) .to(ERROR_TOPIC); } diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorDescribingProcessorTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorDescribingProcessorTopologyTest.java new file mode 100644 index 0000000..e5b6127 --- /dev/null +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorDescribingProcessorTopologyTest.java @@ -0,0 +1,302 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import static org.mockito.Mockito.mock; + +import java.util.List; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; +import org.assertj.core.api.SoftAssertions; +import org.assertj.core.api.junit.jupiter.SoftAssertionsExtension; +import org.jooq.lambda.Seq; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(SoftAssertionsExtension.class) +class ErrorDescribingProcessorTopologyTest extends ErrorCaptureTopologyTest { + + private static final String OUTPUT_TOPIC = "output"; + private static final String INPUT_TOPIC = "input"; + private static final Serde STRING_SERDE = Serdes.String(); + private static final Serde LONG_SERDE = Serdes.Long(); + private static final Serde DOUBLE_SERDE = Serdes.Double(); + private Processor mapper = null; + + @Override + protected void buildTopology(final StreamsBuilder builder) { + final KStream input = builder.stream(INPUT_TOPIC, Consumed.with(null, STRING_SERDE)); + final KStream mapped = + input.process(ErrorDescribingProcessor.describeErrors(() -> this.mapper)); + mapped.to(OUTPUT_TOPIC, Produced.with(DOUBLE_SERDE, LONG_SERDE)); + } + + @Test + void shouldNotAllowNullProcessor(final SoftAssertions softly) { + softly.assertThatThrownBy(() -> ErrorDescribingProcessor.describeErrors( + (Processor) null)) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorDescribingProcessor.describeErrors( + (ProcessorSupplier) null)) + .isInstanceOf(NullPointerException.class); + } + + @Test + void shouldNotCaptureThrowable(final SoftAssertions softly) { + final Error throwable = mock(Error.class); + this.mapper = new Processor<>() { + + @Override + public void init(final ProcessorContext context) { + } + + @Override + public void process(final Record record) { + if (1 == record.key() && "foo".equals(record.value())) { + throw throwable; + } + throw new UnsupportedOperationException(); + } + }; + this.createTopology(); + softly.assertThatThrownBy(() -> this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo")) + .isEqualTo(throwable); + } + + @Test + void shouldCaptureProcessorError(final SoftAssertions softly) { + this.mapper = new Processor<>() { + private ProcessorContext context = null; + + @Override + public void init(final ProcessorContext context) { + this.context = context; + } + + @Override + public void process(final Record record) { + if (1 == record.key() && "foo".equals(record.value())) { + throw new RuntimeException("Cannot process"); + } + if (2 == record.key() && "bar".equals(record.value())) { + this.context.forward(record.withKey(2.0).withValue(2L)); + return; + } + throw new UnsupportedOperationException(); + } + }; + this.createTopology(); + softly.assertThatThrownBy(() -> this.topology.input() + .withValueSerde(STRING_SERDE) + .add(2, "bar") + .add(1, "foo")) + .satisfies(e -> softly.assertThat(e.getCause()) + .hasMessage( + "Cannot process ('" + ErrorUtil.toString(1) + "', '" + ErrorUtil.toString("foo") + "')") + ); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2.0)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ); + } + + @Test + void shouldReturnOnNullInput(final SoftAssertions softly) { + this.mapper = new Processor<>() { + private ProcessorContext context = null; + + @Override + public void init(final ProcessorContext context) { + this.context = context; + } + + @Override + public void process(final Record record) { + if (record.key() == null && record.value() == null) { + this.context.forward(record.withKey(2.0).withValue(2L)); + return; + } + throw new UnsupportedOperationException(); + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2.0)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ); + } + + @Test + void shouldHandleErrorOnNullInput(final SoftAssertions softly) { + this.mapper = new Processor<>() { + + @Override + public void init(final ProcessorContext context) { + } + + @Override + public void process(final Record record) { + if (record.key() == null && record.value() == null) { + throw new RuntimeException("Cannot process"); + } + throw new UnsupportedOperationException(); + } + }; + this.createTopology(); + softly.assertThatThrownBy(() -> this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null)) + .satisfies(e -> softly.assertThat(e.getCause()) + .hasMessage("Cannot process ('" + ErrorUtil.toString(null) + "', '" + ErrorUtil.toString(null) + + "')") + ); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .isEmpty(); + } + + @Test + void shouldForwardOnNullInput(final SoftAssertions softly) { + this.mapper = new Processor<>() { + private ProcessorContext context = null; + + @Override + public void init(final ProcessorContext context) { + this.context = context; + } + + @Override + public void process(final Record record) { + if (record.key() == null && record.value() == null) { + this.context.forward(record.withKey(3.0).withValue(3L)); + return; + } + throw new UnsupportedOperationException(); + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(3.0)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(3L)) + ); + } + + @Test + void shouldHandleForwardedNullKeyValue(final SoftAssertions softly) { + this.mapper = new Processor<>() { + private ProcessorContext context = null; + + @Override + public void init(final ProcessorContext context) { + this.context = context; + } + + @Override + public void process(final Record record) { + if (2 == record.key() && "bar".equals(record.value())) { + this.context.forward(record.withKey(2.0).withValue(2L)); + return; + } + if (3 == record.key() && "baz".equals(record.value())) { + this.context.forward(record.withKey(null).withValue(null)); + return; + } + throw new UnsupportedOperationException(); + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(2, "bar") + .add(3, "baz"); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(2) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2.0)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isNull()) + .extracting(ProducerRecord::value) + .satisfies(value -> softly.assertThat(value).isNull()) + ); + } + +} diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorDescribingValueProcessorTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorDescribingValueProcessorTopologyTest.java new file mode 100644 index 0000000..eb7e65a --- /dev/null +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorDescribingValueProcessorTopologyTest.java @@ -0,0 +1,253 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import static org.mockito.Mockito.mock; + +import java.util.List; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.processor.api.FixedKeyProcessor; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier; +import org.apache.kafka.streams.processor.api.FixedKeyRecord; +import org.assertj.core.api.SoftAssertions; +import org.assertj.core.api.junit.jupiter.SoftAssertionsExtension; +import org.jooq.lambda.Seq; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(SoftAssertionsExtension.class) +class ErrorDescribingValueProcessorTopologyTest extends ErrorCaptureTopologyTest { + + private static final String OUTPUT_TOPIC = "output"; + private static final String INPUT_TOPIC = "input"; + private static final Serde STRING_SERDE = Serdes.String(); + private static final Serde LONG_SERDE = Serdes.Long(); + private static final Serde INTEGER_SERDE = Serdes.Integer(); + private FixedKeyProcessor mapper = null; + + @Override + protected void buildTopology(final StreamsBuilder builder) { + final KStream input = builder.stream(INPUT_TOPIC, Consumed.with(null, STRING_SERDE)); + final KStream mapped = + input.processValues(ErrorDescribingValueProcessor.describeErrors(() -> this.mapper)); + mapped.to(OUTPUT_TOPIC, Produced.with(INTEGER_SERDE, LONG_SERDE)); + } + + @Test + void shouldNotAllowNullProcessor(final SoftAssertions softly) { + softly.assertThatThrownBy(() -> ErrorDescribingValueProcessor.describeErrors( + (FixedKeyProcessor) null)) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorDescribingValueProcessor.describeErrors( + (FixedKeyProcessorSupplier) null)) + .isInstanceOf(NullPointerException.class); + } + + @Test + void shouldNotCaptureThrowable(final SoftAssertions softly) { + final Error throwable = mock(Error.class); + this.mapper = new FixedKeyProcessor<>() { + private FixedKeyProcessorContext context = null; + + @Override + public void init(final FixedKeyProcessorContext context) { + this.context = context; + } + + @Override + public void process(final FixedKeyRecord record) { + if ("foo".equals(record.value())) { + throw throwable; + } + throw new UnsupportedOperationException(); + } + }; + this.createTopology(); + softly.assertThatThrownBy(() -> this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo")) + .isEqualTo(throwable); + } + + @Test + void shouldCaptureProcessorError(final SoftAssertions softly) { + this.mapper = new FixedKeyProcessor<>() { + private FixedKeyProcessorContext context = null; + + @Override + public void init(final FixedKeyProcessorContext context) { + this.context = context; + } + + @Override + public void process(final FixedKeyRecord record) { + if ("foo".equals(record.value())) { + throw new RuntimeException("Cannot process"); + } + if ("bar".equals(record.value())) { + this.context.forward(record.withValue(2L)); + return; + } + throw new UnsupportedOperationException(); + } + }; + this.createTopology(); + softly.assertThatThrownBy(() -> this.topology.input() + .withValueSerde(STRING_SERDE) + .add(2, "bar") + .add(1, "foo")) + .satisfies(e -> softly.assertThat(e.getCause()) + .hasMessage( + "Cannot process ('" + ErrorUtil.toString(1) + "', '" + ErrorUtil.toString("foo") + "')") + ); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(INTEGER_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ); + } + + @Test + void shouldReturnOnNullInput(final SoftAssertions softly) { + this.mapper = new FixedKeyProcessor<>() { + private FixedKeyProcessorContext context = null; + + @Override + public void init(final FixedKeyProcessorContext context) { + this.context = context; + } + + @Override + public void process(final FixedKeyRecord record) { + if (record.value() == null) { + this.context.forward(record.withValue(2L)); + return; + } + throw new UnsupportedOperationException(); + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(INTEGER_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isNull()) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ); + } + + @Test + void shouldHandleErrorOnNullInput(final SoftAssertions softly) { + this.mapper = new FixedKeyProcessor<>() { + + @Override + public void init(final FixedKeyProcessorContext context) { + } + + @Override + public void process(final FixedKeyRecord record) { + if (record.value() == null) { + throw new RuntimeException("Cannot process"); + } + throw new UnsupportedOperationException(); + } + }; + this.createTopology(); + softly.assertThatThrownBy(() -> this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null)) + .satisfies(e -> softly.assertThat(e.getCause()) + .hasMessage("Cannot process ('" + ErrorUtil.toString(null) + "', '" + ErrorUtil.toString(null) + + "')") + ); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(INTEGER_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .isEmpty(); + } + + @Test + void shouldHandleForwardedNullValue(final SoftAssertions softly) { + this.mapper = new FixedKeyProcessor<>() { + private FixedKeyProcessorContext context = null; + + @Override + public void init(final FixedKeyProcessorContext context) { + this.context = context; + } + + @Override + public void process(final FixedKeyRecord record) { + if ("bar".equals(record.value())) { + this.context.forward(record.withValue(null)); + return; + } + throw new UnsupportedOperationException(); + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(2, "bar"); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(INTEGER_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2)) + .extracting(ProducerRecord::value) + .satisfies(value -> softly.assertThat(value).isNull()) + ); + } +} diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorHeaderProcessorTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorHeaderProcessorTopologyTest.java new file mode 100644 index 0000000..e1da34c --- /dev/null +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorHeaderProcessorTopologyTest.java @@ -0,0 +1,185 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.when; + +import java.nio.charset.StandardCharsets; +import java.util.List; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Produced; +import org.assertj.core.api.SoftAssertions; +import org.assertj.core.api.junit.jupiter.SoftAssertionsExtension; +import org.jooq.lambda.Seq; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; + +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.STRICT_STUBS) +@ExtendWith(SoftAssertionsExtension.class) +class ErrorHeaderProcessorTopologyTest extends ErrorCaptureTopologyTest { + + private static final String ERROR_TOPIC = "errors"; + private static final String OUTPUT_TOPIC = "output"; + private static final String INPUT_TOPIC = "input"; + private static final Serde STRING_SERDE = Serdes.String(); + private static final Serde LONG_SERDE = Serdes.Long(); + private static final Serde DOUBLE_SERDE = Serdes.Double(); + @Mock + private KeyValueMapper> mapper; + + private static String getHeader(final Headers headers, final String key) { + final byte[] value = headers.lastHeader(key).value(); + return value == null ? null : new String(value, StandardCharsets.UTF_8); + } + + @Override + protected void buildTopology(final StreamsBuilder builder) { + final KStream input = builder.stream(INPUT_TOPIC, Consumed.with(null, STRING_SERDE)); + final KStream> mapped = + input.map(ErrorCapturingKeyValueMapper.captureErrors(this.mapper)); + mapped.flatMapValues(ProcessedKeyValue::getValues) + .to(OUTPUT_TOPIC, Produced.with(DOUBLE_SERDE, LONG_SERDE)); + mapped.flatMap(ProcessedKeyValue::getErrors) + .processValues(ErrorHeaderProcessor.withErrorHeaders("Description")) + .to(ERROR_TOPIC, Produced.with(null, STRING_SERDE)); + } + + @Test + void shouldCaptureKeyValueMapperError(final SoftAssertions softly) { + doReturn(KeyValue.pair(1.0, 1L)).when(this.mapper).apply(1, "foo"); + doThrow(new RuntimeException("Cannot process")).when(this.mapper).apply(2, "bar"); + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo") + .add(2, "bar"); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .first() + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(1.0)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(1L)); + final List> errors = Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueSerde(Serdes.String())) + .toList(); + softly.assertThat(errors) + .hasSize(1) + .first() + .isNotNull() + .satisfies(record -> { + softly.assertThat(record.key()).isEqualTo(2); + softly.assertThat(record.value()).isEqualTo("bar"); + }) + .extracting(ProducerRecord::headers) + .satisfies(headers -> { + softly.assertThat(getHeader(headers, ErrorHeaderProcessor.OFFSET)).isEqualTo("1"); + softly.assertThat(getHeader(headers, ErrorHeaderProcessor.PARTITION)).isEqualTo("0"); + softly.assertThat(getHeader(headers, ErrorHeaderProcessor.TOPIC)).isEqualTo(INPUT_TOPIC); + softly.assertThat(getHeader(headers, ErrorHeaderProcessor.DESCRIPTION)).isEqualTo("Description"); + softly.assertThat(getHeader(headers, ErrorHeaderProcessor.EXCEPTION_MESSAGE)) + .isEqualTo("Cannot process"); + softly.assertThat(getHeader(headers, ErrorHeaderProcessor.EXCEPTION_CLASS_NAME)) + .isEqualTo(RuntimeException.class.getName()); + softly.assertThat(getHeader(headers, ErrorHeaderProcessor.EXCEPTION_STACK_TRACE)).isNotNull(); + }); + } + + @Test + void shouldHandleErrorOnNullInput(final SoftAssertions softly) { + when(this.mapper.apply(null, null)).thenThrow(new RuntimeException("Cannot process")); + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .isEmpty(); + final List> errors = Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueSerde(Serdes.String())) + .toList(); + softly.assertThat(errors) + .hasSize(1) + .first() + .isNotNull() + .satisfies(record -> { + softly.assertThat(record.key()).isNull(); + softly.assertThat(record.value()).isNull(); + }); + } + + @Test + void shouldHandleExceptionWithoutMessage(final SoftAssertions softly) { + when(this.mapper.apply(1, "foo")).thenThrow(new RuntimeException()); + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo"); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .isEmpty(); + final List> errors = Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueSerde(Serdes.String())) + .toList(); + softly.assertThat(errors) + .hasSize(1) + .first() + .isNotNull() + .satisfies(record -> { + softly.assertThat(record.key()).isEqualTo(1); + softly.assertThat(record.value()).isEqualTo("foo"); + }) + .extracting(ProducerRecord::headers) + .satisfies(headers -> softly.assertThat(getHeader(headers, ErrorHeaderProcessor.EXCEPTION_MESSAGE)) + .isNull()); + } + +} diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorLoggingProcessorTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorLoggingProcessorTopologyTest.java new file mode 100644 index 0000000..371b3bb --- /dev/null +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorLoggingProcessorTopologyTest.java @@ -0,0 +1,373 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import static com.bakdata.kafka.FilterHelper.filterAll; +import static org.mockito.Mockito.mock; + +import java.util.List; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; +import org.assertj.core.api.SoftAssertions; +import org.assertj.core.api.junit.jupiter.SoftAssertionsExtension; +import org.jooq.lambda.Seq; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(SoftAssertionsExtension.class) +class ErrorLoggingProcessorTopologyTest extends ErrorCaptureTopologyTest { + + private static final String OUTPUT_TOPIC = "output"; + private static final String INPUT_TOPIC = "input"; + private static final Serde STRING_SERDE = Serdes.String(); + private static final Serde LONG_SERDE = Serdes.Long(); + private static final Serde DOUBLE_SERDE = Serdes.Double(); + private Processor mapper = null; + + @Override + protected void buildTopology(final StreamsBuilder builder) { + final KStream input = builder.stream(INPUT_TOPIC, Consumed.with(null, STRING_SERDE)); + final KStream mapped = input.process(ErrorLoggingProcessor.logErrors(() -> this.mapper)); + mapped.to(OUTPUT_TOPIC, Produced.with(DOUBLE_SERDE, LONG_SERDE)); + } + + @Test + void shouldNotAllowNullProcessor(final SoftAssertions softly) { + softly.assertThatThrownBy(() -> ErrorLoggingProcessor.logErrors( + (Processor) null)) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorLoggingProcessor.logErrors( + (Processor) null, filterAll())) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorLoggingProcessor.logErrors( + (Processor) null)) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorLoggingProcessor.logErrors( + (Processor) null, filterAll())) + .isInstanceOf(NullPointerException.class); + } + + @Test + void shouldNotAllowNullFilter(final SoftAssertions softly) { + softly.assertThatThrownBy(() -> ErrorLoggingProcessor.logErrors(this.mapper, null)) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorLoggingProcessor.logErrors(() -> this.mapper, null)) + .isInstanceOf(NullPointerException.class); + } + + @Test + void shouldForwardSchemaRegistryTimeout(final SoftAssertions softly) { + final RuntimeException throwable = createSchemaRegistryTimeoutException(); + this.mapper = new Processor<>() { + + @Override + public void init(final ProcessorContext context) { + } + + @Override + public void process(final Record record) { + if (1 == record.key() && "foo".equals(record.value())) { + throw throwable; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + softly.assertThatThrownBy(() -> this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo")) + .hasCauseInstanceOf(SerializationException.class); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .isEmpty(); + } + + @Test + void shouldNotCaptureThrowable(final SoftAssertions softly) { + final Error throwable = mock(Error.class); + this.mapper = new Processor<>() { + + @Override + public void init(final ProcessorContext context) { + } + + @Override + public void process(final Record record) { + if (1 == record.key() && "foo".equals(record.value())) { + throw throwable; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + softly.assertThatThrownBy(() -> this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo")) + .isEqualTo(throwable); + } + + @Test + void shouldCaptureProcessorError(final SoftAssertions softly) { + this.mapper = new Processor<>() { + private ProcessorContext context = null; + + @Override + public void init(final ProcessorContext context) { + this.context = context; + } + + @Override + public void process(final Record record) { + if (1 == record.key() && "foo".equals(record.value())) { + throw new RuntimeException("Cannot process"); + } + if (2 == record.key() && "bar".equals(record.value())) { + this.context.forward(record.withKey(2.0).withValue(2L)); + return; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo") + .add(2, "bar"); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2.0)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ); + } + + @Test + void shouldReturnOnNullInput(final SoftAssertions softly) { + this.mapper = new Processor<>() { + private ProcessorContext context = null; + + @Override + public void init(final ProcessorContext context) { + this.context = context; + } + + @Override + public void process(final Record record) { + if (record.key() == null && record.value() == null) { + this.context.forward(record.withKey(2.0).withValue(2L)); + return; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2.0)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ); + } + + @Test + void shouldHandleErrorOnNullInput(final SoftAssertions softly) { + this.mapper = new Processor<>() { + + @Override + public void init(final ProcessorContext context) { + } + + @Override + public void process(final Record record) { + if (record.key() == null && record.value() == null) { + throw new RuntimeException("Cannot process"); + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .isEmpty(); + } + + @Test + void shouldForwardOnNullInput(final SoftAssertions softly) { + this.mapper = new Processor<>() { + private ProcessorContext context = null; + + @Override + public void init(final ProcessorContext context) { + this.context = context; + } + + @Override + public void process(final Record record) { + if (record.key() == null && record.value() == null) { + this.context.forward(record.withKey(3.0).withValue(3L)); + return; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(3.0)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(3L)) + ); + } + + @Test + void shouldHandleForwardedNullKeyValue(final SoftAssertions softly) { + this.mapper = new Processor<>() { + private ProcessorContext context = null; + + @Override + public void init(final ProcessorContext context) { + this.context = context; + } + + @Override + public void process(final Record record) { + if (2 == record.key() && "bar".equals(record.value())) { + this.context.forward(record.withKey(2.0).withValue(2L)); + return; + } + if (3 == record.key() && "baz".equals(record.value())) { + this.context.forward(record.withKey(null).withValue(null)); + return; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(2, "bar") + .add(3, "baz"); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(DOUBLE_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(2) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2.0)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isNull()) + .extracting(ProducerRecord::value) + .satisfies(value -> softly.assertThat(value).isNull()) + ); + } + +} diff --git a/error-handling-core/src/test/java/com/bakdata/kafka/ErrorLoggingValueProcessorTopologyTest.java b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorLoggingValueProcessorTopologyTest.java new file mode 100644 index 0000000..3d3bbd6 --- /dev/null +++ b/error-handling-core/src/test/java/com/bakdata/kafka/ErrorLoggingValueProcessorTopologyTest.java @@ -0,0 +1,318 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import static com.bakdata.kafka.FilterHelper.filterAll; +import static org.mockito.Mockito.mock; + +import java.util.List; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.processor.api.FixedKeyProcessor; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier; +import org.apache.kafka.streams.processor.api.FixedKeyRecord; +import org.assertj.core.api.SoftAssertions; +import org.assertj.core.api.junit.jupiter.SoftAssertionsExtension; +import org.jooq.lambda.Seq; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(SoftAssertionsExtension.class) +class ErrorLoggingValueProcessorTopologyTest extends ErrorCaptureTopologyTest { + + private static final String OUTPUT_TOPIC = "output"; + private static final String INPUT_TOPIC = "input"; + private static final Serde STRING_SERDE = Serdes.String(); + private static final Serde LONG_SERDE = Serdes.Long(); + private static final Serde INTEGER_SERDE = Serdes.Integer(); + private FixedKeyProcessor mapper = null; + + @Override + protected void buildTopology(final StreamsBuilder builder) { + final KStream input = builder.stream(INPUT_TOPIC, Consumed.with(null, STRING_SERDE)); + final KStream mapped = + input.processValues(ErrorLoggingValueProcessor.logErrors(() -> this.mapper)); + mapped.to(OUTPUT_TOPIC, Produced.with(INTEGER_SERDE, LONG_SERDE)); + } + + @Test + void shouldNotAllowNullProcessor(final SoftAssertions softly) { + softly.assertThatThrownBy(() -> ErrorLoggingValueProcessor.logErrors( + (FixedKeyProcessor) null)) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorLoggingValueProcessor.logErrors( + (FixedKeyProcessor) null, filterAll())) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorLoggingValueProcessor.logErrors( + (FixedKeyProcessorSupplier) null)) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorLoggingValueProcessor.logErrors( + (FixedKeyProcessorSupplier) null, filterAll())) + .isInstanceOf(NullPointerException.class); + } + + @Test + void shouldNotAllowNullFilter(final SoftAssertions softly) { + softly.assertThatThrownBy(() -> ErrorLoggingValueProcessor.logErrors(this.mapper, null)) + .isInstanceOf(NullPointerException.class); + softly.assertThatThrownBy(() -> ErrorLoggingValueProcessor.logErrors(() -> this.mapper, null)) + .isInstanceOf(NullPointerException.class); + } + + @Test + void shouldForwardSchemaRegistryTimeout(final SoftAssertions softly) { + final RuntimeException throwable = createSchemaRegistryTimeoutException(); + this.mapper = new FixedKeyProcessor<>() { + + @Override + public void init(final FixedKeyProcessorContext context) { + } + + @Override + public void process(final FixedKeyRecord record) { + if ("foo".equals(record.value())) { + throw throwable; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + softly.assertThatThrownBy(() -> this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo")) + .hasCauseInstanceOf(SerializationException.class); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .isEmpty(); + } + + @Test + void shouldNotCaptureThrowable(final SoftAssertions softly) { + final Error throwable = mock(Error.class); + this.mapper = new FixedKeyProcessor<>() { + + @Override + public void init(final FixedKeyProcessorContext context) { + } + + @Override + public void process(final FixedKeyRecord record) { + if ("foo".equals(record.value())) { + throw throwable; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + softly.assertThatThrownBy(() -> this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo")) + .isEqualTo(throwable); + } + + @Test + void shouldCaptureProcessorError(final SoftAssertions softly) { + this.mapper = new FixedKeyProcessor<>() { + private FixedKeyProcessorContext context = null; + + @Override + public void init(final FixedKeyProcessorContext context) { + this.context = context; + } + + @Override + public void process(final FixedKeyRecord record) { + if ("foo".equals(record.value())) { + throw new RuntimeException("Cannot process"); + } + if ("bar".equals(record.value())) { + this.context.forward(record.withValue(2L)); + return; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(1, "foo") + .add(2, "bar"); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(INTEGER_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2)) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ); + } + + @Test + void shouldReturnOnNullInput(final SoftAssertions softly) { + this.mapper = new FixedKeyProcessor<>() { + private FixedKeyProcessorContext context = null; + + @Override + public void init(final FixedKeyProcessorContext context) { + this.context = context; + } + + @Override + public void process(final FixedKeyRecord record) { + if (record.value() == null) { + this.context.forward(record.withValue(2L)); + return; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(INTEGER_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isNull()) + .extracting(ProducerRecord::value) + .isInstanceOf(Long.class) + .satisfies(value -> softly.assertThat(value).isEqualTo(2L)) + ); + } + + @Test + void shouldHandleErrorOnNullInput(final SoftAssertions softly) { + this.mapper = new FixedKeyProcessor<>() { + + @Override + public void init(final FixedKeyProcessorContext context) { + } + + @Override + public void process(final FixedKeyRecord record) { + if (record.value() == null) { + throw new RuntimeException("Cannot process"); + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(null, null); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(INTEGER_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .isEmpty(); + } + + @Test + void shouldHandleForwardedNullValue(final SoftAssertions softly) { + this.mapper = new FixedKeyProcessor<>() { + private FixedKeyProcessorContext context = null; + + @Override + public void init(final FixedKeyProcessorContext context) { + this.context = context; + } + + @Override + public void process(final FixedKeyRecord record) { + if ("bar".equals(record.value())) { + this.context.forward(record.withValue(null)); + return; + } + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + }; + this.createTopology(); + this.topology.input() + .withValueSerde(STRING_SERDE) + .add(2, "bar"); + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withKeySerde(INTEGER_SERDE) + .withValueSerde(LONG_SERDE)) + .toList(); + softly.assertThat(records) + .hasSize(1) + .anySatisfy(r -> softly.assertThat(r) + .isNotNull() + .satisfies(record -> softly.assertThat(record.key()).isEqualTo(2)) + .extracting(ProducerRecord::value) + .satisfies(value -> softly.assertThat(value).isNull()) + ); + } +} diff --git a/error-handling-core/src/testFixtures/java/com/bakdata/kafka/ErrorCaptureTopologyTest.java b/error-handling-core/src/testFixtures/java/com/bakdata/kafka/ErrorCaptureTopologyTest.java index 23d2275..6226399 100644 --- a/error-handling-core/src/testFixtures/java/com/bakdata/kafka/ErrorCaptureTopologyTest.java +++ b/error-handling-core/src/testFixtures/java/com/bakdata/kafka/ErrorCaptureTopologyTest.java @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2020 bakdata + * Copyright (c) 2022 bakdata * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -42,16 +42,16 @@ protected Properties getKafkaProperties() { final Properties kafkaConfig = new Properties(); // exactly once and order - kafkaConfig.setProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE); + kafkaConfig.setProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2); kafkaConfig.put(StreamsConfig.producerPrefix(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION), 1); kafkaConfig.setProperty(StreamsConfig.producerPrefix(ProducerConfig.ACKS_CONFIG), "all"); // topology - kafkaConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, "dummy"); + kafkaConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, "fake"); kafkaConfig.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, IntegerSerde.class); kafkaConfig.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, TestDeadLetterSerde.class); - kafkaConfig.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy"); + kafkaConfig.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "fake"); return kafkaConfig; } diff --git a/error-handling-proto/lombok.config b/error-handling-proto/lombok.config index 6aa51d7..189c0be 100644 --- a/error-handling-proto/lombok.config +++ b/error-handling-proto/lombok.config @@ -1,2 +1,3 @@ # This file is generated by the 'io.freefair.lombok' Gradle plugin config.stopBubbling = true +lombok.addLombokGeneratedAnnotation = true diff --git a/error-handling-proto/src/main/java/com/bakdata/kafka/ProtoDeadLetterConverter.java b/error-handling-proto/src/main/java/com/bakdata/kafka/ProtoDeadLetterConverter.java index 2c0667f..61081fe 100644 --- a/error-handling-proto/src/main/java/com/bakdata/kafka/ProtoDeadLetterConverter.java +++ b/error-handling-proto/src/main/java/com/bakdata/kafka/ProtoDeadLetterConverter.java @@ -29,6 +29,7 @@ import com.google.protobuf.Int64Value; import com.google.protobuf.StringValue; import org.apache.kafka.streams.kstream.ValueTransformerSupplier; +import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier; /** @@ -86,9 +87,37 @@ public ProtoDeadLetter convert(final DeadLetterDescription deadLetterDescription * @param description shared description for all errors * @param type of the input value * @return a transformer supplier + * @deprecated Use {@link #asProcessor(String)} */ + @Deprecated(since = "1.4.0") public static ValueTransformerSupplier, ProtoDeadLetter> asTransformer( final String description) { return DeadLetterTransformer.create(description, new ProtoDeadLetterConverter()); } + + /** + * Creates a processor that uses the ProtoDeadLetterConverter + * + *
{@code
+     * // Example, this works for all error capturing topologies
+     * final KeyValueMapper> mapper = ...;
+     * final KStream input = ...;
+     * final KStream> processed = input.map(captureErrors(mapper));
+     * final KStream output = processed.flatMapValues(ProcessedKeyValue::getValues);
+     * final KStream> errors = processed.flatMapValues(ProcessedKeyValue::getErrors);
+     * final KStream deadLetters = errors.processValues(
+     *                      ProtoDeadLetterConverter.asProcessor("Description"));
+     * deadLetters.to(OUTPUT_TOPIC);
+     * }
+     * 
+ * + * @param description shared description for all errors + * @param type of the input key + * @param type of the input value + * @return a processor supplier + */ + public static FixedKeyProcessorSupplier, ProtoDeadLetter> asProcessor( + final String description) { + return DeadLetterProcessor.create(description, new ProtoDeadLetterConverter()); + } } diff --git a/error-handling-proto/src/test/java/com/bakdata/kafka/ProtoDeadLetterProcessorTest.java b/error-handling-proto/src/test/java/com/bakdata/kafka/ProtoDeadLetterProcessorTest.java new file mode 100644 index 0000000..396c56d --- /dev/null +++ b/error-handling-proto/src/test/java/com/bakdata/kafka/ProtoDeadLetterProcessorTest.java @@ -0,0 +1,170 @@ +/* + * MIT License + * + * Copyright (c) 2022 bakdata + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package com.bakdata.kafka; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.when; + +import com.bakdata.fluent_kafka_streams_tests.TestTopology; +import com.bakdata.kafka.proto.v1.ProtoDeadLetter; +import com.bakdata.schemaregistrymock.SchemaRegistryMock; +import com.google.protobuf.Int32Value; +import com.google.protobuf.Int64Value; +import com.google.protobuf.StringValue; +import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchemaProvider; +import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; +import io.confluent.kafka.serializers.protobuf.KafkaProtobufDeserializerConfig; +import io.confluent.kafka.streams.serdes.protobuf.KafkaProtobufSerde; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Produced; +import org.assertj.core.api.SoftAssertions; +import org.assertj.core.api.junit.jupiter.InjectSoftAssertions; +import org.assertj.core.api.junit.jupiter.SoftAssertionsExtension; +import org.jooq.lambda.Seq; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; + +@ExtendWith(MockitoExtension.class) +@ExtendWith(SoftAssertionsExtension.class) +@MockitoSettings(strictness = Strictness.STRICT_STUBS) +class ProtoDeadLetterProcessorTest extends ErrorCaptureTopologyTest { + private static final String ERROR_TOPIC = "errors"; + private static final String OUTPUT_TOPIC = "output"; + private static final String INPUT_TOPIC = "input"; + private static final Serde STRING_SERDE = Serdes.String(); + private static final Serde DEAD_LETTER_SERDE = new KafkaProtobufSerde<>(ProtoDeadLetter.class); + private static final String DEAD_LETTER_DESCRIPTION = "Description"; + private static final String ERROR_MESSAGE = "ERROR!"; + @InjectSoftAssertions + private SoftAssertions softly; + @Mock + private KeyValueMapper> mapper; + + @Override + protected void buildTopology(final StreamsBuilder builder) { + final KStream input = builder.stream(INPUT_TOPIC, Consumed.with(null, STRING_SERDE)); + final KStream> mapped = + input.map(ErrorCapturingKeyValueMapper.captureErrors(this.mapper)); + mapped.flatMapValues(ProcessedKeyValue::getValues) + .to(OUTPUT_TOPIC, Produced.valueSerde(STRING_SERDE)); + mapped.flatMap(ProcessedKeyValue::getErrors) + .processValues(ProtoDeadLetterConverter.asProcessor(DEAD_LETTER_DESCRIPTION)) + .to(ERROR_TOPIC); + } + + @Override + protected Properties getKafkaProperties() { + final Properties kafkaProperties = super.getKafkaProperties(); + kafkaProperties.setProperty( + StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, KafkaProtobufSerde.class.getName()); + kafkaProperties.put(KafkaProtobufDeserializerConfig.SPECIFIC_PROTOBUF_VALUE_TYPE, + ProtoDeadLetter.class); + return kafkaProperties; + } + + @Override + protected void createTopology() { + final StreamsBuilder builder = new StreamsBuilder(); + this.buildTopology(builder); + final Topology topology = builder.build(); + final Properties kafkaProperties = this.getKafkaProperties(); + final SchemaRegistryMock schemaRegistryMock = new SchemaRegistryMock(List.of(new ProtobufSchemaProvider())); + this.topology = new TestTopology(topology, kafkaProperties) + .withSchemaRegistryMock(schemaRegistryMock); + this.topology.start(); + DEAD_LETTER_SERDE.configure( + Map.of(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, + this.topology.getSchemaRegistryUrl()), false); + } + + @Test + void shouldConvertAndSerializeProtoDeadLetter() { + when(this.mapper.apply(any(), any())).thenThrow(new RuntimeException(ERROR_MESSAGE)); + this.createTopology(); + this.topology.input(INPUT_TOPIC).withValueSerde(STRING_SERDE) + .add(1, "foo") + .add(2, "bar"); + + final List> records = Seq.seq(this.topology.streamOutput(OUTPUT_TOPIC) + .withValueSerde(STRING_SERDE)) + .toList(); + this.softly.assertThat(records) + .isEmpty(); + + final List> errors = Seq.seq(this.topology.streamOutput(ERROR_TOPIC) + .withValueType(ProtoDeadLetter.class)) + .toList(); + + this.softly.assertThat(errors) + .hasSize(2) + .extracting(ProducerRecord::value).allSatisfy( + deadLetter -> { + this.softly.assertThat(deadLetter.getDescription()).isEqualTo(DEAD_LETTER_DESCRIPTION); + this.softly.assertThat(deadLetter.getCause().getMessage()).extracting(StringValue::getValue) + .isEqualTo(ERROR_MESSAGE); + this.softly.assertThat(deadLetter.getCause().getErrorClass()).extracting(StringValue::getValue) + .isEqualTo(RuntimeException.class.getCanonicalName()); + // We don't check the exact stack trace, but only that it consists of multiple lines + this.softly.assertThat(deadLetter.getCause().getStackTrace()).extracting(StringValue::getValue) + .extracting(s -> Arrays.asList(s.split("\n"))).asList().hasSizeGreaterThan(1); + this.softly.assertThat(deadLetter.getTopic()).extracting(StringValue::getValue) + .isEqualTo(INPUT_TOPIC); + this.softly.assertThat(deadLetter.getPartition()).extracting(Int32Value::getValue).isEqualTo(0); + } + ); + this.softly.assertThat(errors).extracting(ProducerRecord::value).element(0).satisfies( + deadLetter -> { + this.softly.assertThat(deadLetter.getInputValue()).extracting(StringValue::getValue) + .isEqualTo("foo"); + this.softly.assertThat(deadLetter.getOffset()).extracting(Int64Value::getValue).isEqualTo(0L); + } + ); + this.softly.assertThat(errors).map(ProducerRecord::value).element(1).satisfies( + deadLetter -> { + this.softly.assertThat(deadLetter.getInputValue()).extracting(StringValue::getValue) + .isEqualTo("bar"); + this.softly.assertThat(deadLetter.getOffset()).extracting(Int64Value::getValue).isEqualTo(1L); + } + ); + + } +} diff --git a/gradle.properties b/gradle.properties index b30ec1a..e2599d7 100644 --- a/gradle.properties +++ b/gradle.properties @@ -2,11 +2,11 @@ version=1.3.1-SNAPSHOT org.gradle.caching=true org.gradle.parallel=true org.gradle.jvmargs=-Xmx2048m -kafkaVersion=3.1.0 +kafkaVersion=3.3.1 avroVersion=1.10.2 -confluentVersion=7.1.1 +confluentVersion=7.3.0 jacksonVersion=2.12.5 junitVersion=5.7.2 mockitoVersion=3.12.4 log4jVersion=2.17.2 -kafkaStreamsTestsVersion=2.6.0 +kafkaStreamsTestsVersion=2.8.0 From 393931b136ede8385f672d70be73e78270191ce9 Mon Sep 17 00:00:00 2001 From: Philipp Schirmer Date: Wed, 28 Dec 2022 12:23:00 +0100 Subject: [PATCH 03/10] Upgrade to Avro 1.11 (#19) --- error-handling-avro/build.gradle.kts | 2 +- error-handling-core/build.gradle.kts | 2 +- gradle.properties | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/error-handling-avro/build.gradle.kts b/error-handling-avro/build.gradle.kts index a9d396c..4a60c26 100644 --- a/error-handling-avro/build.gradle.kts +++ b/error-handling-avro/build.gradle.kts @@ -1,7 +1,7 @@ description = "Transform dead letters in Kafka Streams applications to Avro format." plugins { - id("com.github.davidmc24.gradle.plugin.avro") version "1.2.1" + id("com.github.davidmc24.gradle.plugin.avro") version "1.5.0" } // add .avsc files to jar allowing us to use them in other projects as a schema dependency diff --git a/error-handling-core/build.gradle.kts b/error-handling-core/build.gradle.kts index 9131bf6..243c98b 100644 --- a/error-handling-core/build.gradle.kts +++ b/error-handling-core/build.gradle.kts @@ -1,7 +1,7 @@ description = "A library for error handling in Kafka Streams." plugins { - id("com.github.davidmc24.gradle.plugin.avro") version "1.2.1" + id("com.github.davidmc24.gradle.plugin.avro") version "1.5.0" } dependencies { diff --git a/gradle.properties b/gradle.properties index e2599d7..a34668a 100644 --- a/gradle.properties +++ b/gradle.properties @@ -3,7 +3,7 @@ org.gradle.caching=true org.gradle.parallel=true org.gradle.jvmargs=-Xmx2048m kafkaVersion=3.3.1 -avroVersion=1.10.2 +avroVersion=1.11.1 confluentVersion=7.3.0 jacksonVersion=2.12.5 junitVersion=5.7.2 From b511145deee4480ac4b385fae4bea13a5d2e39fa Mon Sep 17 00:00:00 2001 From: bakdata-bot <31185348+bakdata-bot@users.noreply.github.com> Date: Wed, 28 Dec 2022 12:47:15 +0000 Subject: [PATCH 04/10] Changelog for version 1.4.0 --- CHANGELOG.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 65013f1..993a8af 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,18 @@ # Change Log +## [1.4.0](https://github.com/bakdata/kafka-error-handling/tree/1.4.0) (2022-12-28) +[Full Changelog](https://github.com/bakdata/kafka-error-handling/compare/1.3.0...1.4.0) + +**Closed issues:** + +- Protobuf support for dead letters [\#12](https://github.com/bakdata/kafka-error-handling/issues/12) + +**Merged pull requests:** + +- Upgrade to Avro 1.11 [\#19](https://github.com/bakdata/kafka-error-handling/pull/19) ([@philipp94831](https://github.com/philipp94831)) +- Implement new Kafka 3.3 APIs [\#18](https://github.com/bakdata/kafka-error-handling/pull/18) ([@philipp94831](https://github.com/philipp94831)) +- Fix badge for Maven Central [\#17](https://github.com/bakdata/kafka-error-handling/pull/17) ([@philipp94831](https://github.com/philipp94831)) + ## [1.3.0](https://github.com/bakdata/kafka-error-handling/tree/1.3.0) (2022-07-13) [Full Changelog](https://github.com/bakdata/kafka-error-handling/compare/1.2.5...1.3.0) From 93aa82b00e4626f9aeed91808759fc94df09f245 Mon Sep 17 00:00:00 2001 From: bakdata-bot <31185348+bakdata-bot@users.noreply.github.com> Date: Wed, 28 Dec 2022 12:47:19 +0000 Subject: [PATCH 05/10] [Gradle Release Plugin] - pre tag commit: '1.4.0'. --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index a34668a..39a23f7 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,4 +1,4 @@ -version=1.3.1-SNAPSHOT +version=1.4.0 org.gradle.caching=true org.gradle.parallel=true org.gradle.jvmargs=-Xmx2048m From 2f9ed1a382dfec399ed3e7e9c13bdc50efaea921 Mon Sep 17 00:00:00 2001 From: bakdata-bot <31185348+bakdata-bot@users.noreply.github.com> Date: Wed, 28 Dec 2022 12:47:21 +0000 Subject: [PATCH 06/10] [Gradle Release Plugin] - new version commit: '1.4.1-SNAPSHOT'. --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 39a23f7..fca5368 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,4 +1,4 @@ -version=1.4.0 +version=1.4.1-SNAPSHOT org.gradle.caching=true org.gradle.parallel=true org.gradle.jvmargs=-Xmx2048m From df697c34e3f83823541d9a1440929d48d51be764 Mon Sep 17 00:00:00 2001 From: Philipp Schirmer Date: Wed, 4 Jan 2023 12:58:30 +0100 Subject: [PATCH 07/10] Update dependencies (#20) --- azure-pipelines.yml | 1 + build.gradle.kts | 14 ++++++++++---- error-handling-avro/build.gradle.kts | 3 ++- error-handling-core/build.gradle.kts | 3 ++- error-handling-proto/build.gradle.kts | 13 ++++++------- error-handling-proto/gradle.properties | 1 + gradle.properties | 11 ++++++----- gradle/wrapper/gradle-wrapper.properties | 2 +- 8 files changed, 29 insertions(+), 19 deletions(-) create mode 100644 error-handling-proto/gradle.properties diff --git a/azure-pipelines.yml b/azure-pipelines.yml index ceb0cba..1c7b7aa 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -12,6 +12,7 @@ variables: - group: sonarqube - group: sign - group: ossrh +- group: changelog resources: repositories: diff --git a/build.gradle.kts b/build.gradle.kts index 6de4ef7..012bd8c 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -1,9 +1,9 @@ plugins { - id("net.researchgate.release") version "2.8.1" + id("net.researchgate.release") version "3.0.2" id("com.bakdata.sonar") version "1.1.7" id("com.bakdata.sonatype") version "1.1.7" - id("org.hildan.github.changelog") version "0.8.0" - id("io.freefair.lombok") version "5.3.3.3" + id("org.hildan.github.changelog") version "1.12.1" + id("io.freefair.lombok") version "6.6.1" } allprojects { @@ -53,8 +53,14 @@ subprojects { apply(plugin = "java-test-fixtures") apply(plugin = "io.freefair.lombok") - configure { + configure { sourceCompatibility = JavaVersion.VERSION_11 targetCompatibility = JavaVersion.VERSION_11 } } + +release { + git { + requireBranch.set("master") + } +} diff --git a/error-handling-avro/build.gradle.kts b/error-handling-avro/build.gradle.kts index 4a60c26..2206bdf 100644 --- a/error-handling-avro/build.gradle.kts +++ b/error-handling-avro/build.gradle.kts @@ -24,7 +24,8 @@ dependencies { testImplementation(group = "org.jooq", name = "jool", version = "0.9.14") val mockitoVersion: String by project testImplementation(group = "org.mockito", name = "mockito-junit-jupiter", version = mockitoVersion) - testImplementation(group = "org.assertj", name = "assertj-core", version = "3.20.2") + val assertJVersion: String by project + testImplementation(group = "org.assertj", name = "assertj-core", version = assertJVersion) val log4jVersion: String by project testImplementation(group = "org.apache.logging.log4j", name = "log4j-slf4j-impl", version = log4jVersion) val kafkaStreamsTestsVersion: String by project diff --git a/error-handling-core/build.gradle.kts b/error-handling-core/build.gradle.kts index 243c98b..ea8269a 100644 --- a/error-handling-core/build.gradle.kts +++ b/error-handling-core/build.gradle.kts @@ -16,7 +16,8 @@ dependencies { testImplementation(group = "org.junit.jupiter", name = "junit-jupiter-api", version = junitVersion) testImplementation(group = "org.junit.jupiter", name = "junit-jupiter-params", version = junitVersion) testRuntimeOnly(group = "org.junit.jupiter", name = "junit-jupiter-engine", version = junitVersion) - testImplementation(group = "org.assertj", name = "assertj-core", version = "3.20.2") + val assertJVersion: String by project + testImplementation(group = "org.assertj", name = "assertj-core", version = assertJVersion) val mockitoVersion: String by project testImplementation(group = "org.mockito", name = "mockito-core", version = mockitoVersion) testImplementation(group = "org.mockito", name = "mockito-junit-jupiter", version = mockitoVersion) diff --git a/error-handling-proto/build.gradle.kts b/error-handling-proto/build.gradle.kts index 0a88191..8d8fb23 100644 --- a/error-handling-proto/build.gradle.kts +++ b/error-handling-proto/build.gradle.kts @@ -1,22 +1,21 @@ -import com.google.protobuf.gradle.protobuf -import com.google.protobuf.gradle.protoc - description = "Transform dead letters in Kafka Streams applications to protobuf." plugins { - id("com.google.protobuf") version "0.8.18" + id("com.google.protobuf") version "0.9.1" } +val protobufVersion: String by project dependencies { api(project(":error-handling-core")) - api(group = "com.google.protobuf", name = "protobuf-java", version = "3.18.1") + api(group = "com.google.protobuf", name = "protobuf-java", version = protobufVersion) val junitVersion: String by project testRuntimeOnly(group = "org.junit.jupiter", name = "junit-jupiter-engine", version = junitVersion) testImplementation(testFixtures(project(":error-handling-core"))) testImplementation(group = "org.jooq", name = "jool", version = "0.9.14") val mockitoVersion: String by project testImplementation(group = "org.mockito", name = "mockito-junit-jupiter", version = mockitoVersion) - testImplementation(group = "org.assertj", name = "assertj-core", version = "3.20.2") + val assertJVersion: String by project + testImplementation(group = "org.assertj", name = "assertj-core", version = assertJVersion) val log4jVersion: String by project testImplementation(group = "org.apache.logging.log4j", name = "log4j-slf4j-impl", version = log4jVersion) val kafkaStreamsTestsVersion: String by project @@ -36,6 +35,6 @@ dependencies { protobuf { protoc { - artifact = "com.google.protobuf:protoc:3.18.1" + artifact = "com.google.protobuf:protoc:$protobufVersion" } } diff --git a/error-handling-proto/gradle.properties b/error-handling-proto/gradle.properties new file mode 100644 index 0000000..4441762 --- /dev/null +++ b/error-handling-proto/gradle.properties @@ -0,0 +1 @@ +protobufVersion=3.21.12 diff --git a/gradle.properties b/gradle.properties index fca5368..3d39abf 100644 --- a/gradle.properties +++ b/gradle.properties @@ -4,9 +4,10 @@ org.gradle.parallel=true org.gradle.jvmargs=-Xmx2048m kafkaVersion=3.3.1 avroVersion=1.11.1 -confluentVersion=7.3.0 -jacksonVersion=2.12.5 -junitVersion=5.7.2 +confluentVersion=7.3.1 +jacksonVersion=2.14.1 +junitVersion=5.9.1 mockitoVersion=3.12.4 -log4jVersion=2.17.2 -kafkaStreamsTestsVersion=2.8.0 +log4jVersion=2.19.0 +kafkaStreamsTestsVersion=2.8.1 +assertJVersion=3.23.1 diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 1f3fdbc..f42e62f 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,5 +1,5 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-6.7.1-all.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-7.6-all.zip zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists From 02c59feaa2a9656fce596e3e31c6b0c961d80a56 Mon Sep 17 00:00:00 2001 From: bakdata-bot <31185348+bakdata-bot@users.noreply.github.com> Date: Wed, 4 Jan 2023 12:07:40 +0000 Subject: [PATCH 08/10] Changelog for version 1.4.1 --- CHANGELOG.md | 30 +++++++++++++++++++----------- 1 file changed, 19 insertions(+), 11 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 993a8af..5f52845 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,7 +1,14 @@ # Change Log +## [1.4.1](https://github.com/bakdata/kafka-error-handling/tree/1.4.1) (2023-01-04) +[View commits](https://github.com/bakdata/kafka-error-handling/compare/1.4.0...1.4.1) + +**Merged pull requests:** + +- Update dependencies [\#20](https://github.com/bakdata/kafka-error-handling/pull/20) ([@philipp94831](https://github.com/philipp94831)) + ## [1.4.0](https://github.com/bakdata/kafka-error-handling/tree/1.4.0) (2022-12-28) -[Full Changelog](https://github.com/bakdata/kafka-error-handling/compare/1.3.0...1.4.0) +[View commits](https://github.com/bakdata/kafka-error-handling/compare/1.3.0...1.4.0) **Closed issues:** @@ -14,14 +21,14 @@ - Fix badge for Maven Central [\#17](https://github.com/bakdata/kafka-error-handling/pull/17) ([@philipp94831](https://github.com/philipp94831)) ## [1.3.0](https://github.com/bakdata/kafka-error-handling/tree/1.3.0) (2022-07-13) -[Full Changelog](https://github.com/bakdata/kafka-error-handling/compare/1.2.5...1.3.0) +[View commits](https://github.com/bakdata/kafka-error-handling/compare/1.2.5...1.3.0) **Merged pull requests:** - Proto conversion for deadletter [\#13](https://github.com/bakdata/kafka-error-handling/pull/13) ([@mkcode92](https://github.com/mkcode92)) ## [1.2.5](https://github.com/bakdata/kafka-error-handling/tree/1.2.5) (2022-06-23) -[Full Changelog](https://github.com/bakdata/kafka-error-handling/compare/1.2.4...1.2.5) +[View commits](https://github.com/bakdata/kafka-error-handling/compare/1.2.4...1.2.5) **Closed issues:** @@ -32,7 +39,7 @@ - Support exceptions with null message [\#15](https://github.com/bakdata/kafka-error-handling/pull/15) ([@philipp94831](https://github.com/philipp94831)) ## [1.2.4](https://github.com/bakdata/kafka-error-handling/tree/1.2.4) (2022-05-11) -[Full Changelog](https://github.com/bakdata/kafka-error-handling/compare/1.2.3...1.2.4) +[View commits](https://github.com/bakdata/kafka-error-handling/compare/1.2.3...1.2.4) **Closed issues:** @@ -43,49 +50,49 @@ - Fix OFFSET header name in ErrorHeaderTransformer [\#11](https://github.com/bakdata/kafka-error-handling/pull/11) ([@patrickjkennedy](https://github.com/patrickjkennedy)) ## [1.2.3](https://github.com/bakdata/kafka-error-handling/tree/1.2.3) (2022-05-02) -[Full Changelog](https://github.com/bakdata/kafka-error-handling/compare/1.2.2...1.2.3) +[View commits](https://github.com/bakdata/kafka-error-handling/compare/1.2.2...1.2.3) **Merged pull requests:** - Add flat transformers [\#9](https://github.com/bakdata/kafka-error-handling/pull/9) ([@philipp94831](https://github.com/philipp94831)) ## [1.2.2](https://github.com/bakdata/kafka-error-handling/tree/1.2.2) (2022-02-18) -[Full Changelog](https://github.com/bakdata/kafka-error-handling/compare/1.2.1...1.2.2) +[View commits](https://github.com/bakdata/kafka-error-handling/compare/1.2.1...1.2.2) **Merged pull requests:** - Add factory methods for TransformerSuppliers [\#8](https://github.com/bakdata/kafka-error-handling/pull/8) ([@philipp94831](https://github.com/philipp94831)) ## [1.2.1](https://github.com/bakdata/kafka-error-handling/tree/1.2.1) (2022-02-17) -[Full Changelog](https://github.com/bakdata/kafka-error-handling/compare/1.2.0...1.2.1) +[View commits](https://github.com/bakdata/kafka-error-handling/compare/1.2.0...1.2.1) **Merged pull requests:** - Add error class to dead letter [\#7](https://github.com/bakdata/kafka-error-handling/pull/7) ([@philipp94831](https://github.com/philipp94831)) ## [1.2.0](https://github.com/bakdata/kafka-error-handling/tree/1.2.0) (2022-02-15) -[Full Changelog](https://github.com/bakdata/kafka-error-handling/compare/1.1.2...1.2.0) +[View commits](https://github.com/bakdata/kafka-error-handling/compare/1.1.2...1.2.0) **Merged pull requests:** - Add processing context to dead letters [\#6](https://github.com/bakdata/kafka-error-handling/pull/6) ([@philipp94831](https://github.com/philipp94831)) ## [1.1.2](https://github.com/bakdata/kafka-error-handling/tree/1.1.2) (2021-12-10) -[Full Changelog](https://github.com/bakdata/kafka-error-handling/compare/1.1.1...1.1.2) +[View commits](https://github.com/bakdata/kafka-error-handling/compare/1.1.1...1.1.2) **Merged pull requests:** - Update avro to 1.10 [\#4](https://github.com/bakdata/kafka-error-handling/pull/4) ([@philipp94831](https://github.com/philipp94831)) ## [1.1.1](https://github.com/bakdata/kafka-error-handling/tree/1.1.1) (2021-12-10) -[Full Changelog](https://github.com/bakdata/kafka-error-handling/compare/1.1.0...1.1.1) +[View commits](https://github.com/bakdata/kafka-error-handling/compare/1.1.0...1.1.1) **Merged pull requests:** - Update log4j to 2.15.0 [\#5](https://github.com/bakdata/kafka-error-handling/pull/5) ([@philipp94831](https://github.com/philipp94831)) ## [1.1.0](https://github.com/bakdata/kafka-error-handling/tree/1.1.0) (2021-09-24) -[Full Changelog](https://github.com/bakdata/kafka-error-handling/compare/1.0.0...1.1.0) +[View commits](https://github.com/bakdata/kafka-error-handling/compare/1.0.0...1.1.0) **Merged pull requests:** @@ -93,6 +100,7 @@ - Provide error details as header fields [\#2](https://github.com/bakdata/kafka-error-handling/pull/2) ([@philipp94831](https://github.com/philipp94831)) ## [1.0.0](https://github.com/bakdata/kafka-error-handling/tree/1.0.0) (2020-03-09) +[View commits](https://github.com/bakdata/kafka-error-handling/compare/69873530dde0856c890f502f825ce0da046aef1d...1.0.0) **Merged pull requests:** From c66efd8704ce802505a3da12eb0340b9d3ec820f Mon Sep 17 00:00:00 2001 From: bakdata-bot <31185348+bakdata-bot@users.noreply.github.com> Date: Wed, 4 Jan 2023 12:07:44 +0000 Subject: [PATCH 09/10] [Gradle Release Plugin] - pre tag commit: '1.4.1'. --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 3d39abf..0175918 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,4 +1,4 @@ -version=1.4.1-SNAPSHOT +version=1.4.1 org.gradle.caching=true org.gradle.parallel=true org.gradle.jvmargs=-Xmx2048m From 13da160498920775c39fd9cc7ad744ddb295e429 Mon Sep 17 00:00:00 2001 From: bakdata-bot <31185348+bakdata-bot@users.noreply.github.com> Date: Wed, 4 Jan 2023 12:07:47 +0000 Subject: [PATCH 10/10] [Gradle Release Plugin] - new version commit: '1.4.2-SNAPSHOT'. --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 0175918..a416a3e 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,4 +1,4 @@ -version=1.4.1 +version=1.4.2-SNAPSHOT org.gradle.caching=true org.gradle.parallel=true org.gradle.jvmargs=-Xmx2048m