diff --git a/spring-kafka/src/main/java/org/springframework/kafka/core/FailedRecordTuple.java b/spring-kafka/src/main/java/org/springframework/kafka/core/FailedRecordTuple.java new file mode 100644 index 000000000..acf954770 --- /dev/null +++ b/spring-kafka/src/main/java/org/springframework/kafka/core/FailedRecordTuple.java @@ -0,0 +1,32 @@ +/* + * Copyright 2016-2024 the original author or authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * ... + * @author Sanghyeok An + * + * @since 3.3 + */ + +package org.springframework.kafka.core; + +import org.apache.kafka.clients.consumer.ConsumerRecord; + +public record FailedRecordTuple( + ConsumerRecord record, + RuntimeException ex) { + +}; diff --git a/spring-kafka/src/main/java/org/springframework/kafka/listener/KafkaMessageListenerContainer.java b/spring-kafka/src/main/java/org/springframework/kafka/listener/KafkaMessageListenerContainer.java index 7bb04c4b2..b5c4fd5c2 100644 --- a/spring-kafka/src/main/java/org/springframework/kafka/listener/KafkaMessageListenerContainer.java +++ b/spring-kafka/src/main/java/org/springframework/kafka/listener/KafkaMessageListenerContainer.java @@ -16,7 +16,6 @@ package org.springframework.kafka.listener; -import java.lang.Thread.UncaughtExceptionHandler; import java.nio.ByteBuffer; import java.time.Duration; import java.util.AbstractMap.SimpleEntry; @@ -39,6 +38,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledFuture; @@ -84,6 +84,7 @@ import org.springframework.core.task.SimpleAsyncTaskExecutor; import org.springframework.kafka.KafkaException; import org.springframework.kafka.core.ConsumerFactory; +import org.springframework.kafka.core.FailedRecordTuple; import org.springframework.kafka.core.KafkaAdmin; import org.springframework.kafka.core.KafkaResourceHolder; import org.springframework.kafka.event.ConsumerFailedToStartEvent; @@ -107,6 +108,7 @@ import org.springframework.kafka.listener.ContainerProperties.AckMode; import org.springframework.kafka.listener.ContainerProperties.AssignmentCommitOption; import org.springframework.kafka.listener.ContainerProperties.EOSMode; +import org.springframework.kafka.listener.FailedRecordTracker.FailedRecord; import org.springframework.kafka.listener.adapter.AsyncRepliesAware; import org.springframework.kafka.support.Acknowledgment; import org.springframework.kafka.support.KafkaHeaders; @@ -842,6 +844,9 @@ private final class ListenerConsumer implements SchedulingAwareRunnable, Consume private volatile long lastPoll = System.currentTimeMillis(); + private final ConcurrentLinkedDeque> failedRecords = new ConcurrentLinkedDeque(); + + @SuppressWarnings(UNCHECKED) ListenerConsumer(GenericMessageListener listener, ListenerType listenerType, ObservationRegistry observationRegistry) { @@ -899,10 +904,9 @@ else if (listener instanceof MessageListener) { this.observationEnabled = this.containerProperties.isObservationEnabled(); if (!AopUtils.isAopProxy(listener)) { - final BiConsumer, RuntimeException> asyncRetryCallback - = (cRecord, runtimeException) -> - this.invokeErrorHandlerBySingleRecord(cRecord, runtimeException); - this.listener.setAsyncRetryCallback(asyncRetryCallback); + final java.util.function.Consumer callbackForAsyncFailureQueue = + (fRecord) -> this.failedRecords.addLast(fRecord); + this.listener.setCallbackForAsyncFailureQueue(callbackForAsyncFailureQueue); } } else { @@ -1303,6 +1307,15 @@ public void run() { boolean failedAuthRetry = false; this.lastReceive = System.currentTimeMillis(); while (isRunning()) { + + try { + handleAsyncFailure(); + } catch (Exception e) { + // TODO: Need to improve error handling. + // TODO: Need to determine how to handle a failed message. + logger.error("Failed to process re-try messages. "); + } + try { pollAndInvoke(); if (failedAuthRetry) { @@ -1444,6 +1457,19 @@ protected void pollAndInvoke() { } } + protected void handleAsyncFailure() { + List copyFailedRecords = new ArrayList<>(); + while (!this.failedRecords.isEmpty()) { + FailedRecordTuple failedRecordTuple = this.failedRecords.pollFirst(); + copyFailedRecords.add(failedRecordTuple); + } + + if (!copyFailedRecords.isEmpty()) { + copyFailedRecords.forEach(failedRecordTuple -> + this.invokeErrorHandlerBySingleRecord(failedRecordTuple.record(), failedRecordTuple.ex())); + } + } + private void doProcessCommits() { if (!this.autoCommit && !this.isRecordAck) { try { diff --git a/spring-kafka/src/main/java/org/springframework/kafka/listener/MessageListener.java b/spring-kafka/src/main/java/org/springframework/kafka/listener/MessageListener.java index ccce3541e..388478a49 100644 --- a/spring-kafka/src/main/java/org/springframework/kafka/listener/MessageListener.java +++ b/spring-kafka/src/main/java/org/springframework/kafka/listener/MessageListener.java @@ -16,9 +16,10 @@ package org.springframework.kafka.listener; -import java.util.function.BiConsumer; +import java.util.function.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.springframework.kafka.core.FailedRecordTuple; /** * Listener for handling individual incoming Kafka messages. @@ -28,11 +29,12 @@ * * @author Marius Bogoevici * @author Gary Russell + * @author Sanghyeok An */ @FunctionalInterface public interface MessageListener extends GenericMessageListener> { - default void setAsyncRetryCallback(BiConsumer, RuntimeException> asyncRetryCallback) { + default void setCallbackForAsyncFailureQueue(Consumer asyncRetryCallback) { // } } diff --git a/spring-kafka/src/main/java/org/springframework/kafka/listener/adapter/KafkaBackoffAwareMessageListenerAdapter.java b/spring-kafka/src/main/java/org/springframework/kafka/listener/adapter/KafkaBackoffAwareMessageListenerAdapter.java index 74ea973da..ce38a17cf 100644 --- a/spring-kafka/src/main/java/org/springframework/kafka/listener/adapter/KafkaBackoffAwareMessageListenerAdapter.java +++ b/spring-kafka/src/main/java/org/springframework/kafka/listener/adapter/KafkaBackoffAwareMessageListenerAdapter.java @@ -26,6 +26,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.springframework.kafka.core.FailedRecordTuple; import org.springframework.kafka.listener.AcknowledgingConsumerAwareMessageListener; import org.springframework.kafka.listener.KafkaBackoffException; import org.springframework.kafka.listener.KafkaConsumerBackoffManager; @@ -44,6 +45,7 @@ * @param the record key type. * @param the record value type. * @author Tomaz Fernandes + * @author Sanghyeok An * @since 2.7 * */ @@ -94,17 +96,6 @@ public void onMessage(ConsumerRecord consumerRecord, @Nullable Acknowledgm maybeGetBackoffTimestamp(consumerRecord) .ifPresent(nextExecutionTimestamp -> this.kafkaConsumerBackoffManager .backOffIfNecessary(createContext(consumerRecord, nextExecutionTimestamp, consumer))); - // ConsumerRecord(topic = myRetryFutureTopic5, partition = 1, leaderEpoch = 0, offset = 0, CreateTime = 1727697786519, serialized key size = 1, serialized value size = 19, headers = RecordHeaders(headers = [], isReadOnly = false), key = 0, value = Testing topic 5 - 1) - // ConsumerRecord(topic = myRetryFutureTopic5-listener2-0, partition = 1, leaderEpoch = 0, offset = 0, CreateTime = 1727697787141, serialized key size = 1, serialized value size = 19, headers = RecordHeaders(headers = [RecordHeader(key = kafka_exception-fqcn, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-cause-fqcn, value = [106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-message, value = [65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 50, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 64, 55, 49, 56, 57, 49, 100, 54, 98, 93, 59, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53]), RecordHeader(key = kafka_exception-stacktrace, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 50, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 64, 55, 49, 56, 57, 49, 100, 54, 98, 93, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 97, 115, 121, 110, 99, 70, 97, 105, 108, 117, 114, 101, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 54, 55, 55, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 108, 97, 109, 98, 100, 97, 36, 104, 97, 110, 100, 108, 101, 82, 101, 115, 117, 108, 116, 36, 49, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 52, 57, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 117, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 54, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 117, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 83, 116, 97, 103, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 56, 55, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 119, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 50, 51, 50, 53, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 104, 97, 110, 100, 108, 101, 82, 101, 115, 117, 108, 116, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 52, 56, 54, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 105, 110, 118, 111, 107, 101, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 51, 57, 50, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 82, 101, 99, 111, 114, 100, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 111, 110, 77, 101, 115, 115, 97, 103, 101, 40, 82, 101, 99, 111, 114, 100, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 56, 53, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 82, 101, 99, 111, 114, 100, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 111, 110, 77, 101, 115, 115, 97, 103, 101, 40, 82, 101, 99, 111, 114, 100, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 53, 48, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 75, 97, 102, 107, 97, 66, 97, 99, 107, 111, 102, 102, 65, 119, 97, 114, 101, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 105, 110, 118, 111, 107, 101, 68, 101, 108, 101, 103, 97, 116, 101, 79, 110, 77, 101, 115, 115, 97, 103, 101, 40, 75, 97, 102, 107, 97, 66, 97, 99, 107, 111, 102, 102, 65, 119, 97, 114, 101, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 49, 49, 53, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 75, 97, 102, 107, 97, 66, 97, 99, 107, 111, 102, 102, 65, 119, 97, 114, 101, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 111, 110, 77, 101, 115, 115, 97, 103, 101, 40, 75, 97, 102, 107, 97, 66, 97, 99, 107, 111, 102, 102, 65, 119, 97, 114, 101, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 49, 48, 54, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 75, 97, 102, 107, 97, 66, 97, 99, 107, 111, 102, 102, 65, 119, 97, 114, 101, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 111, 110, 77, 101, 115, 115, 97, 103, 101, 40, 75, 97, 102, 107, 97, 66, 97, 99, 107, 111, 102, 102, 65, 119, 97, 114, 101, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 53, 48, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 100, 111, 73, 110, 118, 111, 107, 101, 79, 110, 77, 101, 115, 115, 97, 103, 101, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 50, 56, 50, 49, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 105, 110, 118, 111, 107, 101, 79, 110, 77, 101, 115, 115, 97, 103, 101, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 50, 55, 57, 57, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 108, 97, 109, 98, 100, 97, 36, 100, 111, 73, 110, 118, 111, 107, 101, 82, 101, 99, 111, 114, 100, 76, 105, 115, 116, 101, 110, 101, 114, 36, 53, 52, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 50, 55, 49, 55, 41, 10, 9, 97, 116, 32, 105, 111, 46, 109, 105, 99, 114, 111, 109, 101, 116, 101, 114, 46, 111, 98, 115, 101, 114, 118, 97, 116, 105, 111, 110, 46, 79, 98, 115, 101, 114, 118, 97, 116, 105, 111, 110, 46, 111, 98, 115, 101, 114, 118, 101, 40, 79, 98, 115, 101, 114, 118, 97, 116, 105, 111, 110, 46, 106, 97, 118, 97, 58, 53, 54, 53, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 100, 111, 73, 110, 118, 111, 107, 101, 82, 101, 99, 111, 114, 100, 76, 105, 115, 116, 101, 110, 101, 114, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 50, 55, 49, 53, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 100, 111, 73, 110, 118, 111, 107, 101, 87, 105, 116, 104, 82, 101, 99, 111, 114, 100, 115, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 50, 53, 53, 55, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 105, 110, 118, 111, 107, 101, 82, 101, 99, 111, 114, 100, 76, 105, 115, 116, 101, 110, 101, 114, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 50, 52, 52, 54, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 105, 110, 118, 111, 107, 101, 76, 105, 115, 116, 101, 110, 101, 114, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 50, 48, 57, 55, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 105, 110, 118, 111, 107, 101, 73, 102, 72, 97, 118, 101, 82, 101, 99, 111, 114, 100, 115, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 49, 52, 55, 51, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 112, 111, 108, 108, 65, 110, 100, 73, 110, 118, 111, 107, 101, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 49, 52, 51, 56, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 114, 117, 110, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 49, 51, 48, 55, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 82, 117, 110, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 56, 48, 52, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 84, 104, 114, 101, 97, 100, 46, 114, 117, 110, 40, 84, 104, 114, 101, 97, 100, 46, 106, 97, 118, 97, 58, 56, 51, 51, 41, 10, 67, 97, 117, 115, 101, 100, 32, 98, 121, 58, 32, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 46, 108, 97, 109, 98, 100, 97, 36, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 50, 36, 48, 40, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 46, 106, 97, 118, 97, 58, 54, 53, 55, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 56, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 101, 120, 101, 99, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 48, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 100, 111, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 106, 97, 118, 97, 58, 51, 55, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 36, 87, 111, 114, 107, 81, 117, 101, 117, 101, 46, 116, 111, 112, 76, 101, 118, 101, 108, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 49, 56, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 115, 99, 97, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 53, 53, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 114, 117, 110, 87, 111, 114, 107, 101, 114, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 50, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 114, 117, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 106, 97, 118, 97, 58, 49, 54, 53, 41, 10]), RecordHeader(key = kafka_original-topic, value = [109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53]), RecordHeader(key = kafka_original-partition, value = [0, 0, 0, 1]), RecordHeader(key = kafka_original-offset, value = [0, 0, 0, 0, 0, 0, 0, 0]), RecordHeader(key = kafka_original-timestamp, value = [0, 0, 1, -110, 66, -48, 22, -105]), RecordHeader(key = kafka_original-timestamp-type, value = [67, 114, 101, 97, 116, 101, 84, 105, 109, 101]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -105]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 2]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 28, -21])], isReadOnly = false), key = 0, value = Testing topic 5 - 1) - // ConsumerRecord(topic = myRetryFutureTopic5-listener2-1, partition = 1, leaderEpoch = 0, offset = 0, CreateTime = 1727697788166, serialized key size = 1, serialized value size = 19, headers = RecordHeaders(headers = [RecordHeader(key = kafka_original-topic, value = [109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53]), RecordHeader(key = kafka_original-partition, value = [0, 0, 0, 1]), RecordHeader(key = kafka_original-offset, value = [0, 0, 0, 0, 0, 0, 0, 0]), RecordHeader(key = kafka_original-timestamp, value = [0, 0, 1, -110, 66, -48, 22, -105]), RecordHeader(key = kafka_original-timestamp-type, value = [67, 114, 101, 97, 116, 101, 84, 105, 109, 101]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -105]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 2]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 28, -21]), RecordHeader(key = kafka_exception-fqcn, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-cause-fqcn, value = [106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-message, value = [65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 50, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 64, 55, 49, 56, 57, 49, 100, 54, 98, 93, 59, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53, 45, 108, 105, 115, 116, 101, 110, 101, 114, 50, 45, 48]), RecordHeader(key = kafka_exception-stacktrace, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 50, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 64, 55, 49, 56, 57, 49, 100, 54, 98, 93, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 97, 115, 121, 110, 99, 70, 97, 105, 108, 117, 114, 101, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 54, 55, 55, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 108, 97, 109, 98, 100, 97, 36, 104, 97, 110, 100, 108, 101, 82, 101, 115, 117, 108, 116, 36, 49, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 52, 57, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 117, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 54, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 85, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 46, 116, 114, 121, 70, 105, 114, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 52, 49, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 112, 111, 115, 116, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 53, 49, 48, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 55, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 101, 120, 101, 99, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 48, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 100, 111, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 106, 97, 118, 97, 58, 51, 55, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 36, 87, 111, 114, 107, 81, 117, 101, 117, 101, 46, 116, 111, 112, 76, 101, 118, 101, 108, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 49, 56, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 115, 99, 97, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 53, 53, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 114, 117, 110, 87, 111, 114, 107, 101, 114, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 50, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 114, 117, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 106, 97, 118, 97, 58, 49, 54, 53, 41, 10, 67, 97, 117, 115, 101, 100, 32, 98, 121, 58, 32, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53, 45, 108, 105, 115, 116, 101, 110, 101, 114, 50, 45, 48, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 46, 108, 97, 109, 98, 100, 97, 36, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 50, 36, 48, 40, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 46, 106, 97, 118, 97, 58, 54, 53, 55, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 56, 41, 10, 9, 46, 46, 46, 32, 54, 32, 109, 111, 114, 101, 10]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -105]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 3]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 32, -19])], isReadOnly = false), key = 0, value = Testing topic 5 - 1) - // ConsumerRecord(topic = myRetryFutureTopic5-listener2-2, partition = 1, leaderEpoch = 0, offset = 0, CreateTime = 1727697789173, serialized key size = 1, serialized value size = 19, headers = RecordHeaders(headers = [RecordHeader(key = kafka_original-topic, value = [109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53]), RecordHeader(key = kafka_original-partition, value = [0, 0, 0, 1]), RecordHeader(key = kafka_original-offset, value = [0, 0, 0, 0, 0, 0, 0, 0]), RecordHeader(key = kafka_original-timestamp, value = [0, 0, 1, -110, 66, -48, 22, -105]), RecordHeader(key = kafka_original-timestamp-type, value = [67, 114, 101, 97, 116, 101, 84, 105, 109, 101]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -105]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 2]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 28, -21]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -105]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 3]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 32, -19]), RecordHeader(key = kafka_exception-fqcn, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-cause-fqcn, value = [106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-message, value = [65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 50, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 64, 55, 49, 56, 57, 49, 100, 54, 98, 93, 59, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53, 45, 108, 105, 115, 116, 101, 110, 101, 114, 50, 45, 49]), RecordHeader(key = kafka_exception-stacktrace, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 50, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 64, 55, 49, 56, 57, 49, 100, 54, 98, 93, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 97, 115, 121, 110, 99, 70, 97, 105, 108, 117, 114, 101, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 54, 55, 55, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 108, 97, 109, 98, 100, 97, 36, 104, 97, 110, 100, 108, 101, 82, 101, 115, 117, 108, 116, 36, 49, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 52, 57, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 117, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 54, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 85, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 46, 116, 114, 121, 70, 105, 114, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 52, 49, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 112, 111, 115, 116, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 53, 49, 48, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 55, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 101, 120, 101, 99, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 48, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 100, 111, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 106, 97, 118, 97, 58, 51, 55, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 36, 87, 111, 114, 107, 81, 117, 101, 117, 101, 46, 116, 111, 112, 76, 101, 118, 101, 108, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 49, 56, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 115, 99, 97, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 53, 53, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 114, 117, 110, 87, 111, 114, 107, 101, 114, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 50, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 114, 117, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 106, 97, 118, 97, 58, 49, 54, 53, 41, 10, 67, 97, 117, 115, 101, 100, 32, 98, 121, 58, 32, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53, 45, 108, 105, 115, 116, 101, 110, 101, 114, 50, 45, 49, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 46, 108, 97, 109, 98, 100, 97, 36, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 50, 36, 48, 40, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 46, 106, 97, 118, 97, 58, 54, 53, 55, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 56, 41, 10, 9, 46, 46, 46, 32, 54, 32, 109, 111, 114, 101, 10]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -105]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 4]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 36, -35])], isReadOnly = false), key = 0, value = Testing topic 5 - 1) - // ConsumerRecord(topic = myRetryFutureTopic5-listener2-dlt, partition = 1, leaderEpoch = 0, offset = 0, CreateTime = 1727697790187, serialized key size = 1, serialized value size = 19, headers = RecordHeaders(headers = [RecordHeader(key = kafka_original-topic, value = [109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53]), RecordHeader(key = kafka_original-partition, value = [0, 0, 0, 1]), RecordHeader(key = kafka_original-offset, value = [0, 0, 0, 0, 0, 0, 0, 0]), RecordHeader(key = kafka_original-timestamp, value = [0, 0, 1, -110, 66, -48, 22, -105]), RecordHeader(key = kafka_original-timestamp-type, value = [67, 114, 101, 97, 116, 101, 84, 105, 109, 101]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -105]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 2]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 28, -21]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -105]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 3]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 32, -19]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -105]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 4]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 36, -35]), RecordHeader(key = kafka_exception-fqcn, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-cause-fqcn, value = [106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-message, value = [65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 50, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 64, 55, 49, 56, 57, 49, 100, 54, 98, 93, 59, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53, 45, 108, 105, 115, 116, 101, 110, 101, 114, 50, 45, 50]), RecordHeader(key = kafka_exception-stacktrace, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 50, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 64, 55, 49, 56, 57, 49, 100, 54, 98, 93, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 97, 115, 121, 110, 99, 70, 97, 105, 108, 117, 114, 101, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 54, 55, 55, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 108, 97, 109, 98, 100, 97, 36, 104, 97, 110, 100, 108, 101, 82, 101, 115, 117, 108, 116, 36, 49, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 52, 57, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 117, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 54, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 85, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 46, 116, 114, 121, 70, 105, 114, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 52, 49, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 112, 111, 115, 116, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 53, 49, 48, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 55, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 101, 120, 101, 99, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 48, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 100, 111, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 106, 97, 118, 97, 58, 51, 55, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 36, 87, 111, 114, 107, 81, 117, 101, 117, 101, 46, 116, 111, 112, 76, 101, 118, 101, 108, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 49, 56, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 115, 99, 97, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 53, 53, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 114, 117, 110, 87, 111, 114, 107, 101, 114, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 50, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 114, 117, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 106, 97, 118, 97, 58, 49, 54, 53, 41, 10, 67, 97, 117, 115, 101, 100, 32, 98, 121, 58, 32, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53, 45, 108, 105, 115, 116, 101, 110, 101, 114, 50, 45, 50, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 50, 46, 108, 97, 109, 98, 100, 97, 36, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 50, 36, 48, 40, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 46, 106, 97, 118, 97, 58, 54, 53, 55, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 56, 41, 10, 9, 46, 46, 46, 32, 54, 32, 109, 111, 114, 101, 10]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -105]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 5]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 36, -22])], isReadOnly = false), key = 0, value = Testing topic 5 - 1) - - // ConsumerRecord(topic = myRetryFutureTopic5, partition = 0, leaderEpoch = 0, offset = 0, CreateTime = 1727697786516, serialized key size = 1, serialized value size = 19, headers = RecordHeaders(headers = [], isReadOnly = false), key = 0, value = Testing topic 5 - 0) - // ConsumerRecord(topic = myRetryFutureTopic5-listener1-0, partition = 0, leaderEpoch = 0, offset = 0, CreateTime = 1727697787141, serialized key size = 1, serialized value size = 19, headers = RecordHeaders(headers = [RecordHeader(key = kafka_exception-fqcn, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-cause-fqcn, value = [106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-message, value = [65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 64, 52, 52, 101, 48, 99, 51, 100, 93, 59, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53]), RecordHeader(key = kafka_exception-stacktrace, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 64, 52, 52, 101, 48, 99, 51, 100, 93, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 97, 115, 121, 110, 99, 70, 97, 105, 108, 117, 114, 101, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 54, 55, 55, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 108, 97, 109, 98, 100, 97, 36, 104, 97, 110, 100, 108, 101, 82, 101, 115, 117, 108, 116, 36, 49, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 52, 57, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 117, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 54, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 117, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 83, 116, 97, 103, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 56, 55, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 119, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 50, 51, 50, 53, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 104, 97, 110, 100, 108, 101, 82, 101, 115, 117, 108, 116, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 52, 56, 54, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 105, 110, 118, 111, 107, 101, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 51, 57, 50, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 82, 101, 99, 111, 114, 100, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 111, 110, 77, 101, 115, 115, 97, 103, 101, 40, 82, 101, 99, 111, 114, 100, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 56, 53, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 82, 101, 99, 111, 114, 100, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 111, 110, 77, 101, 115, 115, 97, 103, 101, 40, 82, 101, 99, 111, 114, 100, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 53, 48, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 75, 97, 102, 107, 97, 66, 97, 99, 107, 111, 102, 102, 65, 119, 97, 114, 101, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 105, 110, 118, 111, 107, 101, 68, 101, 108, 101, 103, 97, 116, 101, 79, 110, 77, 101, 115, 115, 97, 103, 101, 40, 75, 97, 102, 107, 97, 66, 97, 99, 107, 111, 102, 102, 65, 119, 97, 114, 101, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 49, 49, 53, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 75, 97, 102, 107, 97, 66, 97, 99, 107, 111, 102, 102, 65, 119, 97, 114, 101, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 111, 110, 77, 101, 115, 115, 97, 103, 101, 40, 75, 97, 102, 107, 97, 66, 97, 99, 107, 111, 102, 102, 65, 119, 97, 114, 101, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 49, 48, 54, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 75, 97, 102, 107, 97, 66, 97, 99, 107, 111, 102, 102, 65, 119, 97, 114, 101, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 111, 110, 77, 101, 115, 115, 97, 103, 101, 40, 75, 97, 102, 107, 97, 66, 97, 99, 107, 111, 102, 102, 65, 119, 97, 114, 101, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 53, 48, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 100, 111, 73, 110, 118, 111, 107, 101, 79, 110, 77, 101, 115, 115, 97, 103, 101, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 50, 56, 50, 49, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 105, 110, 118, 111, 107, 101, 79, 110, 77, 101, 115, 115, 97, 103, 101, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 50, 55, 57, 57, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 108, 97, 109, 98, 100, 97, 36, 100, 111, 73, 110, 118, 111, 107, 101, 82, 101, 99, 111, 114, 100, 76, 105, 115, 116, 101, 110, 101, 114, 36, 53, 52, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 50, 55, 49, 55, 41, 10, 9, 97, 116, 32, 105, 111, 46, 109, 105, 99, 114, 111, 109, 101, 116, 101, 114, 46, 111, 98, 115, 101, 114, 118, 97, 116, 105, 111, 110, 46, 79, 98, 115, 101, 114, 118, 97, 116, 105, 111, 110, 46, 111, 98, 115, 101, 114, 118, 101, 40, 79, 98, 115, 101, 114, 118, 97, 116, 105, 111, 110, 46, 106, 97, 118, 97, 58, 53, 54, 53, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 100, 111, 73, 110, 118, 111, 107, 101, 82, 101, 99, 111, 114, 100, 76, 105, 115, 116, 101, 110, 101, 114, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 50, 55, 49, 53, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 100, 111, 73, 110, 118, 111, 107, 101, 87, 105, 116, 104, 82, 101, 99, 111, 114, 100, 115, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 50, 53, 53, 55, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 105, 110, 118, 111, 107, 101, 82, 101, 99, 111, 114, 100, 76, 105, 115, 116, 101, 110, 101, 114, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 50, 52, 52, 54, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 105, 110, 118, 111, 107, 101, 76, 105, 115, 116, 101, 110, 101, 114, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 50, 48, 57, 55, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 105, 110, 118, 111, 107, 101, 73, 102, 72, 97, 118, 101, 82, 101, 99, 111, 114, 100, 115, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 49, 52, 55, 51, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 112, 111, 108, 108, 65, 110, 100, 73, 110, 118, 111, 107, 101, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 49, 52, 51, 56, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 36, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 115, 117, 109, 101, 114, 46, 114, 117, 110, 40, 75, 97, 102, 107, 97, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 67, 111, 110, 116, 97, 105, 110, 101, 114, 46, 106, 97, 118, 97, 58, 49, 51, 48, 55, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 82, 117, 110, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 56, 48, 52, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 84, 104, 114, 101, 97, 100, 46, 114, 117, 110, 40, 84, 104, 114, 101, 97, 100, 46, 106, 97, 118, 97, 58, 56, 51, 51, 41, 10, 67, 97, 117, 115, 101, 100, 32, 98, 121, 58, 32, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 46, 108, 97, 109, 98, 100, 97, 36, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 36, 48, 40, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 46, 106, 97, 118, 97, 58, 54, 50, 53, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 56, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 101, 120, 101, 99, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 48, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 100, 111, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 106, 97, 118, 97, 58, 51, 55, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 36, 87, 111, 114, 107, 81, 117, 101, 117, 101, 46, 116, 111, 112, 76, 101, 118, 101, 108, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 49, 56, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 115, 99, 97, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 53, 53, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 114, 117, 110, 87, 111, 114, 107, 101, 114, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 50, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 114, 117, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 106, 97, 118, 97, 58, 49, 54, 53, 41, 10]), RecordHeader(key = kafka_original-topic, value = [109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53]), RecordHeader(key = kafka_original-partition, value = [0, 0, 0, 0]), RecordHeader(key = kafka_original-offset, value = [0, 0, 0, 0, 0, 0, 0, 0]), RecordHeader(key = kafka_original-timestamp, value = [0, 0, 1, -110, 66, -48, 22, -108]), RecordHeader(key = kafka_original-timestamp-type, value = [67, 114, 101, 97, 116, 101, 84, 105, 109, 101]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -108]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 2]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 28, -21])], isReadOnly = false), key = 0, value = Testing topic 5 - 0) - // ConsumerRecord(topic = myRetryFutureTopic5-listener1-1, partition = 0, leaderEpoch = 0, offset = 0, CreateTime = 1727697788165, serialized key size = 1, serialized value size = 19, headers = RecordHeaders(headers = [RecordHeader(key = kafka_original-topic, value = [109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53]), RecordHeader(key = kafka_original-partition, value = [0, 0, 0, 0]), RecordHeader(key = kafka_original-offset, value = [0, 0, 0, 0, 0, 0, 0, 0]), RecordHeader(key = kafka_original-timestamp, value = [0, 0, 1, -110, 66, -48, 22, -108]), RecordHeader(key = kafka_original-timestamp-type, value = [67, 114, 101, 97, 116, 101, 84, 105, 109, 101]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -108]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 2]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 28, -21]), RecordHeader(key = kafka_exception-fqcn, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-cause-fqcn, value = [106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-message, value = [65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 64, 52, 52, 101, 48, 99, 51, 100, 93, 59, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53, 45, 108, 105, 115, 116, 101, 110, 101, 114, 49, 45, 48]), RecordHeader(key = kafka_exception-stacktrace, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 64, 52, 52, 101, 48, 99, 51, 100, 93, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 97, 115, 121, 110, 99, 70, 97, 105, 108, 117, 114, 101, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 54, 55, 55, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 108, 97, 109, 98, 100, 97, 36, 104, 97, 110, 100, 108, 101, 82, 101, 115, 117, 108, 116, 36, 49, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 52, 57, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 117, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 54, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 85, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 46, 116, 114, 121, 70, 105, 114, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 52, 49, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 112, 111, 115, 116, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 53, 49, 48, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 55, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 101, 120, 101, 99, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 48, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 100, 111, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 106, 97, 118, 97, 58, 51, 55, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 36, 87, 111, 114, 107, 81, 117, 101, 117, 101, 46, 116, 111, 112, 76, 101, 118, 101, 108, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 49, 56, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 115, 99, 97, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 53, 53, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 114, 117, 110, 87, 111, 114, 107, 101, 114, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 50, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 114, 117, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 106, 97, 118, 97, 58, 49, 54, 53, 41, 10, 67, 97, 117, 115, 101, 100, 32, 98, 121, 58, 32, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53, 45, 108, 105, 115, 116, 101, 110, 101, 114, 49, 45, 48, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 46, 108, 97, 109, 98, 100, 97, 36, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 36, 48, 40, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 46, 106, 97, 118, 97, 58, 54, 50, 53, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 56, 41, 10, 9, 46, 46, 46, 32, 54, 32, 109, 111, 114, 101, 10]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -108]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 3]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 32, -20])], isReadOnly = false), key = 0, value = Testing topic 5 - 0) - // ConsumerRecord(topic = myRetryFutureTopic5-listener1-2, partition = 0, leaderEpoch = 0, offset = 0, CreateTime = 1727697789173, serialized key size = 1, serialized value size = 19, headers = RecordHeaders(headers = [RecordHeader(key = kafka_original-topic, value = [109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53]), RecordHeader(key = kafka_original-partition, value = [0, 0, 0, 0]), RecordHeader(key = kafka_original-offset, value = [0, 0, 0, 0, 0, 0, 0, 0]), RecordHeader(key = kafka_original-timestamp, value = [0, 0, 1, -110, 66, -48, 22, -108]), RecordHeader(key = kafka_original-timestamp-type, value = [67, 114, 101, 97, 116, 101, 84, 105, 109, 101]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -108]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 2]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 28, -21]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -108]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 3]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 32, -20]), RecordHeader(key = kafka_exception-fqcn, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-cause-fqcn, value = [106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-message, value = [65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 64, 52, 52, 101, 48, 99, 51, 100, 93, 59, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53, 45, 108, 105, 115, 116, 101, 110, 101, 114, 49, 45, 49]), RecordHeader(key = kafka_exception-stacktrace, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 64, 52, 52, 101, 48, 99, 51, 100, 93, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 97, 115, 121, 110, 99, 70, 97, 105, 108, 117, 114, 101, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 54, 55, 55, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 108, 97, 109, 98, 100, 97, 36, 104, 97, 110, 100, 108, 101, 82, 101, 115, 117, 108, 116, 36, 49, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 52, 57, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 117, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 54, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 85, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 46, 116, 114, 121, 70, 105, 114, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 52, 49, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 112, 111, 115, 116, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 53, 49, 48, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 55, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 101, 120, 101, 99, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 48, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 100, 111, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 106, 97, 118, 97, 58, 51, 55, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 36, 87, 111, 114, 107, 81, 117, 101, 117, 101, 46, 116, 111, 112, 76, 101, 118, 101, 108, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 49, 56, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 115, 99, 97, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 53, 53, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 114, 117, 110, 87, 111, 114, 107, 101, 114, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 50, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 114, 117, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 106, 97, 118, 97, 58, 49, 54, 53, 41, 10, 67, 97, 117, 115, 101, 100, 32, 98, 121, 58, 32, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53, 45, 108, 105, 115, 116, 101, 110, 101, 114, 49, 45, 49, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 46, 108, 97, 109, 98, 100, 97, 36, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 36, 48, 40, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 46, 106, 97, 118, 97, 58, 54, 50, 53, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 56, 41, 10, 9, 46, 46, 46, 32, 54, 32, 109, 111, 114, 101, 10]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -108]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 4]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 36, -35])], isReadOnly = false), key = 0, value = Testing topic 5 - 0) - // ConsumerRecord(topic = myRetryFutureTopic5-listener1-dlt, partition = 0, leaderEpoch = 0, offset = 0, CreateTime = 1727697790187, serialized key size = 1, serialized value size = 19, headers = RecordHeaders(headers = [RecordHeader(key = kafka_original-topic, value = [109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53]), RecordHeader(key = kafka_original-partition, value = [0, 0, 0, 0]), RecordHeader(key = kafka_original-offset, value = [0, 0, 0, 0, 0, 0, 0, 0]), RecordHeader(key = kafka_original-timestamp, value = [0, 0, 1, -110, 66, -48, 22, -108]), RecordHeader(key = kafka_original-timestamp-type, value = [67, 114, 101, 97, 116, 101, 84, 105, 109, 101]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -108]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 2]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 28, -21]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -108]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 3]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 32, -20]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -108]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 4]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 36, -35]), RecordHeader(key = kafka_exception-fqcn, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-cause-fqcn, value = [106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110]), RecordHeader(key = kafka_exception-message, value = [65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 64, 52, 52, 101, 48, 99, 51, 100, 93, 59, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53, 45, 108, 105, 115, 116, 101, 110, 101, 114, 49, 45, 50]), RecordHeader(key = kafka_exception-stacktrace, value = [111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 76, 105, 115, 116, 101, 110, 101, 114, 69, 120, 101, 99, 117, 116, 105, 111, 110, 70, 97, 105, 108, 101, 100, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 115, 121, 110, 99, 32, 70, 97, 105, 108, 10, 69, 110, 100, 112, 111, 105, 110, 116, 32, 104, 97, 110, 100, 108, 101, 114, 32, 100, 101, 116, 97, 105, 108, 115, 58, 10, 77, 101, 116, 104, 111, 100, 32, 91, 112, 117, 98, 108, 105, 99, 32, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 60, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 86, 111, 105, 100, 62, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 46, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 40, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 44, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 83, 116, 114, 105, 110, 103, 41, 93, 10, 66, 101, 97, 110, 32, 91, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 64, 52, 52, 101, 48, 99, 51, 100, 93, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 97, 115, 121, 110, 99, 70, 97, 105, 108, 117, 114, 101, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 54, 55, 55, 41, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 108, 105, 115, 116, 101, 110, 101, 114, 46, 97, 100, 97, 112, 116, 101, 114, 46, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 108, 97, 109, 98, 100, 97, 36, 104, 97, 110, 100, 108, 101, 82, 101, 115, 117, 108, 116, 36, 49, 40, 77, 101, 115, 115, 97, 103, 105, 110, 103, 77, 101, 115, 115, 97, 103, 101, 76, 105, 115, 116, 101, 110, 101, 114, 65, 100, 97, 112, 116, 101, 114, 46, 106, 97, 118, 97, 58, 52, 57, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 117, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 54, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 85, 110, 105, 87, 104, 101, 110, 67, 111, 109, 112, 108, 101, 116, 101, 46, 116, 114, 121, 70, 105, 114, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 56, 52, 49, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 112, 111, 115, 116, 67, 111, 109, 112, 108, 101, 116, 101, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 53, 49, 48, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 55, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 101, 120, 101, 99, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 48, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 100, 111, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 84, 97, 115, 107, 46, 106, 97, 118, 97, 58, 51, 55, 51, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 36, 87, 111, 114, 107, 81, 117, 101, 117, 101, 46, 116, 111, 112, 76, 101, 118, 101, 108, 69, 120, 101, 99, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 49, 56, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 115, 99, 97, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 53, 53, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 114, 117, 110, 87, 111, 114, 107, 101, 114, 40, 70, 111, 114, 107, 74, 111, 105, 110, 80, 111, 111, 108, 46, 106, 97, 118, 97, 58, 49, 54, 50, 50, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 114, 117, 110, 40, 70, 111, 114, 107, 74, 111, 105, 110, 87, 111, 114, 107, 101, 114, 84, 104, 114, 101, 97, 100, 46, 106, 97, 118, 97, 58, 49, 54, 53, 41, 10, 67, 97, 117, 115, 101, 100, 32, 98, 121, 58, 32, 106, 97, 118, 97, 46, 108, 97, 110, 103, 46, 82, 117, 110, 116, 105, 109, 101, 69, 120, 99, 101, 112, 116, 105, 111, 110, 58, 32, 65, 110, 110, 111, 116, 97, 116, 101, 100, 32, 119, 111, 111, 111, 111, 112, 115, 46, 46, 46, 32, 109, 121, 82, 101, 116, 114, 121, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 53, 45, 108, 105, 115, 116, 101, 110, 101, 114, 49, 45, 50, 10, 9, 97, 116, 32, 111, 114, 103, 46, 115, 112, 114, 105, 110, 103, 102, 114, 97, 109, 101, 119, 111, 114, 107, 46, 107, 97, 102, 107, 97, 46, 114, 101, 116, 114, 121, 116, 111, 112, 105, 99, 46, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 36, 70, 105, 102, 116, 104, 70, 117, 116, 117, 114, 101, 84, 111, 112, 105, 99, 76, 105, 115, 116, 101, 110, 101, 114, 49, 46, 108, 97, 109, 98, 100, 97, 36, 108, 105, 115, 116, 101, 110, 87, 105, 116, 104, 65, 110, 110, 111, 116, 97, 116, 105, 111, 110, 36, 48, 40, 65, 115, 121, 110, 99, 82, 101, 116, 114, 121, 84, 111, 112, 105, 99, 67, 108, 97, 115, 115, 76, 101, 118, 101, 108, 73, 110, 116, 101, 103, 114, 97, 116, 105, 111, 110, 84, 101, 115, 116, 115, 46, 106, 97, 118, 97, 58, 54, 50, 53, 41, 10, 9, 97, 116, 32, 106, 97, 118, 97, 46, 98, 97, 115, 101, 47, 106, 97, 118, 97, 46, 117, 116, 105, 108, 46, 99, 111, 110, 99, 117, 114, 114, 101, 110, 116, 46, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 36, 65, 115, 121, 110, 99, 83, 117, 112, 112, 108, 121, 46, 114, 117, 110, 40, 67, 111, 109, 112, 108, 101, 116, 97, 98, 108, 101, 70, 117, 116, 117, 114, 101, 46, 106, 97, 118, 97, 58, 49, 55, 54, 56, 41, 10, 9, 46, 46, 46, 32, 54, 32, 109, 111, 114, 101, 10]), RecordHeader(key = retry_topic-original-timestamp, value = [1, -110, 66, -48, 22, -108]), RecordHeader(key = retry_topic-attempts, value = [0, 0, 0, 5]), RecordHeader(key = retry_topic-backoff-timestamp, value = [1, -110, 66, -48, 36, -22])], isReadOnly = false), key = 0, value = Testing topic 5 - 0) try { invokeDelegateOnMessage(consumerRecord, acknowledgment, consumer); } @@ -157,7 +148,7 @@ public void onMessage(ConsumerRecord data, Consumer consumer) { } @Override - public void setAsyncRetryCallback(BiConsumer, RuntimeException> asyncRetryCallback) { - this.delegate.setAsyncRetryCallback(asyncRetryCallback); + public void setCallbackForAsyncFailureQueue(java.util.function.Consumer callbackForAsyncFailureQueue) { + this.delegate.setCallbackForAsyncFailureQueue(callbackForAsyncFailureQueue); } } diff --git a/spring-kafka/src/main/java/org/springframework/kafka/listener/adapter/MessagingMessageListenerAdapter.java b/spring-kafka/src/main/java/org/springframework/kafka/listener/adapter/MessagingMessageListenerAdapter.java index 4091a3c27..fed47f5f5 100644 --- a/spring-kafka/src/main/java/org/springframework/kafka/listener/adapter/MessagingMessageListenerAdapter.java +++ b/spring-kafka/src/main/java/org/springframework/kafka/listener/adapter/MessagingMessageListenerAdapter.java @@ -47,8 +47,8 @@ import org.springframework.expression.spel.standard.SpelExpressionParser; import org.springframework.expression.spel.support.StandardEvaluationContext; import org.springframework.expression.spel.support.StandardTypeConverter; +import org.springframework.kafka.core.FailedRecordTuple; import org.springframework.kafka.core.KafkaTemplate; -import org.springframework.kafka.listener.AsyncRetryableException; import org.springframework.kafka.listener.ConsumerSeekAware; import org.springframework.kafka.listener.KafkaListenerErrorHandler; import org.springframework.kafka.listener.ListenerExecutionFailedException; @@ -94,6 +94,7 @@ * @author Wang ZhiYang * @author Huijin Hong * @author Soby Chacko + * @author Sanghyeok An */ public abstract class MessagingMessageListenerAdapter implements ConsumerSeekAware, AsyncRepliesAware { @@ -157,6 +158,8 @@ public abstract class MessagingMessageListenerAdapter implements ConsumerS private BiConsumer, RuntimeException> asyncRetryCallback; + private java.util.function.Consumer callbackForAsyncFailureQueue; + /** * Create an instance with the provided bean and method. * @param bean the bean. @@ -164,6 +167,7 @@ public abstract class MessagingMessageListenerAdapter implements ConsumerS */ protected MessagingMessageListenerAdapter(Object bean, Method method) { this(bean, method, null); + System.out.println("here"); } /** @@ -689,7 +693,8 @@ protected void asyncFailure(Object request, @Nullable Acknowledgment acknowledgm if (request instanceof ConsumerRecord && ex instanceof RuntimeException) { ConsumerRecord record = (ConsumerRecord) request; - asyncRetryCallback.accept(record, (RuntimeException) ex); + FailedRecordTuple failedRecordTuple = new FailedRecordTuple(record, (RuntimeException) ex); + this.callbackForAsyncFailureQueue.accept(failedRecordTuple); } } } @@ -910,4 +915,8 @@ public void setAsyncRetryCallback(BiConsumer, RuntimeExcept this.asyncRetryCallback = asyncRetryCallback; } + public void putInAsyncFailureQueue(java.util.function.Consumer callbackForAsyncFailureQueue) { + this.callbackForAsyncFailureQueue = callbackForAsyncFailureQueue; + } + } diff --git a/spring-kafka/src/test/java/org/springframework/kafka/retrytopic/AsyncRetryTopicClassLevelIntegrationTests.java b/spring-kafka/src/test/java/org/springframework/kafka/retrytopic/AsyncCompletableFutureRetryTopicClassLevelIntegrationTests.java similarity index 52% rename from spring-kafka/src/test/java/org/springframework/kafka/retrytopic/AsyncRetryTopicClassLevelIntegrationTests.java rename to spring-kafka/src/test/java/org/springframework/kafka/retrytopic/AsyncCompletableFutureRetryTopicClassLevelIntegrationTests.java index 97a629b3e..45f4d3947 100644 --- a/spring-kafka/src/test/java/org/springframework/kafka/retrytopic/AsyncRetryTopicClassLevelIntegrationTests.java +++ b/spring-kafka/src/test/java/org/springframework/kafka/retrytopic/AsyncCompletableFutureRetryTopicClassLevelIntegrationTests.java @@ -1,7 +1,24 @@ +/* + * Copyright 2018-2021 the original author or authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.springframework.kafka.retrytopic; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.fail; +import static org.awaitility.Awaitility.await; import java.lang.reflect.Method; import java.util.ArrayList; @@ -11,6 +28,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -20,11 +38,14 @@ import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.jupiter.api.Test; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.context.annotation.Bean; @@ -35,7 +56,6 @@ import org.springframework.kafka.annotation.KafkaListener; import org.springframework.kafka.annotation.PartitionOffset; import org.springframework.kafka.annotation.RetryableTopic; -import org.springframework.kafka.annotation.TopicPartition; import org.springframework.kafka.config.ConcurrentKafkaListenerContainerFactory; import org.springframework.kafka.config.KafkaListenerEndpointRegistry; import org.springframework.kafka.config.TopicBuilder; @@ -46,14 +66,11 @@ import org.springframework.kafka.core.KafkaAdmin.NewTopics; import org.springframework.kafka.core.KafkaTemplate; import org.springframework.kafka.core.ProducerFactory; -import org.springframework.kafka.listener.AsyncRetryableException; import org.springframework.kafka.listener.ConcurrentMessageListenerContainer; import org.springframework.kafka.listener.ContainerProperties; import org.springframework.kafka.listener.ContainerProperties.AckMode; import org.springframework.kafka.listener.KafkaListenerErrorHandler; - -import org.springframework.kafka.retrytopic.RetryTopicClassLevelIntegrationTests.AbstractFifthTopicListener; -import org.springframework.kafka.retrytopic.RetryTopicClassLevelIntegrationTests.CountDownLatchContainer; +import org.springframework.kafka.support.Acknowledgment; import org.springframework.kafka.support.KafkaHeaders; import org.springframework.kafka.test.EmbeddedKafkaBroker; import org.springframework.kafka.test.context.EmbeddedKafka; @@ -65,60 +82,49 @@ import org.springframework.retry.annotation.Backoff; import org.springframework.scheduling.TaskScheduler; import org.springframework.scheduling.concurrent.ThreadPoolTaskScheduler; -import org.springframework.stereotype.Component; import org.springframework.test.annotation.DirtiesContext; import org.springframework.test.context.TestPropertySource; import org.springframework.test.context.junit.jupiter.SpringJUnitConfig; -import org.springframework.util.ReflectionUtils; -import reactor.core.publisher.Mono; + +/** + * @author Sanghyeok An + * @since 3.3.0 + */ @SpringJUnitConfig @DirtiesContext -@EmbeddedKafka(topics = { RetryTopicClassLevelIntegrationTests.FIRST_TOPIC, - RetryTopicClassLevelIntegrationTests.SECOND_TOPIC, - RetryTopicClassLevelIntegrationTests.THIRD_TOPIC, - RetryTopicClassLevelIntegrationTests.FOURTH_TOPIC, - RetryTopicClassLevelIntegrationTests.TWO_LISTENERS_TOPIC, - RetryTopicClassLevelIntegrationTests.MANUAL_TOPIC }) +@EmbeddedKafka(topics = { AsyncCompletableFutureRetryTopicClassLevelIntegrationTests.FIRST_TOPIC, + AsyncCompletableFutureRetryTopicClassLevelIntegrationTests.SECOND_TOPIC, + AsyncCompletableFutureRetryTopicClassLevelIntegrationTests.THIRD_TOPIC, + AsyncCompletableFutureRetryTopicClassLevelIntegrationTests.FOURTH_TOPIC, + AsyncCompletableFutureRetryTopicClassLevelIntegrationTests.TWO_LISTENERS_TOPIC, + AsyncCompletableFutureRetryTopicClassLevelIntegrationTests.MANUAL_TOPIC }) @TestPropertySource(properties = { "five.attempts=5", "kafka.template=customKafkaTemplate"}) -public class AsyncRetryTopicClassLevelIntegrationTests { - - public final static String FIRST_FUTURE_TOPIC = "myRetryFutureTopic1"; - - public final static String FIRST_FUTURE_LISTENER_ID = "firstFutureTopicId"; - - public final static String FIRST_MONO_TOPIC = "myRetryMonoTopic1"; - - public final static String FIRST_MONO_LISTENER_ID = "firstMonoTopicId"; +public class AsyncCompletableFutureRetryTopicClassLevelIntegrationTests { - public final static String SECOND_FUTURE_TOPIC = "myRetryFutureTopic2"; + public final static String FIRST_TOPIC = "myRetryTopic1"; - public final static String SECOND_MONO_TOPIC = "myRetryMonoTopic2"; + public final static String SECOND_TOPIC = "myRetryTopic2"; - public final static String THIRD_FUTURE_TOPIC = "myRetryFutureTopic3"; + public final static String THIRD_TOPIC = "myRetryTopic3"; - public final static String THIRD_FUTURE_LISTENER_ID = "thirdFutureTopicId"; + public final static String FOURTH_TOPIC = "myRetryTopic4"; - public final static String THIRD_MONO_TOPIC = "myRetryMonoTopic3"; + public final static String TWO_LISTENERS_TOPIC = "myRetryTopic5"; - public final static String THIRD_MONO_LISTENER_ID = "thirdMonoTopicId"; + public final static String MANUAL_TOPIC = "myRetryTopic6"; - public final static String FOURTH_FUTURE_TOPIC = "myRetryFutureTopic4"; + public final static String NOT_RETRYABLE_EXCEPTION_TOPIC = "noRetryTopic"; - public final static String FOURTH_MONO_TOPIC = "myRetryMonoTopic4"; + public final static String FIRST_REUSE_RETRY_TOPIC = "reuseRetry1"; - public final static String TWO_LISTENERS_FUTURE_TOPIC = "myRetryFutureTopic5"; + public final static String SECOND_REUSE_RETRY_TOPIC = "reuseRetry2"; - public final static String TWO_LISTENERS_MONO_TOPIC = "myRetryMonoTopic5"; + public final static String THIRD_REUSE_RETRY_TOPIC = "reuseRetry3"; private final static String MAIN_TOPIC_CONTAINER_FACTORY = "kafkaListenerContainerFactory"; - - public final static String NOT_RETRYABLE_EXCEPTION_FUTURE_TOPIC = "noRetryFutureTopic"; - - public final static String NOT_RETRYABLE_EXCEPTION_MONO_TOPIC = "noRetryMonoTopic"; - @Autowired private KafkaTemplate kafkaTemplate; @@ -129,47 +135,20 @@ public class AsyncRetryTopicClassLevelIntegrationTests { DestinationTopicContainer topicContainer; @Test - void shouldRetryFirstFutureTopic(@Autowired KafkaListenerEndpointRegistry registry) { - - kafkaTemplate.send(FIRST_FUTURE_TOPIC, "Testing topic 1"); - assertThat(topicContainer.getNextDestinationTopicFor(FIRST_FUTURE_LISTENER_ID, FIRST_FUTURE_TOPIC).getDestinationName()) - .isEqualTo(FIRST_FUTURE_TOPIC + "-retry"); - assertThat(awaitLatch(latchContainer.futureCountDownLatch1)).isTrue(); + void shouldRetryFirstTopic(@Autowired KafkaListenerEndpointRegistry registry) { + kafkaTemplate.send(FIRST_TOPIC, "Testing topic 1"); + assertThat(topicContainer.getNextDestinationTopicFor("firstTopicId", FIRST_TOPIC).getDestinationName()) + .isEqualTo("myRetryTopic1-retry"); + assertThat(awaitLatch(latchContainer.countDownLatch1)).isTrue(); assertThat(awaitLatch(latchContainer.customDltCountdownLatch)).isTrue(); - assertThat(awaitLatch(latchContainer.customMessageConverterCountdownLatch)).isTrue(); assertThat(awaitLatch(latchContainer.customErrorHandlerCountdownLatch)).isTrue(); - registry.getListenerContainerIds().stream() - .filter(id -> id.startsWith(FIRST_FUTURE_LISTENER_ID)) - .forEach(id -> { - ConcurrentMessageListenerContainer container - = (ConcurrentMessageListenerContainer) registry.getListenerContainer(id); - if (id.equals(FIRST_FUTURE_LISTENER_ID)) { - assertThat(container.getConcurrency()).isEqualTo(2); - } - else { - assertThat(container.getConcurrency()) - .describedAs("Expected %s to have concurrency", id) - .isEqualTo(1); - } - }); - } - - @Test - void shouldRetryFirstMonoTopic(@Autowired KafkaListenerEndpointRegistry registry) { - - kafkaTemplate.send(FIRST_MONO_TOPIC, "Testing topic Mono 1"); - assertThat(topicContainer.getNextDestinationTopicFor(FIRST_MONO_LISTENER_ID, FIRST_MONO_TOPIC).getDestinationName()) - .isEqualTo(FIRST_MONO_TOPIC + "-retry"); - assertThat(awaitLatch(latchContainer.monoCountDownLatch1)).isTrue(); - assertThat(awaitLatch(latchContainer.customDltCountdownLatch)).isTrue(); assertThat(awaitLatch(latchContainer.customMessageConverterCountdownLatch)).isTrue(); - assertThat(awaitLatch(latchContainer.customErrorHandlerCountdownLatch)).isTrue(); registry.getListenerContainerIds().stream() - .filter(id -> id.startsWith(FIRST_MONO_LISTENER_ID)) + .filter(id -> id.startsWith("first")) .forEach(id -> { - ConcurrentMessageListenerContainer container - = (ConcurrentMessageListenerContainer) registry.getListenerContainer(id); - if (id.equals(FIRST_MONO_LISTENER_ID)) { + ConcurrentMessageListenerContainer container = (ConcurrentMessageListenerContainer) registry + .getListenerContainer(id); + if (id.equals("firstTopicId")) { assertThat(container.getConcurrency()).isEqualTo(2); } else { @@ -181,74 +160,25 @@ void shouldRetryFirstMonoTopic(@Autowired KafkaListenerEndpointRegistry registry } @Test - void shouldRetrySecondFutureTopic() { - kafkaTemplate.send(SECOND_FUTURE_TOPIC, "Testing topic 2"); - assertThat(awaitLatch(latchContainer.futureCountDownLatch2)).isTrue(); + void shouldRetrySecondTopic() { + kafkaTemplate.send(SECOND_TOPIC, "Testing topic 2"); + assertThat(awaitLatch(latchContainer.countDownLatch2)).isTrue(); assertThat(awaitLatch(latchContainer.customDltCountdownLatch)).isTrue(); } @Test - void shouldRetrySecondMonoTopic() { - kafkaTemplate.send(SECOND_MONO_TOPIC, "Testing topic 2"); - assertThat(awaitLatch(latchContainer.monoCountDownLatch2)).isTrue(); - assertThat(awaitLatch(latchContainer.customDltCountdownLatch)).isTrue(); - } - - - @Test - void shouldRetryThirdFutureTopicWithTimeout(@Autowired KafkaAdmin admin, + void shouldRetryThirdTopicWithTimeout(@Autowired KafkaAdmin admin, @Autowired KafkaListenerEndpointRegistry registry) throws Exception { - kafkaTemplate.send(THIRD_FUTURE_TOPIC, "Testing topic 3"); - assertThat(awaitLatch(latchContainer.futureCountDownLatch3)).isTrue(); - assertThat(awaitLatch(latchContainer.countDownLatchDltOne)).isTrue(); - Map topics = admin.describeTopics(THIRD_FUTURE_TOPIC, THIRD_FUTURE_TOPIC + "-dlt", FOURTH_FUTURE_TOPIC); - assertThat(topics.get(THIRD_FUTURE_TOPIC).partitions()).hasSize(2); - assertThat(topics.get(THIRD_FUTURE_TOPIC + "-dlt").partitions()).hasSize(3); - assertThat(topics.get(FOURTH_FUTURE_TOPIC).partitions()).hasSize(2); - AtomicReference method = new AtomicReference<>(); - ReflectionUtils.doWithMethods(KafkaAdmin.class, m -> { - m.setAccessible(true); - method.set(m); - }, m -> m.getName().equals("newTopics")); - @SuppressWarnings("unchecked") - Collection weededTopics = (Collection) method.get().invoke(admin); - AtomicInteger weeded = new AtomicInteger(); - weededTopics.forEach(topic -> { - if (topic.name().equals(THIRD_FUTURE_TOPIC) || topic.name().equals(FOURTH_FUTURE_TOPIC)) { - assertThat(topic).isExactlyInstanceOf(NewTopic.class); - weeded.incrementAndGet(); - } - }); - assertThat(weeded.get()).isEqualTo(2); - registry.getListenerContainerIds().stream() - .filter(id -> id.startsWith(THIRD_FUTURE_LISTENER_ID)) - .forEach(id -> { - ConcurrentMessageListenerContainer container = - (ConcurrentMessageListenerContainer) registry.getListenerContainer(id); - if (id.equals(THIRD_FUTURE_LISTENER_ID)) { - assertThat(container.getConcurrency()).isEqualTo(2); - } - else { - assertThat(container.getConcurrency()) - .describedAs("Expected %s to have concurrency", id) - .isEqualTo(1); - } - }); - } - - @Test - void shouldRetryThirdMonoTopicWithTimeout(@Autowired KafkaAdmin admin, - @Autowired KafkaListenerEndpointRegistry registry) throws Exception { - kafkaTemplate.send(THIRD_MONO_TOPIC, "Testing topic 3"); - assertThat(awaitLatch(latchContainer.monoCountDownLatch3)).isTrue(); + kafkaTemplate.send(THIRD_TOPIC, "Testing topic 3"); + assertThat(awaitLatch(latchContainer.countDownLatch3)).isTrue(); assertThat(awaitLatch(latchContainer.countDownLatchDltOne)).isTrue(); - Map topics = admin.describeTopics(THIRD_MONO_TOPIC, THIRD_MONO_TOPIC + "-dlt", FOURTH_MONO_TOPIC); - assertThat(topics.get(THIRD_MONO_TOPIC).partitions()).hasSize(2); - assertThat(topics.get(THIRD_MONO_TOPIC + "-dlt").partitions()).hasSize(3); - assertThat(topics.get(FOURTH_MONO_TOPIC).partitions()).hasSize(2); + Map topics = admin.describeTopics(THIRD_TOPIC, THIRD_TOPIC + "-dlt", FOURTH_TOPIC); + assertThat(topics.get(THIRD_TOPIC).partitions()).hasSize(2); + assertThat(topics.get(THIRD_TOPIC + "-dlt").partitions()).hasSize(3); + assertThat(topics.get(FOURTH_TOPIC).partitions()).hasSize(2); AtomicReference method = new AtomicReference<>(); - ReflectionUtils.doWithMethods(KafkaAdmin.class, m -> { + org.springframework.util.ReflectionUtils.doWithMethods(KafkaAdmin.class, m -> { m.setAccessible(true); method.set(m); }, m -> m.getName().equals("newTopics")); @@ -256,18 +186,18 @@ void shouldRetryThirdMonoTopicWithTimeout(@Autowired KafkaAdmin admin, Collection weededTopics = (Collection) method.get().invoke(admin); AtomicInteger weeded = new AtomicInteger(); weededTopics.forEach(topic -> { - if (topic.name().equals(THIRD_MONO_TOPIC) || topic.name().equals(FOURTH_MONO_TOPIC)) { + if (topic.name().equals(THIRD_TOPIC) || topic.name().equals(FOURTH_TOPIC)) { assertThat(topic).isExactlyInstanceOf(NewTopic.class); weeded.incrementAndGet(); } }); assertThat(weeded.get()).isEqualTo(2); registry.getListenerContainerIds().stream() - .filter(id -> id.startsWith(THIRD_MONO_LISTENER_ID)) + .filter(id -> id.startsWith("third")) .forEach(id -> { ConcurrentMessageListenerContainer container = (ConcurrentMessageListenerContainer) registry.getListenerContainer(id); - if (id.equals(THIRD_MONO_LISTENER_ID)) { + if (id.equals("thirdTopicId")) { assertThat(container.getConcurrency()).isEqualTo(2); } else { @@ -278,52 +208,88 @@ void shouldRetryThirdMonoTopicWithTimeout(@Autowired KafkaAdmin admin, }); } - @Test - void shouldRetryFourthFutureTopicWithNoDlt() { - kafkaTemplate.send(FOURTH_FUTURE_TOPIC, "Testing topic 4"); - assertThat(awaitLatch(latchContainer.futureCountDownLatch4)).isTrue(); + void shouldRetryFourthTopicWithNoDlt() { + kafkaTemplate.send(FOURTH_TOPIC, "Testing topic 4"); + assertThat(awaitLatch(latchContainer.countDownLatch4)).isTrue(); } @Test - void shouldRetryFourthMonoTopicWithNoDlt() { - kafkaTemplate.send(FOURTH_MONO_TOPIC, "Testing topic 4"); - assertThat(awaitLatch(latchContainer.monoCountDownLatch4)).isTrue(); - } - - - - @Test - void shouldRetryFifthTopicWithTwoListenersAndManualAssignment1(@Autowired FifthFutureTopicListener1 listener1, - @Autowired FifthFutureTopicListener2 listener2) { + void shouldRetryFifthTopicWithTwoListenersAndManualAssignment(@Autowired + FifthTopicListener1 listener1, + @Autowired FifthTopicListener2 listener2) { - kafkaTemplate.send(TWO_LISTENERS_FUTURE_TOPIC, 0, "0", "Testing topic 5 - 0"); - kafkaTemplate.send(TWO_LISTENERS_FUTURE_TOPIC, 1, "0", "Testing topic 5 - 1"); - - assertThat(awaitLatch(latchContainer.countDownLatchDltThree)).isTrue(); + kafkaTemplate.send(TWO_LISTENERS_TOPIC, 0, "0", "Testing topic 5 - 0"); + kafkaTemplate.send(TWO_LISTENERS_TOPIC, 1, "0", "Testing topic 5 - 1"); assertThat(awaitLatch(latchContainer.countDownLatch51)).isTrue(); assertThat(awaitLatch(latchContainer.countDownLatch52)).isTrue(); - assertThat(listener1.topics).containsExactly(TWO_LISTENERS_FUTURE_TOPIC, TWO_LISTENERS_FUTURE_TOPIC - + "-listener1-0", TWO_LISTENERS_FUTURE_TOPIC + "-listener1-1", TWO_LISTENERS_FUTURE_TOPIC + "-listener1-2", - TWO_LISTENERS_FUTURE_TOPIC + "-listener1-dlt"); - assertThat(listener2.topics).containsExactly(TWO_LISTENERS_FUTURE_TOPIC, TWO_LISTENERS_FUTURE_TOPIC - + "-listener2-0", TWO_LISTENERS_FUTURE_TOPIC + "-listener2-1", TWO_LISTENERS_FUTURE_TOPIC + "-listener2-2", - TWO_LISTENERS_FUTURE_TOPIC + "-listener2-dlt"); + assertThat(awaitLatch(latchContainer.countDownLatchDltThree)).isTrue(); + assertThat(listener1.topics).containsExactly(TWO_LISTENERS_TOPIC, TWO_LISTENERS_TOPIC + + "-listener1-0", TWO_LISTENERS_TOPIC + "-listener1-1", TWO_LISTENERS_TOPIC + "-listener1-2", + TWO_LISTENERS_TOPIC + "-listener1-dlt"); + assertThat(listener2.topics).containsExactly(TWO_LISTENERS_TOPIC, TWO_LISTENERS_TOPIC + + "-listener2-0", TWO_LISTENERS_TOPIC + "-listener2-1", TWO_LISTENERS_TOPIC + "-listener2-2", + TWO_LISTENERS_TOPIC + "-listener2-dlt"); } + @Test + void shouldRetryManualTopicWithDefaultDlt(@Autowired KafkaListenerEndpointRegistry registry, + @Autowired ConsumerFactory cf) { + kafkaTemplate.send(MANUAL_TOPIC, "Testing topic 6"); + assertThat(awaitLatch(latchContainer.countDownLatch6)).isTrue(); + registry.getListenerContainerIds().stream() + .filter(id -> id.startsWith("manual")) + .forEach(id -> { + ConcurrentMessageListenerContainer container = + (ConcurrentMessageListenerContainer) registry.getListenerContainer(id); + assertThat(container).extracting("commonErrorHandler") + .extracting("seekAfterError", InstanceOfAssertFactories.BOOLEAN) + .isFalse(); + }); + Consumer consumer = cf.createConsumer("manual-dlt", ""); + Set tp = + Set.of(new org.apache.kafka.common.TopicPartition(MANUAL_TOPIC + "-dlt", 0)); + consumer.assign(tp); + try { + await().untilAsserted(() -> { + OffsetAndMetadata offsetAndMetadata = consumer.committed(tp).get(tp.iterator().next()); + assertThat(offsetAndMetadata).isNotNull(); + assertThat(offsetAndMetadata.offset()).isEqualTo(1L); + }); + } + finally { + consumer.close(); + } + } @Test - void shouldGoStraightToDltInFuture() { - kafkaTemplate.send(NOT_RETRYABLE_EXCEPTION_FUTURE_TOPIC, "Testing topic with annotation 1"); - assertThat(awaitLatch(latchContainer.futureCountDownLatchNoRetry)).isTrue(); - assertThat(awaitLatch(latchContainer.countDownLatchDltTwo)).isTrue(); + void shouldFirstReuseRetryTopic(@Autowired + FirstReuseRetryTopicListener listener1, + @Autowired + SecondReuseRetryTopicListener listener2, @Autowired + ThirdReuseRetryTopicListener listener3) { + + kafkaTemplate.send(FIRST_REUSE_RETRY_TOPIC, "Testing reuse topic 1"); + kafkaTemplate.send(SECOND_REUSE_RETRY_TOPIC, "Testing reuse topic 2"); + kafkaTemplate.send(THIRD_REUSE_RETRY_TOPIC, "Testing reuse topic 3"); + assertThat(awaitLatch(latchContainer.countDownLatchReuseOne)).isTrue(); + assertThat(awaitLatch(latchContainer.countDownLatchReuseTwo)).isTrue(); + assertThat(awaitLatch(latchContainer.countDownLatchReuseThree)).isTrue(); + assertThat(listener1.topics).containsExactly(FIRST_REUSE_RETRY_TOPIC, + FIRST_REUSE_RETRY_TOPIC + "-retry"); + assertThat(listener2.topics).containsExactly(SECOND_REUSE_RETRY_TOPIC, + SECOND_REUSE_RETRY_TOPIC + "-retry-30", SECOND_REUSE_RETRY_TOPIC + "-retry-60", + SECOND_REUSE_RETRY_TOPIC + "-retry-100", SECOND_REUSE_RETRY_TOPIC + "-retry-100"); + assertThat(listener3.topics).containsExactly(THIRD_REUSE_RETRY_TOPIC, + THIRD_REUSE_RETRY_TOPIC + "-retry", THIRD_REUSE_RETRY_TOPIC + "-retry", + THIRD_REUSE_RETRY_TOPIC + "-retry", THIRD_REUSE_RETRY_TOPIC + "-retry"); } @Test - void shouldGoStraightToDltInMono() { - kafkaTemplate.send(NOT_RETRYABLE_EXCEPTION_MONO_TOPIC, "Testing topic with annotation 1"); - assertThat(awaitLatch(latchContainer.monoCountDownLatchNoRetry)).isTrue(); + void shouldGoStraightToDlt() { + kafkaTemplate.send(NOT_RETRYABLE_EXCEPTION_TOPIC, "Testing topic with annotation 1"); + assertThat(awaitLatch(latchContainer.countDownLatchNoRetry)).isTrue(); assertThat(awaitLatch(latchContainer.countDownLatchDltTwo)).isTrue(); } @@ -338,13 +304,13 @@ private boolean awaitLatch(CountDownLatch latch) { } @KafkaListener( - id = FIRST_FUTURE_LISTENER_ID, - topics = FIRST_FUTURE_TOPIC, + id = "firstTopicId", + topics = FIRST_TOPIC, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY, errorHandler = "myCustomErrorHandler", contentTypeConverter = "myCustomMessageConverter", concurrency = "2") - static class FirstFutureTopicListener { + static class FirstTopicListener { @Autowired DestinationTopicContainer topicContainer; @@ -352,11 +318,10 @@ static class FirstFutureTopicListener { @Autowired CountDownLatchContainer container; - @KafkaHandler public CompletableFuture listen(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { return CompletableFuture.supplyAsync(() -> { - container.futureCountDownLatch1.countDown(); + container.countDownLatch1.countDown(); try { Thread.sleep(1); } catch (InterruptedException e) { @@ -365,41 +330,11 @@ public CompletableFuture listen(String message, @Header(KafkaHeaders.RECEI throw new RuntimeException("Woooops... in topic " + receivedTopic); }); } - } - - - @KafkaListener( - id = FIRST_MONO_LISTENER_ID, - topics = FIRST_MONO_TOPIC, - containerFactory = MAIN_TOPIC_CONTAINER_FACTORY, - errorHandler = "myCustomErrorHandler", - contentTypeConverter = "myCustomMessageConverter", - concurrency = "2") - static class FirstTopicMonoListener { - - @Autowired - DestinationTopicContainer topicContainer; - @Autowired - CountDownLatchContainer container; - - @KafkaHandler - public Mono listen(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { - return Mono.fromCallable(() -> { - container.monoCountDownLatch1.countDown(); - try { - Thread.sleep(1); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - throw new RuntimeException("Woooops... in topic " + receivedTopic); - }).then(); - } } - - @KafkaListener(topics = SECOND_FUTURE_TOPIC, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) - static class SecondFutureTopicListener { + @KafkaListener(topics = SECOND_TOPIC, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) + static class SecondTopicListener { @Autowired CountDownLatchContainer container; @@ -407,7 +342,7 @@ static class SecondFutureTopicListener { @KafkaHandler public CompletableFuture listenAgain(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { return CompletableFuture.supplyAsync(() -> { - container.countDownIfNotKnown(receivedTopic, container.futureCountDownLatch2); + container.countDownIfNotKnown(receivedTopic, container.countDownLatch2); try { Thread.sleep(1); } catch (InterruptedException e) { @@ -418,42 +353,20 @@ public CompletableFuture listenAgain(String message, @Header(KafkaHeaders. } } - - @KafkaListener(topics = SECOND_MONO_TOPIC, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) - static class SecondMonoTopicListener { - - @Autowired - CountDownLatchContainer container; - - @KafkaHandler - public Mono listenAgain(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { - return Mono.fromCallable(() -> { - container.countDownIfNotKnown(receivedTopic, container.monoCountDownLatch2); - try { - Thread.sleep(1); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - throw new IllegalStateException("Another woooops... " + receivedTopic); - }).then(); - } - } - - @RetryableTopic( attempts = "${five.attempts}", backoff = @Backoff(delay = 250, maxDelay = 1000, multiplier = 1.5), numPartitions = "#{3}", - timeout = "${missing.property:2000}", + timeout = "${missing.property:100000}", include = MyRetryException.class, kafkaTemplate = "${kafka.template}", topicSuffixingStrategy = TopicSuffixingStrategy.SUFFIX_WITH_INDEX_VALUE, concurrency = "1") @KafkaListener( - id = THIRD_FUTURE_LISTENER_ID, - topics = THIRD_FUTURE_TOPIC, + id = "thirdTopicId", + topics = THIRD_TOPIC, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY, concurrency = "2") - static class ThirdFutureTopicListener { + static class ThirdTopicListener { @Autowired CountDownLatchContainer container; @@ -461,7 +374,7 @@ static class ThirdFutureTopicListener { @KafkaHandler public CompletableFuture listenWithAnnotation(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { return CompletableFuture.supplyAsync(() -> { - container.countDownIfNotKnown(receivedTopic, container.futureCountDownLatch3); + container.countDownIfNotKnown(receivedTopic, container.countDownLatch3); try { Thread.sleep(1); } catch (InterruptedException e) { @@ -477,54 +390,11 @@ public void annotatedDltMethod(Object message) { } } - @Component - @RetryableTopic( - attempts = "${five.attempts}", - backoff = @Backoff(delay = 250, maxDelay = 1000, multiplier = 1.5), - numPartitions = "#{3}", - timeout = "${missing.property:2000}", - include = MyRetryException.class, kafkaTemplate = "${kafka.template}", - topicSuffixingStrategy = TopicSuffixingStrategy.SUFFIX_WITH_INDEX_VALUE, - concurrency = "1") - @KafkaListener( - id = THIRD_MONO_LISTENER_ID, - topics = THIRD_MONO_TOPIC, - containerFactory = MAIN_TOPIC_CONTAINER_FACTORY, - concurrency = "2") - static class ThirdMonoTopicListener { - - @Autowired - CountDownLatchContainer container; - - @KafkaHandler - public Mono listenWithAnnotation(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { - return Mono.fromCallable(() -> { - container.countDownIfNotKnown(receivedTopic, container.monoCountDownLatch3); - try { - Thread.sleep(1); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - throw new MyRetryException("Annotated woooops... " + receivedTopic); - }).then(); - } - - @DltHandler - public void annotatedDltMethod(Object message) { - container.countDownLatchDltOne.countDown(); - } - } - - - - @RetryableTopic( - dltStrategy = DltStrategy.NO_DLT, - attempts = "4", - backoff = @Backoff(300), + @RetryableTopic(dltStrategy = DltStrategy.NO_DLT, attempts = "4", backoff = @Backoff(300), sameIntervalTopicReuseStrategy = SameIntervalTopicReuseStrategy.MULTIPLE_TOPICS, kafkaTemplate = "${kafka.template}") - @KafkaListener(topics = FOURTH_FUTURE_TOPIC, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) - static class FourthFutureTopicListener { + @KafkaListener(topics = FOURTH_TOPIC, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) + static class FourthTopicListener { @Autowired CountDownLatchContainer container; @@ -532,7 +402,7 @@ static class FourthFutureTopicListener { @KafkaHandler public CompletableFuture listenNoDlt(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { return CompletableFuture.supplyAsync(() -> { - container.countDownIfNotKnown(receivedTopic, container.futureCountDownLatch4); + container.countDownIfNotKnown(receivedTopic, container.countDownLatch4); try { Thread.sleep(1); } catch (InterruptedException e) { @@ -548,38 +418,6 @@ public void shouldNotGetHere() { } } - @RetryableTopic( - dltStrategy = DltStrategy.NO_DLT, - attempts = "4", - backoff = @Backoff(300), - sameIntervalTopicReuseStrategy = SameIntervalTopicReuseStrategy.MULTIPLE_TOPICS, - kafkaTemplate = "${kafka.template}") - @KafkaListener(topics = FOURTH_MONO_TOPIC, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) - static class FourthMonoTopicListener { - - @Autowired - CountDownLatchContainer container; - - @KafkaHandler - public Mono listenNoDlt(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { - return Mono.fromCallable(() -> { - container.countDownIfNotKnown(receivedTopic, container.monoCountDownLatch4); - try { - Thread.sleep(1); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - throw new IllegalStateException("Another woooops... " + receivedTopic); - }).then(); - } - - @DltHandler - public void shouldNotGetHere() { - fail("Dlt should not be processed!"); - } - } - - static class AbstractFifthTopicListener { final List topics = Collections.synchronizedList(new ArrayList<>()); @@ -597,7 +435,7 @@ public void annotatedDltMethod(ConsumerRecord record) { @RetryableTopic( attempts = "4", - backoff = @Backoff(1), + backoff = @Backoff(250), numPartitions = "2", retryTopicSuffix = "-listener1", dltTopicSuffix = "-listener1-dlt", @@ -606,17 +444,16 @@ public void annotatedDltMethod(ConsumerRecord record) { kafkaTemplate = "${kafka.template}") @KafkaListener( id = "fifthTopicId1", - topicPartitions = {@TopicPartition(topic = TWO_LISTENERS_FUTURE_TOPIC, + topicPartitions = {@org.springframework.kafka.annotation.TopicPartition(topic = TWO_LISTENERS_TOPIC, partitionOffsets = @PartitionOffset(partition = "0", initialOffset = "0"))}, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) - static class FifthFutureTopicListener1 extends AbstractFifthTopicListener { + static class FifthTopicListener1 extends AbstractFifthTopicListener { @KafkaHandler public CompletableFuture listenWithAnnotation(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { this.topics.add(receivedTopic); - container.countDownIfNotKnown(receivedTopic, container.countDownLatch51); return CompletableFuture.supplyAsync(() -> { - + container.countDownIfNotKnown(receivedTopic, container.countDownLatch51); try { Thread.sleep(1); } catch (InterruptedException e) { @@ -625,11 +462,12 @@ public CompletableFuture listenWithAnnotation(String message, @Header(Kafk throw new RuntimeException("Annotated woooops... " + receivedTopic); }); } + } @RetryableTopic( attempts = "4", - backoff = @Backoff(1), + backoff = @Backoff(250), numPartitions = "2", retryTopicSuffix = "-listener2", dltTopicSuffix = "-listener2-dlt", @@ -638,17 +476,16 @@ public CompletableFuture listenWithAnnotation(String message, @Header(Kafk kafkaTemplate = "${kafka.template}") @KafkaListener( id = "fifthTopicId2", - topicPartitions = {@TopicPartition(topic = TWO_LISTENERS_FUTURE_TOPIC, + topicPartitions = {@org.springframework.kafka.annotation.TopicPartition(topic = TWO_LISTENERS_TOPIC, partitionOffsets = @PartitionOffset(partition = "1", initialOffset = "0"))}, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) - static class FifthFutureTopicListener2 extends AbstractFifthTopicListener { + static class FifthTopicListener2 extends AbstractFifthTopicListener { @KafkaHandler public CompletableFuture listenWithAnnotation2(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { this.topics.add(receivedTopic); - container.countDownIfNotKnown(receivedTopic, container.countDownLatch52); return CompletableFuture.supplyAsync(() -> { - + container.countDownLatch52.countDown(); try { Thread.sleep(1); } catch (InterruptedException e) { @@ -660,57 +497,59 @@ public CompletableFuture listenWithAnnotation2(String message, @Header(Kaf } - - @Component - @RetryableTopic(attempts = "3", numPartitions = "3", exclude = MyDontRetryException.class, - backoff = @Backoff(delay = 50, maxDelay = 100, multiplier = 3), - traversingCauses = "true", kafkaTemplate = "${kafka.template}") - @KafkaListener(topics = NOT_RETRYABLE_EXCEPTION_FUTURE_TOPIC, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) - static class NoRetryFutureTopicListener { + @RetryableTopic( + attempts = "4", + backoff = @Backoff(50), + sameIntervalTopicReuseStrategy = SameIntervalTopicReuseStrategy.MULTIPLE_TOPICS) + @KafkaListener( + id = "manual", + topics = MANUAL_TOPIC, + containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) + static class SixthTopicDefaultDLTListener { @Autowired CountDownLatchContainer container; @KafkaHandler - public CompletableFuture listenWithAnnotation2(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + public CompletableFuture listenNoDlt(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic, + @SuppressWarnings("unused") Acknowledgment ack) { return CompletableFuture.supplyAsync(() -> { - container.countDownIfNotKnown(receivedTopic, container.futureCountDownLatchNoRetry); + container.countDownIfNotKnown(receivedTopic, container.countDownLatch6); try { Thread.sleep(1); } catch (InterruptedException e) { throw new RuntimeException(e); } - throw new MyDontRetryException("Annotated second woooops... " + receivedTopic); + throw new IllegalStateException("Another woooops... " + receivedTopic); }); } - @DltHandler - public void annotatedDltMethod(Object message) { - container.countDownLatchDltTwo.countDown(); - } } - @Component - @RetryableTopic(attempts = "3", numPartitions = "3", exclude = MyDontRetryException.class, + @RetryableTopic( + attempts = "3", + numPartitions = "3", + exclude = MyDontRetryException.class, backoff = @Backoff(delay = 50, maxDelay = 100, multiplier = 3), - traversingCauses = "true", kafkaTemplate = "${kafka.template}") - @KafkaListener(topics = NOT_RETRYABLE_EXCEPTION_MONO_TOPIC, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) - static class NoRetryMonoTopicListener { + traversingCauses = "true", + kafkaTemplate = "${kafka.template}") + @KafkaListener(topics = NOT_RETRYABLE_EXCEPTION_TOPIC, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) + static class NoRetryTopicListener { @Autowired CountDownLatchContainer container; @KafkaHandler - public Mono listenWithAnnotation2(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { - return Mono.fromCallable(() -> { - container.countDownIfNotKnown(receivedTopic, container.monoCountDownLatchNoRetry); + public CompletableFuture listenWithAnnotation2(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + return CompletableFuture.supplyAsync(() -> { + container.countDownIfNotKnown(receivedTopic, container.countDownLatchNoRetry); try { Thread.sleep(1); } catch (InterruptedException e) { throw new RuntimeException(e); } throw new MyDontRetryException("Annotated second woooops... " + receivedTopic); - }).then(); + }); } @DltHandler @@ -719,57 +558,102 @@ public void annotatedDltMethod(Object message) { } } + @RetryableTopic( + attempts = "2", + backoff = @Backoff(50)) + @KafkaListener( + id = "reuseRetry1", + topics = FIRST_REUSE_RETRY_TOPIC, + containerFactory = "retryTopicListenerContainerFactory") + static class FirstReuseRetryTopicListener { - @Configuration - static class KafkaProducerConfig { + final List topics = Collections.synchronizedList(new ArrayList<>()); @Autowired - EmbeddedKafkaBroker broker; + CountDownLatchContainer container; - @Bean - ProducerFactory producerFactory() { - Map configProps = new HashMap<>(); - configProps.put( - ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, - this.broker.getBrokersAsString()); - configProps.put( - ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, - StringSerializer.class); - configProps.put( - ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, - StringSerializer.class); - return new DefaultKafkaProducerFactory<>(configProps); + @KafkaHandler + public CompletableFuture listen1(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + this.topics.add(receivedTopic); + return CompletableFuture.supplyAsync(() -> { + container.countDownLatchReuseOne.countDown(); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new RuntimeException("Another woooops... " + receivedTopic); + }); } - @Bean("customKafkaTemplate") - KafkaTemplate kafkaTemplate() { - return new KafkaTemplate<>(producerFactory()); - } + } - @Bean - CountDownLatchContainer latchContainer() { - return new CountDownLatchContainer(); + @RetryableTopic( + attempts = "5", + backoff = @Backoff(delay = 30, maxDelay = 100, multiplier = 2)) + @KafkaListener( + id = "reuseRetry2", + topics = SECOND_REUSE_RETRY_TOPIC, + containerFactory = "retryTopicListenerContainerFactory") + static class SecondReuseRetryTopicListener { + + final List topics = Collections.synchronizedList(new ArrayList<>()); + + @Autowired + CountDownLatchContainer container; + + @KafkaHandler + public CompletableFuture listen2(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + this.topics.add(receivedTopic); + return CompletableFuture.supplyAsync(() -> { + container.countDownLatchReuseTwo.countDown(); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new RuntimeException("Another woooops... " + receivedTopic); + }); } + } - @Component - static class CountDownLatchContainer { + @RetryableTopic(attempts = "5", backoff = @Backoff(delay = 1, maxDelay = 5, multiplier = 1.4)) + @KafkaListener(id = "reuseRetry3", topics = THIRD_REUSE_RETRY_TOPIC, + containerFactory = "retryTopicListenerContainerFactory") + static class ThirdReuseRetryTopicListener { - CountDownLatch futureCountDownLatch1 = new CountDownLatch(5); + final List topics = Collections.synchronizedList(new ArrayList<>()); + + @Autowired + CountDownLatchContainer container; + + @KafkaHandler + public CompletableFuture listen3(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + this.topics.add(receivedTopic); + return CompletableFuture.supplyAsync(() -> { + container.countDownLatchReuseThree.countDown(); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new RuntimeException("Another woooops... " + receivedTopic); + }); - CountDownLatch monoCountDownLatch1 = new CountDownLatch(5); + } - CountDownLatch futureCountDownLatch2 = new CountDownLatch(3); + } - CountDownLatch monoCountDownLatch2 = new CountDownLatch(3); + static class CountDownLatchContainer { - CountDownLatch futureCountDownLatch3 = new CountDownLatch(3); + CountDownLatch countDownLatch1 = new CountDownLatch(5); - CountDownLatch monoCountDownLatch3 = new CountDownLatch(3); + CountDownLatch countDownLatch2 = new CountDownLatch(3); - CountDownLatch futureCountDownLatch4 = new CountDownLatch(4); + CountDownLatch countDownLatch3 = new CountDownLatch(3); - CountDownLatch monoCountDownLatch4 = new CountDownLatch(4); + CountDownLatch countDownLatch4 = new CountDownLatch(4); CountDownLatch countDownLatch51 = new CountDownLatch(4); @@ -777,9 +661,7 @@ static class CountDownLatchContainer { CountDownLatch countDownLatch6 = new CountDownLatch(4); - CountDownLatch futureCountDownLatchNoRetry = new CountDownLatch(1); - - CountDownLatch monoCountDownLatchNoRetry = new CountDownLatch(1); + CountDownLatch countDownLatchNoRetry = new CountDownLatch(1); CountDownLatch countDownLatchDltOne = new CountDownLatch(1); @@ -811,6 +693,20 @@ private void countDownIfNotKnown(String receivedTopic, CountDownLatch countDownL } } + static class MyCustomDltProcessor { + + @Autowired + KafkaTemplate kafkaTemplate; + + @Autowired + CountDownLatchContainer container; + + public void processDltMessage(Object message) { + container.customDltCountdownLatch.countDown(); + throw new RuntimeException("Dlt Error!"); + } + } + @SuppressWarnings("serial") static class MyRetryException extends RuntimeException { MyRetryException(String msg) { @@ -831,166 +727,144 @@ static class RetryTopicConfigurations extends RetryTopicConfigurationSupport { private static final String DLT_METHOD_NAME = "processDltMessage"; @Bean - FirstFutureTopicListener firstTopicListener() { - return new FirstFutureTopicListener(); - } - - @Bean - FirstTopicMonoListener firstTopicMonoListener() { - return new FirstTopicMonoListener(); + RetryTopicConfiguration firstRetryTopic(KafkaTemplate template) { + return RetryTopicConfigurationBuilder + .newInstance() + .fixedBackOff(50) + .maxAttempts(5) + .concurrency(1) + .useSingleTopicForSameIntervals() + .includeTopic(FIRST_TOPIC) + .doNotRetryOnDltFailure() + .dltHandlerMethod("myCustomDltProcessor", DLT_METHOD_NAME) + .create(template); } @Bean - SecondFutureTopicListener secondFutureTopicListener() { - return new SecondFutureTopicListener(); + RetryTopicConfiguration secondRetryTopic(KafkaTemplate template) { + return RetryTopicConfigurationBuilder + .newInstance() + .exponentialBackoff(500, 2, 10000) + .retryOn(Arrays.asList(IllegalStateException.class, IllegalAccessException.class)) + .traversingCauses() + .includeTopic(SECOND_TOPIC) + .doNotRetryOnDltFailure() + .dltHandlerMethod("myCustomDltProcessor", DLT_METHOD_NAME) + .create(template); } @Bean - SecondMonoTopicListener secondMonoTopicListener() { - return new SecondMonoTopicListener(); + FirstTopicListener firstTopicListener() { + return new FirstTopicListener(); } @Bean - ThirdFutureTopicListener thirdFutureTopicListener() { - return new ThirdFutureTopicListener(); + KafkaListenerErrorHandler myCustomErrorHandler( + CountDownLatchContainer container) { + return (message, exception) -> { + container.customErrorHandlerCountdownLatch.countDown(); + throw exception; + }; } @Bean - ThirdMonoTopicListener thirdMonoTopicListener() { - return new ThirdMonoTopicListener(); - } + SmartMessageConverter myCustomMessageConverter( + CountDownLatchContainer container) { + return new CompositeMessageConverter(Collections.singletonList(new GenericMessageConverter())) { - @Bean - FourthFutureTopicListener fourthFutureTopicListener() { - return new FourthFutureTopicListener(); + @Override + public Object fromMessage(Message message, Class targetClass, Object conversionHint) { + container.customMessageConverterCountdownLatch.countDown(); + return super.fromMessage(message, targetClass, conversionHint); + } + }; } @Bean - FourthMonoTopicListener fourthMonoTopicListener() { - return new FourthMonoTopicListener(); + SecondTopicListener secondTopicListener() { + return new SecondTopicListener(); } @Bean - NoRetryFutureTopicListener noRetryFutureTopicListener() { - return new NoRetryFutureTopicListener(); + ThirdTopicListener thirdTopicListener() { + return new ThirdTopicListener(); } @Bean - NoRetryMonoTopicListener noRetryMonoTopicListener() { - return new NoRetryMonoTopicListener(); + FourthTopicListener fourthTopicListener() { + return new FourthTopicListener(); } @Bean - FifthFutureTopicListener1 fifthFutureTopicListener1() { - return new FifthFutureTopicListener1(); + FifthTopicListener1 fifthTopicListener1() { + return new FifthTopicListener1(); } @Bean - FifthFutureTopicListener2 fifthFutureTopicListener2() { - return new FifthFutureTopicListener2(); + FifthTopicListener2 fifthTopicListener2() { + return new FifthTopicListener2(); } @Bean - MyCustomDltProcessor myCustomDltProcessor() { - return new MyCustomDltProcessor(); + SixthTopicDefaultDLTListener manualTopicListener() { + return new SixthTopicDefaultDLTListener(); } @Bean - TaskScheduler taskScheduler() { - return new ThreadPoolTaskScheduler(); + NoRetryTopicListener noRetryTopicListener() { + return new NoRetryTopicListener(); } @Bean - KafkaListenerErrorHandler myCustomErrorHandler( - CountDownLatchContainer container) { - return (message, exception) -> { - container.customErrorHandlerCountdownLatch.countDown(); - throw exception; - }; + FirstReuseRetryTopicListener firstReuseRetryTopicListener() { + return new FirstReuseRetryTopicListener(); } @Bean - SmartMessageConverter myCustomMessageConverter( - CountDownLatchContainer container) { - return new CompositeMessageConverter(Collections.singletonList(new GenericMessageConverter())) { - - @Override - public Object fromMessage(Message message, Class targetClass, Object conversionHint) { - container.customMessageConverterCountdownLatch.countDown(); - return super.fromMessage(message, targetClass, conversionHint); - } - }; + SecondReuseRetryTopicListener secondReuseRetryTopicListener() { + return new SecondReuseRetryTopicListener(); } - @Bean - RetryTopicConfiguration firstRetryTopic(KafkaTemplate template) { - return RetryTopicConfigurationBuilder - .newInstance() - .fixedBackOff(50) - .maxAttempts(5) - .concurrency(1) - .useSingleTopicForSameIntervals() - .includeTopic(FIRST_FUTURE_TOPIC) - .doNotRetryOnDltFailure() - .dltHandlerMethod("myCustomDltProcessor", DLT_METHOD_NAME) - .create(template); + ThirdReuseRetryTopicListener thirdReuseRetryTopicListener() { + return new ThirdReuseRetryTopicListener(); } @Bean - RetryTopicConfiguration firstRetryMonoTopic(KafkaTemplate template) { - return RetryTopicConfigurationBuilder - .newInstance() - .fixedBackOff(50) - .maxAttempts(5) - .concurrency(1) - .useSingleTopicForSameIntervals() - .includeTopic(FIRST_MONO_TOPIC) - .doNotRetryOnDltFailure() - .dltHandlerMethod("myCustomDltProcessor", DLT_METHOD_NAME) - .create(template); + CountDownLatchContainer latchContainer() { + return new CountDownLatchContainer(); } @Bean - RetryTopicConfiguration secondRetryFutureTopic(KafkaTemplate template) { - return RetryTopicConfigurationBuilder - .newInstance() - .exponentialBackoff(500, 2, 10000) - .retryOn(Arrays.asList(IllegalStateException.class, IllegalAccessException.class)) - .traversingCauses() - .includeTopic(SECOND_FUTURE_TOPIC) - .doNotRetryOnDltFailure() - .dltHandlerMethod("myCustomDltProcessor", DLT_METHOD_NAME) - .create(template); + MyCustomDltProcessor myCustomDltProcessor() { + return new MyCustomDltProcessor(); } + } - @Bean - RetryTopicConfiguration secondRetryMonoTopic(KafkaTemplate template) { - return RetryTopicConfigurationBuilder - .newInstance() - .exponentialBackoff(500, 2, 10000) - .retryOn(Arrays.asList(IllegalStateException.class, IllegalAccessException.class)) - .traversingCauses() - .includeTopic(SECOND_MONO_TOPIC) - .doNotRetryOnDltFailure() - .dltHandlerMethod("myCustomDltProcessor", DLT_METHOD_NAME) - .create(template); - } + @Configuration + static class KafkaProducerConfig { + @Autowired + EmbeddedKafkaBroker broker; @Bean - NewTopics topics() { - - NewTopic thirdFutureTopic = TopicBuilder.name(THIRD_FUTURE_TOPIC).partitions(2).replicas(1).build(); - NewTopic thirdMonoTopic = TopicBuilder.name(THIRD_MONO_TOPIC).partitions(2).replicas(1).build(); - NewTopic fourthFutureTopic = TopicBuilder.name(FOURTH_FUTURE_TOPIC).partitions(2).replicas(1).build(); - NewTopic fourthMonoTopic = TopicBuilder.name(FOURTH_MONO_TOPIC).partitions(2).replicas(1).build(); + ProducerFactory producerFactory() { + Map configProps = new HashMap<>(); + configProps.put( + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, + this.broker.getBrokersAsString()); + configProps.put( + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + StringSerializer.class); + configProps.put( + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + StringSerializer.class); + return new DefaultKafkaProducerFactory<>(configProps); + } - return new NewTopics( - thirdFutureTopic, - thirdMonoTopic, - fourthFutureTopic, - fourthMonoTopic); + @Bean("customKafkaTemplate") + KafkaTemplate kafkaTemplate() { + return new KafkaTemplate<>(producerFactory()); } } @@ -1008,15 +882,15 @@ KafkaAdmin kafkaAdmin() { return new KafkaAdmin(configs); } -// @Bean -// NewTopic topic() { -// return TopicBuilder.name(THIRD_TOPIC).partitions(2).replicas(1).build(); -// } -// -// @Bean -// NewTopics topics() { -// return new NewTopics(TopicBuilder.name(FOURTH_TOPIC).partitions(2).replicas(1).build()); -// } + @Bean + NewTopic topic() { + return TopicBuilder.name(THIRD_TOPIC).partitions(2).replicas(1).build(); + } + + @Bean + NewTopics topics() { + return new NewTopics(TopicBuilder.name(FOURTH_TOPIC).partitions(2).replicas(1).build()); + } @Bean ConsumerFactory consumerFactory() { @@ -1072,25 +946,11 @@ ConcurrentKafkaListenerContainerFactory kafkaListenerContainerFa return factory; } -// @Bean -// TaskScheduler sched() { -// return new ThreadPoolTaskScheduler(); -// } + @Bean + TaskScheduler sched() { + return new ThreadPoolTaskScheduler(); + } } - @Component - static class MyCustomDltProcessor { - - @Autowired - KafkaTemplate kafkaTemplate; - - @Autowired - CountDownLatchContainer container; - - public void processDltMessage(Object message) { - container.customDltCountdownLatch.countDown(); - throw new RuntimeException("Dlt Error!"); - } - } } diff --git a/spring-kafka/src/test/java/org/springframework/kafka/retrytopic/AsyncMonoFutureRetryTopicClassLevelIntegrationTests.java b/spring-kafka/src/test/java/org/springframework/kafka/retrytopic/AsyncMonoFutureRetryTopicClassLevelIntegrationTests.java new file mode 100644 index 000000000..6f27b1fea --- /dev/null +++ b/spring-kafka/src/test/java/org/springframework/kafka/retrytopic/AsyncMonoFutureRetryTopicClassLevelIntegrationTests.java @@ -0,0 +1,958 @@ +/* + * Copyright 2018-2021 the original author or authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.springframework.kafka.retrytopic; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; +import static org.awaitility.Awaitility.await; + +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.assertj.core.api.InstanceOfAssertFactories; +import org.junit.jupiter.api.Test; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; +import org.springframework.kafka.annotation.DltHandler; +import org.springframework.kafka.annotation.EnableKafka; +import org.springframework.kafka.annotation.KafkaHandler; +import org.springframework.kafka.annotation.KafkaListener; +import org.springframework.kafka.annotation.PartitionOffset; +import org.springframework.kafka.annotation.RetryableTopic; +import org.springframework.kafka.config.ConcurrentKafkaListenerContainerFactory; +import org.springframework.kafka.config.KafkaListenerEndpointRegistry; +import org.springframework.kafka.config.TopicBuilder; +import org.springframework.kafka.core.ConsumerFactory; +import org.springframework.kafka.core.DefaultKafkaConsumerFactory; +import org.springframework.kafka.core.DefaultKafkaProducerFactory; +import org.springframework.kafka.core.KafkaAdmin; +import org.springframework.kafka.core.KafkaAdmin.NewTopics; +import org.springframework.kafka.core.KafkaTemplate; +import org.springframework.kafka.core.ProducerFactory; +import org.springframework.kafka.listener.ConcurrentMessageListenerContainer; +import org.springframework.kafka.listener.ContainerProperties; +import org.springframework.kafka.listener.ContainerProperties.AckMode; +import org.springframework.kafka.listener.KafkaListenerErrorHandler; +import org.springframework.kafka.support.Acknowledgment; +import org.springframework.kafka.support.KafkaHeaders; +import org.springframework.kafka.test.EmbeddedKafkaBroker; +import org.springframework.kafka.test.context.EmbeddedKafka; +import org.springframework.messaging.Message; +import org.springframework.messaging.converter.CompositeMessageConverter; +import org.springframework.messaging.converter.GenericMessageConverter; +import org.springframework.messaging.converter.SmartMessageConverter; +import org.springframework.messaging.handler.annotation.Header; +import org.springframework.retry.annotation.Backoff; +import org.springframework.scheduling.TaskScheduler; +import org.springframework.scheduling.concurrent.ThreadPoolTaskScheduler; +import org.springframework.test.annotation.DirtiesContext; +import org.springframework.test.context.TestPropertySource; +import org.springframework.test.context.junit.jupiter.SpringJUnitConfig; + +import reactor.core.publisher.Mono; + +/** + * @author Sanghyeok An + * @since 3.3.0 + */ + +@SpringJUnitConfig +@DirtiesContext +@EmbeddedKafka(topics = { AsyncMonoFutureRetryTopicClassLevelIntegrationTests.FIRST_TOPIC, + AsyncMonoFutureRetryTopicClassLevelIntegrationTests.SECOND_TOPIC, + AsyncMonoFutureRetryTopicClassLevelIntegrationTests.THIRD_TOPIC, + AsyncMonoFutureRetryTopicClassLevelIntegrationTests.FOURTH_TOPIC, + AsyncMonoFutureRetryTopicClassLevelIntegrationTests.TWO_LISTENERS_TOPIC, + AsyncMonoFutureRetryTopicClassLevelIntegrationTests.MANUAL_TOPIC }) +@TestPropertySource(properties = { "five.attempts=5", "kafka.template=customKafkaTemplate"}) +public class AsyncMonoFutureRetryTopicClassLevelIntegrationTests { + + public final static String FIRST_TOPIC = "myRetryTopic1"; + + public final static String SECOND_TOPIC = "myRetryTopic2"; + + public final static String THIRD_TOPIC = "myRetryTopic3"; + + public final static String FOURTH_TOPIC = "myRetryTopic4"; + + public final static String TWO_LISTENERS_TOPIC = "myRetryTopic5"; + + public final static String MANUAL_TOPIC = "myRetryTopic6"; + + public final static String NOT_RETRYABLE_EXCEPTION_TOPIC = "noRetryTopic"; + + public final static String FIRST_REUSE_RETRY_TOPIC = "reuseRetry1"; + + public final static String SECOND_REUSE_RETRY_TOPIC = "reuseRetry2"; + + public final static String THIRD_REUSE_RETRY_TOPIC = "reuseRetry3"; + + private final static String MAIN_TOPIC_CONTAINER_FACTORY = "kafkaListenerContainerFactory"; + + @Autowired + private KafkaTemplate kafkaTemplate; + + @Autowired + private CountDownLatchContainer latchContainer; + + @Autowired + DestinationTopicContainer topicContainer; + + @Test + void shouldRetryFirstTopic(@Autowired KafkaListenerEndpointRegistry registry) { + kafkaTemplate.send(FIRST_TOPIC, "Testing topic 1"); + assertThat(topicContainer.getNextDestinationTopicFor("firstTopicId", FIRST_TOPIC).getDestinationName()) + .isEqualTo("myRetryTopic1-retry"); + assertThat(awaitLatch(latchContainer.countDownLatch1)).isTrue(); + assertThat(awaitLatch(latchContainer.customDltCountdownLatch)).isTrue(); + assertThat(awaitLatch(latchContainer.customErrorHandlerCountdownLatch)).isTrue(); + assertThat(awaitLatch(latchContainer.customMessageConverterCountdownLatch)).isTrue(); + registry.getListenerContainerIds().stream() + .filter(id -> id.startsWith("first")) + .forEach(id -> { + ConcurrentMessageListenerContainer container = (ConcurrentMessageListenerContainer) registry + .getListenerContainer(id); + if (id.equals("firstTopicId")) { + assertThat(container.getConcurrency()).isEqualTo(2); + } + else { + assertThat(container.getConcurrency()) + .describedAs("Expected %s to have concurrency", id) + .isEqualTo(1); + } + }); + } + + @Test + void shouldRetrySecondTopic() { + kafkaTemplate.send(SECOND_TOPIC, "Testing topic 2"); + assertThat(awaitLatch(latchContainer.countDownLatch2)).isTrue(); + assertThat(awaitLatch(latchContainer.customDltCountdownLatch)).isTrue(); + } + + @Test + void shouldRetryThirdTopicWithTimeout(@Autowired KafkaAdmin admin, + @Autowired KafkaListenerEndpointRegistry registry) throws Exception { + + kafkaTemplate.send(THIRD_TOPIC, "Testing topic 3"); + assertThat(awaitLatch(latchContainer.countDownLatch3)).isTrue(); + assertThat(awaitLatch(latchContainer.countDownLatchDltOne)).isTrue(); + Map topics = admin.describeTopics(THIRD_TOPIC, THIRD_TOPIC + "-dlt", FOURTH_TOPIC); + assertThat(topics.get(THIRD_TOPIC).partitions()).hasSize(2); + assertThat(topics.get(THIRD_TOPIC + "-dlt").partitions()).hasSize(3); + assertThat(topics.get(FOURTH_TOPIC).partitions()).hasSize(2); + AtomicReference method = new AtomicReference<>(); + org.springframework.util.ReflectionUtils.doWithMethods(KafkaAdmin.class, m -> { + m.setAccessible(true); + method.set(m); + }, m -> m.getName().equals("newTopics")); + @SuppressWarnings("unchecked") + Collection weededTopics = (Collection) method.get().invoke(admin); + AtomicInteger weeded = new AtomicInteger(); + weededTopics.forEach(topic -> { + if (topic.name().equals(THIRD_TOPIC) || topic.name().equals(FOURTH_TOPIC)) { + assertThat(topic).isExactlyInstanceOf(NewTopic.class); + weeded.incrementAndGet(); + } + }); + assertThat(weeded.get()).isEqualTo(2); + registry.getListenerContainerIds().stream() + .filter(id -> id.startsWith("third")) + .forEach(id -> { + ConcurrentMessageListenerContainer container = + (ConcurrentMessageListenerContainer) registry.getListenerContainer(id); + if (id.equals("thirdTopicId")) { + assertThat(container.getConcurrency()).isEqualTo(2); + } + else { + assertThat(container.getConcurrency()) + .describedAs("Expected %s to have concurrency", id) + .isEqualTo(1); + } + }); + } + + @Test + void shouldRetryFourthTopicWithNoDlt() { + kafkaTemplate.send(FOURTH_TOPIC, "Testing topic 4"); + assertThat(awaitLatch(latchContainer.countDownLatch4)).isTrue(); + } + + @Test + void shouldRetryFifthTopicWithTwoListenersAndManualAssignment(@Autowired + FifthTopicListener1 listener1, + @Autowired + FifthTopicListener2 listener2) { + + kafkaTemplate.send(TWO_LISTENERS_TOPIC, 0, "0", "Testing topic 5 - 0"); + kafkaTemplate.send(TWO_LISTENERS_TOPIC, 1, "0", "Testing topic 5 - 1"); + assertThat(awaitLatch(latchContainer.countDownLatch51)).isTrue(); + assertThat(awaitLatch(latchContainer.countDownLatch52)).isTrue(); + assertThat(awaitLatch(latchContainer.countDownLatchDltThree)).isTrue(); + assertThat(listener1.topics).containsExactly(TWO_LISTENERS_TOPIC, TWO_LISTENERS_TOPIC + + "-listener1-0", TWO_LISTENERS_TOPIC + "-listener1-1", TWO_LISTENERS_TOPIC + "-listener1-2", + TWO_LISTENERS_TOPIC + "-listener1-dlt"); + assertThat(listener2.topics).containsExactly(TWO_LISTENERS_TOPIC, TWO_LISTENERS_TOPIC + + "-listener2-0", TWO_LISTENERS_TOPIC + "-listener2-1", TWO_LISTENERS_TOPIC + "-listener2-2", + TWO_LISTENERS_TOPIC + "-listener2-dlt"); + } + + @Test + void shouldRetryManualTopicWithDefaultDlt(@Autowired KafkaListenerEndpointRegistry registry, + @Autowired ConsumerFactory cf) { + + kafkaTemplate.send(MANUAL_TOPIC, "Testing topic 6"); + assertThat(awaitLatch(latchContainer.countDownLatch6)).isTrue(); + registry.getListenerContainerIds().stream() + .filter(id -> id.startsWith("manual")) + .forEach(id -> { + ConcurrentMessageListenerContainer container = + (ConcurrentMessageListenerContainer) registry.getListenerContainer(id); + assertThat(container).extracting("commonErrorHandler") + .extracting("seekAfterError", InstanceOfAssertFactories.BOOLEAN) + .isFalse(); + }); + Consumer consumer = cf.createConsumer("manual-dlt", ""); + Set tp = + Set.of(new org.apache.kafka.common.TopicPartition(MANUAL_TOPIC + "-dlt", 0)); + consumer.assign(tp); + try { + await().untilAsserted(() -> { + OffsetAndMetadata offsetAndMetadata = consumer.committed(tp).get(tp.iterator().next()); + assertThat(offsetAndMetadata).isNotNull(); + assertThat(offsetAndMetadata.offset()).isEqualTo(1L); + }); + } + finally { + consumer.close(); + } + } + + @Test + void shouldFirstReuseRetryTopic(@Autowired + FirstReuseRetryTopicListener listener1, + @Autowired + SecondReuseRetryTopicListener listener2, @Autowired + ThirdReuseRetryTopicListener listener3) { + + kafkaTemplate.send(FIRST_REUSE_RETRY_TOPIC, "Testing reuse topic 1"); + kafkaTemplate.send(SECOND_REUSE_RETRY_TOPIC, "Testing reuse topic 2"); + kafkaTemplate.send(THIRD_REUSE_RETRY_TOPIC, "Testing reuse topic 3"); + assertThat(awaitLatch(latchContainer.countDownLatchReuseOne)).isTrue(); + assertThat(awaitLatch(latchContainer.countDownLatchReuseTwo)).isTrue(); + assertThat(awaitLatch(latchContainer.countDownLatchReuseThree)).isTrue(); + assertThat(listener1.topics).containsExactly(FIRST_REUSE_RETRY_TOPIC, + FIRST_REUSE_RETRY_TOPIC + "-retry"); + assertThat(listener2.topics).containsExactly(SECOND_REUSE_RETRY_TOPIC, + SECOND_REUSE_RETRY_TOPIC + "-retry-30", SECOND_REUSE_RETRY_TOPIC + "-retry-60", + SECOND_REUSE_RETRY_TOPIC + "-retry-100", SECOND_REUSE_RETRY_TOPIC + "-retry-100"); + assertThat(listener3.topics).containsExactly(THIRD_REUSE_RETRY_TOPIC, + THIRD_REUSE_RETRY_TOPIC + "-retry", THIRD_REUSE_RETRY_TOPIC + "-retry", + THIRD_REUSE_RETRY_TOPIC + "-retry", THIRD_REUSE_RETRY_TOPIC + "-retry"); + } + + @Test + void shouldGoStraightToDlt() { + kafkaTemplate.send(NOT_RETRYABLE_EXCEPTION_TOPIC, "Testing topic with annotation 1"); + assertThat(awaitLatch(latchContainer.countDownLatchNoRetry)).isTrue(); + assertThat(awaitLatch(latchContainer.countDownLatchDltTwo)).isTrue(); + } + + private boolean awaitLatch(CountDownLatch latch) { + try { + return latch.await(60, TimeUnit.SECONDS); + } + catch (Exception e) { + fail(e.getMessage()); + throw new RuntimeException(e); + } + } + + @KafkaListener( + id = "firstTopicId", + topics = FIRST_TOPIC, + containerFactory = MAIN_TOPIC_CONTAINER_FACTORY, + errorHandler = "myCustomErrorHandler", + contentTypeConverter = "myCustomMessageConverter", + concurrency = "2") + static class FirstTopicListener { + + @Autowired + DestinationTopicContainer topicContainer; + + @Autowired + CountDownLatchContainer container; + + @KafkaHandler + public Mono listen(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + return Mono.fromCallable(() -> { + container.countDownLatch1.countDown(); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new RuntimeException("Woooops... in topic " + receivedTopic); + }).then(); + } + + } + + @KafkaListener(topics = SECOND_TOPIC, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) + static class SecondTopicListener { + + @Autowired + CountDownLatchContainer container; + + @KafkaHandler + public Mono listenAgain(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + return Mono.fromCallable(() -> { + container.countDownIfNotKnown(receivedTopic, container.countDownLatch2); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new IllegalStateException("Another woooops... " + receivedTopic); + }).then(); + } + } + + @RetryableTopic( + attempts = "${five.attempts}", + backoff = @Backoff(delay = 250, maxDelay = 1000, multiplier = 1.5), + numPartitions = "#{3}", + timeout = "${missing.property:100000}", + include = MyRetryException.class, kafkaTemplate = "${kafka.template}", + topicSuffixingStrategy = TopicSuffixingStrategy.SUFFIX_WITH_INDEX_VALUE, + concurrency = "1") + @KafkaListener( + id = "thirdTopicId", + topics = THIRD_TOPIC, + containerFactory = MAIN_TOPIC_CONTAINER_FACTORY, + concurrency = "2") + static class ThirdTopicListener { + + @Autowired + CountDownLatchContainer container; + + @KafkaHandler + public Mono listenWithAnnotation(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + return Mono.fromCallable(() -> { + container.countDownIfNotKnown(receivedTopic, container.countDownLatch3); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new MyRetryException("Annotated woooops... " + receivedTopic); + }).then(); + } + + @DltHandler + public void annotatedDltMethod(Object message) { + container.countDownLatchDltOne.countDown(); + } + } + + @RetryableTopic(dltStrategy = DltStrategy.NO_DLT, attempts = "4", backoff = @Backoff(300), + sameIntervalTopicReuseStrategy = SameIntervalTopicReuseStrategy.MULTIPLE_TOPICS, + kafkaTemplate = "${kafka.template}") + @KafkaListener(topics = FOURTH_TOPIC, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) + static class FourthTopicListener { + + @Autowired + CountDownLatchContainer container; + + @KafkaHandler + public Mono listenNoDlt(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + return Mono.fromCallable(() -> { + container.countDownIfNotKnown(receivedTopic, container.countDownLatch4); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new IllegalStateException("Another woooops... " + receivedTopic); + }).then(); + } + + @DltHandler + public void shouldNotGetHere() { + fail("Dlt should not be processed!"); + } + } + + static class AbstractFifthTopicListener { + + final List topics = Collections.synchronizedList(new ArrayList<>()); + + @Autowired + CountDownLatchContainer container; + + @DltHandler + public void annotatedDltMethod(ConsumerRecord record) { + this.topics.add(record.topic()); + container.countDownLatchDltThree.countDown(); + } + + } + + @RetryableTopic( + attempts = "4", + backoff = @Backoff(250), + numPartitions = "2", + retryTopicSuffix = "-listener1", + dltTopicSuffix = "-listener1-dlt", + topicSuffixingStrategy = TopicSuffixingStrategy.SUFFIX_WITH_INDEX_VALUE, + sameIntervalTopicReuseStrategy = SameIntervalTopicReuseStrategy.MULTIPLE_TOPICS, + kafkaTemplate = "${kafka.template}") + @KafkaListener( + id = "fifthTopicId1", + topicPartitions = {@org.springframework.kafka.annotation.TopicPartition(topic = TWO_LISTENERS_TOPIC, + partitionOffsets = @PartitionOffset(partition = "0", initialOffset = "0"))}, + containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) + static class FifthTopicListener1 extends + AbstractFifthTopicListener { + + @KafkaHandler + public Mono listenWithAnnotation(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + this.topics.add(receivedTopic); + return Mono.fromCallable(() -> { + container.countDownIfNotKnown(receivedTopic, container.countDownLatch51); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new RuntimeException("Annotated woooops... " + receivedTopic); + }).then(); + } + + } + + @RetryableTopic( + attempts = "4", + backoff = @Backoff(250), + numPartitions = "2", + retryTopicSuffix = "-listener2", + dltTopicSuffix = "-listener2-dlt", + topicSuffixingStrategy = TopicSuffixingStrategy.SUFFIX_WITH_INDEX_VALUE, + sameIntervalTopicReuseStrategy = SameIntervalTopicReuseStrategy.MULTIPLE_TOPICS, + kafkaTemplate = "${kafka.template}") + @KafkaListener( + id = "fifthTopicId2", + topicPartitions = {@org.springframework.kafka.annotation.TopicPartition(topic = TWO_LISTENERS_TOPIC, + partitionOffsets = @PartitionOffset(partition = "1", initialOffset = "0"))}, + containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) + static class FifthTopicListener2 extends + AbstractFifthTopicListener { + + @KafkaHandler + public Mono listenWithAnnotation2(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + this.topics.add(receivedTopic); + return Mono.fromCallable(() -> { + container.countDownLatch52.countDown(); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new RuntimeException("Annotated woooops... " + receivedTopic); + }).then(); + } + + } + + @RetryableTopic( + attempts = "4", + backoff = @Backoff(50), + sameIntervalTopicReuseStrategy = SameIntervalTopicReuseStrategy.MULTIPLE_TOPICS) + @KafkaListener( + id = "manual", + topics = MANUAL_TOPIC, + containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) + static class SixthTopicDefaultDLTListener { + + @Autowired + CountDownLatchContainer container; + + @KafkaHandler + public Mono listenNoDlt(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic, + @SuppressWarnings("unused") Acknowledgment ack) { + return Mono.fromCallable(() -> { + container.countDownIfNotKnown(receivedTopic, container.countDownLatch6); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new IllegalStateException("Another woooops... " + receivedTopic); + }).then(); + } + + } + + @RetryableTopic( + attempts = "3", + numPartitions = "3", + exclude = MyDontRetryException.class, + backoff = @Backoff(delay = 50, maxDelay = 100, multiplier = 3), + traversingCauses = "true", + kafkaTemplate = "${kafka.template}") + @KafkaListener(topics = NOT_RETRYABLE_EXCEPTION_TOPIC, containerFactory = MAIN_TOPIC_CONTAINER_FACTORY) + static class NoRetryTopicListener { + + @Autowired + CountDownLatchContainer container; + + @KafkaHandler + public Mono listenWithAnnotation2(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + return Mono.fromCallable(() -> { + container.countDownIfNotKnown(receivedTopic, container.countDownLatchNoRetry); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new MyDontRetryException("Annotated second woooops... " + receivedTopic); + }).then(); + } + + @DltHandler + public void annotatedDltMethod(Object message) { + container.countDownLatchDltTwo.countDown(); + } + } + + @RetryableTopic( + attempts = "2", + backoff = @Backoff(50)) + @KafkaListener( + id = "reuseRetry1", + topics = FIRST_REUSE_RETRY_TOPIC, + containerFactory = "retryTopicListenerContainerFactory") + static class FirstReuseRetryTopicListener { + + final List topics = Collections.synchronizedList(new ArrayList<>()); + + @Autowired + CountDownLatchContainer container; + + @KafkaHandler + public Mono listen1(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + this.topics.add(receivedTopic); + return Mono.fromCallable(() -> { + container.countDownLatchReuseOne.countDown(); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new RuntimeException("Another woooops... " + receivedTopic); + }).then(); + } + + } + + @RetryableTopic( + attempts = "5", + backoff = @Backoff(delay = 30, maxDelay = 100, multiplier = 2)) + @KafkaListener( + id = "reuseRetry2", + topics = SECOND_REUSE_RETRY_TOPIC, + containerFactory = "retryTopicListenerContainerFactory") + static class SecondReuseRetryTopicListener { + + final List topics = Collections.synchronizedList(new ArrayList<>()); + + @Autowired + CountDownLatchContainer container; + + @KafkaHandler + public Mono listen2(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + this.topics.add(receivedTopic); + return Mono.fromCallable(() -> { + container.countDownLatchReuseTwo.countDown(); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new RuntimeException("Another woooops... " + receivedTopic); + }).then(); + } + + } + + @RetryableTopic(attempts = "5", backoff = @Backoff(delay = 1, maxDelay = 5, multiplier = 1.4)) + @KafkaListener(id = "reuseRetry3", topics = THIRD_REUSE_RETRY_TOPIC, + containerFactory = "retryTopicListenerContainerFactory") + static class ThirdReuseRetryTopicListener { + + final List topics = Collections.synchronizedList(new ArrayList<>()); + + @Autowired + CountDownLatchContainer container; + + @KafkaHandler + public Mono listen3(String message, @Header(KafkaHeaders.RECEIVED_TOPIC) String receivedTopic) { + this.topics.add(receivedTopic); + return Mono.fromCallable(() -> { + container.countDownLatchReuseThree.countDown(); + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new RuntimeException("Another woooops... " + receivedTopic); + }).then(); + } + + } + + static class CountDownLatchContainer { + + CountDownLatch countDownLatch1 = new CountDownLatch(5); + + CountDownLatch countDownLatch2 = new CountDownLatch(3); + + CountDownLatch countDownLatch3 = new CountDownLatch(3); + + CountDownLatch countDownLatch4 = new CountDownLatch(4); + + CountDownLatch countDownLatch51 = new CountDownLatch(4); + + CountDownLatch countDownLatch52 = new CountDownLatch(4); + + CountDownLatch countDownLatch6 = new CountDownLatch(4); + + CountDownLatch countDownLatchNoRetry = new CountDownLatch(1); + + CountDownLatch countDownLatchDltOne = new CountDownLatch(1); + + CountDownLatch countDownLatchDltTwo = new CountDownLatch(1); + + CountDownLatch countDownLatchDltThree = new CountDownLatch(2); + + CountDownLatch countDownLatchReuseOne = new CountDownLatch(2); + + CountDownLatch countDownLatchReuseTwo = new CountDownLatch(5); + + CountDownLatch countDownLatchReuseThree = new CountDownLatch(5); + + CountDownLatch customDltCountdownLatch = new CountDownLatch(1); + + CountDownLatch customErrorHandlerCountdownLatch = new CountDownLatch(6); + + CountDownLatch customMessageConverterCountdownLatch = new CountDownLatch(6); + + final List knownTopics = new ArrayList<>(); + + private void countDownIfNotKnown(String receivedTopic, CountDownLatch countDownLatch) { + synchronized (knownTopics) { + if (!knownTopics.contains(receivedTopic)) { + knownTopics.add(receivedTopic); + countDownLatch.countDown(); + } + } + } + } + + static class MyCustomDltProcessor { + + @Autowired + KafkaTemplate kafkaTemplate; + + @Autowired + CountDownLatchContainer container; + + public void processDltMessage(Object message) { + container.customDltCountdownLatch.countDown(); + throw new RuntimeException("Dlt Error!"); + } + } + + @SuppressWarnings("serial") + static class MyRetryException extends RuntimeException { + MyRetryException(String msg) { + super(msg); + } + } + + @SuppressWarnings("serial") + static class MyDontRetryException extends RuntimeException { + MyDontRetryException(String msg) { + super(msg); + } + } + + @Configuration + static class RetryTopicConfigurations extends RetryTopicConfigurationSupport { + + private static final String DLT_METHOD_NAME = "processDltMessage"; + + @Bean + RetryTopicConfiguration firstRetryTopic(KafkaTemplate template) { + return RetryTopicConfigurationBuilder + .newInstance() + .fixedBackOff(50) + .maxAttempts(5) + .concurrency(1) + .useSingleTopicForSameIntervals() + .includeTopic(FIRST_TOPIC) + .doNotRetryOnDltFailure() + .dltHandlerMethod("myCustomDltProcessor", DLT_METHOD_NAME) + .create(template); + } + + @Bean + RetryTopicConfiguration secondRetryTopic(KafkaTemplate template) { + return RetryTopicConfigurationBuilder + .newInstance() + .exponentialBackoff(500, 2, 10000) + .retryOn(Arrays.asList(IllegalStateException.class, IllegalAccessException.class)) + .traversingCauses() + .includeTopic(SECOND_TOPIC) + .doNotRetryOnDltFailure() + .dltHandlerMethod("myCustomDltProcessor", DLT_METHOD_NAME) + .create(template); + } + + @Bean + FirstTopicListener firstTopicListener() { + return new FirstTopicListener(); + } + + @Bean + KafkaListenerErrorHandler myCustomErrorHandler( + CountDownLatchContainer container) { + return (message, exception) -> { + container.customErrorHandlerCountdownLatch.countDown(); + throw exception; + }; + } + + @Bean + SmartMessageConverter myCustomMessageConverter( + CountDownLatchContainer container) { + return new CompositeMessageConverter(Collections.singletonList(new GenericMessageConverter())) { + + @Override + public Object fromMessage(Message message, Class targetClass, Object conversionHint) { + container.customMessageConverterCountdownLatch.countDown(); + return super.fromMessage(message, targetClass, conversionHint); + } + }; + } + + @Bean + SecondTopicListener secondTopicListener() { + return new SecondTopicListener(); + } + + @Bean + ThirdTopicListener thirdTopicListener() { + return new ThirdTopicListener(); + } + + @Bean + FourthTopicListener fourthTopicListener() { + return new FourthTopicListener(); + } + + @Bean + FifthTopicListener1 fifthTopicListener1() { + return new FifthTopicListener1(); + } + + @Bean + FifthTopicListener2 fifthTopicListener2() { + return new FifthTopicListener2(); + } + + @Bean + SixthTopicDefaultDLTListener manualTopicListener() { + return new SixthTopicDefaultDLTListener(); + } + + @Bean + NoRetryTopicListener noRetryTopicListener() { + return new NoRetryTopicListener(); + } + + @Bean + FirstReuseRetryTopicListener firstReuseRetryTopicListener() { + return new FirstReuseRetryTopicListener(); + } + + @Bean + SecondReuseRetryTopicListener secondReuseRetryTopicListener() { + return new SecondReuseRetryTopicListener(); + } + + @Bean + ThirdReuseRetryTopicListener thirdReuseRetryTopicListener() { + return new ThirdReuseRetryTopicListener(); + } + + @Bean + CountDownLatchContainer latchContainer() { + return new CountDownLatchContainer(); + } + + @Bean + MyCustomDltProcessor myCustomDltProcessor() { + return new MyCustomDltProcessor(); + } + } + + @Configuration + static class KafkaProducerConfig { + + @Autowired + EmbeddedKafkaBroker broker; + + @Bean + ProducerFactory producerFactory() { + Map configProps = new HashMap<>(); + configProps.put( + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, + this.broker.getBrokersAsString()); + configProps.put( + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + StringSerializer.class); + configProps.put( + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + StringSerializer.class); + return new DefaultKafkaProducerFactory<>(configProps); + } + + @Bean("customKafkaTemplate") + KafkaTemplate kafkaTemplate() { + return new KafkaTemplate<>(producerFactory()); + } + } + + @EnableKafka + @Configuration + static class KafkaConsumerConfig { + + @Autowired + EmbeddedKafkaBroker broker; + + @Bean + KafkaAdmin kafkaAdmin() { + Map configs = new HashMap<>(); + configs.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, this.broker.getBrokersAsString()); + return new KafkaAdmin(configs); + } + + @Bean + NewTopic topic() { + return TopicBuilder.name(THIRD_TOPIC).partitions(2).replicas(1).build(); + } + + @Bean + NewTopics topics() { + return new NewTopics(TopicBuilder.name(FOURTH_TOPIC).partitions(2).replicas(1).build()); + } + + @Bean + ConsumerFactory consumerFactory() { + Map props = new HashMap<>(); + props.put( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, + this.broker.getBrokersAsString()); + props.put( + ConsumerConfig.GROUP_ID_CONFIG, + "groupId"); + props.put( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + StringDeserializer.class); + props.put( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + StringDeserializer.class); + props.put( + ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, false); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + + return new DefaultKafkaConsumerFactory<>(props); + } + + @Bean + ConcurrentKafkaListenerContainerFactory retryTopicListenerContainerFactory( + ConsumerFactory consumerFactory) { + + ConcurrentKafkaListenerContainerFactory factory = new ConcurrentKafkaListenerContainerFactory<>(); + ContainerProperties props = factory.getContainerProperties(); + props.setIdleEventInterval(100L); + props.setPollTimeout(50L); + props.setIdlePartitionEventInterval(100L); + factory.setConsumerFactory(consumerFactory); + factory.setConcurrency(1); + factory.setContainerCustomizer( + container -> container.getContainerProperties().setIdlePartitionEventInterval(100L)); + return factory; + } + + @Bean + ConcurrentKafkaListenerContainerFactory kafkaListenerContainerFactory( + ConsumerFactory consumerFactory) { + + ConcurrentKafkaListenerContainerFactory factory = new ConcurrentKafkaListenerContainerFactory<>(); + factory.setConsumerFactory(consumerFactory); + factory.setConcurrency(1); + factory.setContainerCustomizer(container -> { + if (container.getListenerId().startsWith("manual")) { + container.getContainerProperties().setAckMode(AckMode.MANUAL); + container.getContainerProperties().setAsyncAcks(true); + } + }); + return factory; + } + + @Bean + TaskScheduler sched() { + return new ThreadPoolTaskScheduler(); + } + + } + +}