Skip to content

Commit

Permalink
GH-3276: Draft and test codes.
Browse files Browse the repository at this point in the history
  • Loading branch information
chickenchickenlove committed Oct 1, 2024
1 parent fdc59f0 commit 87ac522
Show file tree
Hide file tree
Showing 7 changed files with 1,413 additions and 535 deletions.
Original file line number Diff line number Diff line change
@@ -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<K, V>(
ConsumerRecord<K, V> record,
RuntimeException ex) {

};
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -842,6 +844,9 @@ 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) {
Expand Down Expand Up @@ -899,10 +904,9 @@ else if (listener instanceof MessageListener) {
this.observationEnabled = this.containerProperties.isObservationEnabled();

if (!AopUtils.isAopProxy(listener)) {
final BiConsumer<ConsumerRecord<K, V>, RuntimeException> asyncRetryCallback
= (cRecord, runtimeException) ->
this.invokeErrorHandlerBySingleRecord(cRecord, runtimeException);
this.listener.setAsyncRetryCallback(asyncRetryCallback);
final java.util.function.Consumer<FailedRecordTuple> callbackForAsyncFailureQueue =
(fRecord) -> this.failedRecords.addLast(fRecord);
this.listener.setCallbackForAsyncFailureQueue(callbackForAsyncFailureQueue);
}
}
else {
Expand Down Expand Up @@ -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) {
Expand Down Expand Up @@ -1444,6 +1457,19 @@ protected void pollAndInvoke() {
}
}

protected void handleAsyncFailure() {
List<FailedRecordTuple> 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 {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand All @@ -28,11 +29,12 @@
*
* @author Marius Bogoevici
* @author Gary Russell
* @author Sanghyeok An
*/
@FunctionalInterface
public interface MessageListener<K, V> extends GenericMessageListener<ConsumerRecord<K, V>> {

default void setAsyncRetryCallback(BiConsumer<ConsumerRecord<K, V>, RuntimeException> asyncRetryCallback) {
default void setCallbackForAsyncFailureQueue(Consumer<FailedRecordTuple> asyncRetryCallback) {
//
}
}

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -94,6 +94,7 @@
* @author Wang ZhiYang
* @author Huijin Hong
* @author Soby Chacko
* @author Sanghyeok An
*/
public abstract class MessagingMessageListenerAdapter<K, V> implements ConsumerSeekAware, AsyncRepliesAware {

Expand Down Expand Up @@ -157,13 +158,16 @@ public abstract class MessagingMessageListenerAdapter<K, V> implements ConsumerS

private BiConsumer<ConsumerRecord<K, V>, RuntimeException> asyncRetryCallback;

private java.util.function.Consumer<FailedRecordTuple> callbackForAsyncFailureQueue;

/**
* Create an instance with the provided bean and method.
* @param bean the bean.
* @param method the method.
*/
protected MessagingMessageListenerAdapter(Object bean, Method method) {
this(bean, method, null);
System.out.println("here");
}

/**
Expand Down Expand Up @@ -689,7 +693,8 @@ protected void asyncFailure(Object request, @Nullable Acknowledgment acknowledgm
if (request instanceof ConsumerRecord &&
ex instanceof RuntimeException) {
ConsumerRecord<K, V> record = (ConsumerRecord<K, V>) request;
asyncRetryCallback.accept(record, (RuntimeException) ex);
FailedRecordTuple failedRecordTuple = new FailedRecordTuple(record, (RuntimeException) ex);
this.callbackForAsyncFailureQueue.accept(failedRecordTuple);
}
}
}
Expand Down Expand Up @@ -910,4 +915,8 @@ public void setAsyncRetryCallback(BiConsumer<ConsumerRecord<K, V>, RuntimeExcept
this.asyncRetryCallback = asyncRetryCallback;
}

public void putInAsyncFailureQueue(java.util.function.Consumer<FailedRecordTuple> callbackForAsyncFailureQueue) {
this.callbackForAsyncFailureQueue = callbackForAsyncFailureQueue;
}

}
Loading

0 comments on commit 87ac522

Please sign in to comment.