-
Notifications
You must be signed in to change notification settings - Fork 1.7k
GH-3276: Support async retry with @RetryableTopic #3523
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 11 commits
eee3515
fdc59f0
87ac522
723cc18
ac110b4
e30319d
6399b81
4a4a1ee
08e7721
c0fea39
e74072c
645d7ce
0c89c92
3f6b447
01de80e
97f05be
d219037
e8378da
967a7c4
f2c67c7
56e6fd3
5e70c76
1d9f7b9
e418b10
002aaf4
3b79d59
82ecac7
0afae4e
796397f
4e89d53
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -38,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; | ||
|
|
@@ -73,6 +74,7 @@ | |
| import org.apache.kafka.common.header.Header; | ||
| import org.apache.kafka.common.header.internals.RecordHeader; | ||
|
|
||
| import org.springframework.aop.support.AopUtils; | ||
| import org.springframework.beans.BeanUtils; | ||
| import org.springframework.beans.factory.ObjectProvider; | ||
| import org.springframework.context.ApplicationContext; | ||
|
|
@@ -106,6 +108,8 @@ | |
| import org.springframework.kafka.listener.ContainerProperties.AssignmentCommitOption; | ||
| import org.springframework.kafka.listener.ContainerProperties.EOSMode; | ||
| import org.springframework.kafka.listener.adapter.AsyncRepliesAware; | ||
| import org.springframework.kafka.listener.adapter.KafkaBackoffAwareMessageListenerAdapter; | ||
| import org.springframework.kafka.listener.adapter.RecordMessagingMessageListenerAdapter; | ||
| import org.springframework.kafka.support.Acknowledgment; | ||
| import org.springframework.kafka.support.KafkaHeaders; | ||
| import org.springframework.kafka.support.KafkaUtils; | ||
|
|
@@ -840,6 +844,8 @@ private final class ListenerConsumer implements SchedulingAwareRunnable, Consume | |
|
|
||
| private volatile long lastPoll = System.currentTimeMillis(); | ||
|
|
||
| private final ConcurrentLinkedDeque<FailedRecordTuple<K, V>> failedRecords = new ConcurrentLinkedDeque<>(); | ||
|
|
||
| @SuppressWarnings(UNCHECKED) | ||
| ListenerConsumer(GenericMessageListener<?> listener, ListenerType listenerType, | ||
| ObservationRegistry observationRegistry) { | ||
|
|
@@ -895,6 +901,22 @@ else if (listener instanceof MessageListener) { | |
| this.wantsFullRecords = false; | ||
| this.pollThreadStateProcessor = setUpPollProcessor(false); | ||
| this.observationEnabled = this.containerProperties.isObservationEnabled(); | ||
|
|
||
| if (!AopUtils.isAopProxy(this.genericListener) && | ||
| this.genericListener instanceof KafkaBackoffAwareMessageListenerAdapter<?, ?>) { | ||
| KafkaBackoffAwareMessageListenerAdapter<K, V> genListener = | ||
| (KafkaBackoffAwareMessageListenerAdapter<K, V>) this.genericListener; | ||
| if (genListener.getDelegate() instanceof RecordMessagingMessageListenerAdapter<K, V>) { | ||
|
||
|
|
||
| RecordMessagingMessageListenerAdapter<K, V> recordAdapterListener = | ||
| (RecordMessagingMessageListenerAdapter<K, V>) genListener.getDelegate(); | ||
|
|
||
| BiConsumer<ConsumerRecord<K, V>, RuntimeException> callbackForAsyncFailure = | ||
| (cRecord, ex) -> this.failedRecords.addLast(new FailedRecordTuple<>(cRecord, ex)); | ||
| recordAdapterListener.setCallbackForAsyncFailure(callbackForAsyncFailure); | ||
| } | ||
|
|
||
| } | ||
| } | ||
| else { | ||
| throw new IllegalArgumentException("Listener must be one of 'MessageListener', " | ||
|
|
@@ -1294,6 +1316,15 @@ public void run() { | |
| boolean failedAuthRetry = false; | ||
| this.lastReceive = System.currentTimeMillis(); | ||
| while (isRunning()) { | ||
|
|
||
| try { | ||
| handleAsyncFailure(); | ||
| } | ||
| catch (Exception e) { | ||
| ListenerConsumer.this.logger.error( | ||
| "Failed to process async retry messages. skip this time, try it again next loop."); | ||
| } | ||
|
|
||
| try { | ||
| pollAndInvoke(); | ||
| if (failedAuthRetry) { | ||
|
|
@@ -1435,6 +1466,29 @@ protected void pollAndInvoke() { | |
| } | ||
| } | ||
|
|
||
| protected void handleAsyncFailure() { | ||
| List<FailedRecordTuple<K, V>> copyFailedRecords = new ArrayList<>(); | ||
| while (!this.failedRecords.isEmpty()) { | ||
| FailedRecordTuple<K, V> failedRecordTuple = this.failedRecords.pollFirst(); | ||
| copyFailedRecords.add(failedRecordTuple); | ||
| } | ||
|
|
||
| // If any copied and failed record fails to complete due to an unexpected error, | ||
| // We will give up on retrying with the remaining copied and failed Records. | ||
| for (FailedRecordTuple<K, V> copyFailedRecord : copyFailedRecords) { | ||
| try { | ||
| invokeErrorHandlerBySingleRecord(copyFailedRecord); | ||
| } | ||
| catch (Exception e) { | ||
| this.logger.warn(() -> | ||
| "Async failed record failed to complete, thus skip it. record :" | ||
| + copyFailedRecord.toString() | ||
| + ", Exception : " | ||
| + e.getMessage()); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private void doProcessCommits() { | ||
| if (!this.autoCommit && !this.isRecordAck) { | ||
| try { | ||
|
|
@@ -2827,6 +2881,44 @@ private void doInvokeOnMessage(final ConsumerRecord<K, V> recordArg) { | |
| } | ||
| } | ||
|
|
||
| private void invokeErrorHandlerBySingleRecord(final FailedRecordTuple<K, V> failedRecordTuple) { | ||
| final ConsumerRecord<K, V> cRecord = failedRecordTuple.record; | ||
| RuntimeException rte = failedRecordTuple.ex; | ||
| if (this.commonErrorHandler.seeksAfterHandling() || rte instanceof CommitFailedException) { | ||
| try { | ||
| if (this.producer == null) { | ||
| processCommits(); | ||
| } | ||
| } | ||
| catch (Exception ex) { // NO SONAR | ||
| this.logger.error(ex, "Failed to commit before handling error"); | ||
| } | ||
| List<ConsumerRecord<?, ?>> records = new ArrayList<>(); | ||
| records.add(cRecord); | ||
| this.commonErrorHandler.handleRemaining(rte, records, this.consumer, | ||
| KafkaMessageListenerContainer.this.thisOrParentContainer); | ||
| } | ||
| else { | ||
| boolean handled = false; | ||
| try { | ||
| handled = this.commonErrorHandler.handleOne(rte, cRecord, this.consumer, | ||
| KafkaMessageListenerContainer.this.thisOrParentContainer); | ||
| } | ||
| catch (Exception ex) { | ||
| this.logger.error(ex, "ErrorHandler threw unexpected exception"); | ||
| } | ||
| Map<TopicPartition, List<ConsumerRecord<K, V>>> records = new LinkedHashMap<>(); | ||
| if (!handled) { | ||
| records.computeIfAbsent(new TopicPartition(cRecord.topic(), cRecord.partition()), | ||
| tp -> new ArrayList<>()).add(cRecord); | ||
| } | ||
| if (!records.isEmpty()) { | ||
| this.remainingRecords = new ConsumerRecords<>(records); | ||
| this.pauseForPending = true; | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private void invokeErrorHandler(final ConsumerRecord<K, V> cRecord, | ||
| Iterator<ConsumerRecord<K, V>> iterator, RuntimeException rte) { | ||
|
|
||
|
|
@@ -3913,4 +4005,6 @@ private static class StopAfterFenceException extends KafkaException { | |
|
|
||
| } | ||
|
|
||
| private record FailedRecordTuple<K, V>(ConsumerRecord<K, V> record, RuntimeException ex) { }; | ||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -1,5 +1,5 @@ | ||
| /* | ||
| * Copyright 2015-2019 the original author or authors. | ||
| * Copyright 2015-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. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -1,5 +1,5 @@ | ||
| /* | ||
| * Copyright 2018-2023 the original author or authors. | ||
| * Copyright 2018-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. | ||
|
|
@@ -143,4 +143,5 @@ public void onMessage(ConsumerRecord<K, V> data, Acknowledgment acknowledgment) | |
| public void onMessage(ConsumerRecord<K, V> data, Consumer<?, ?> consumer) { | ||
| onMessage(data, null, consumer); | ||
| } | ||
|
|
||
|
||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.