Skip to content

Commit eb7648a

Browse files
garyrussellartembilan
authored andcommitted
GH-2357: Remove Remaining Uses of ListenableFuture
Resolves #2357 **main only; will issue separate PR to deprecate setters in 2.9**
1 parent f4b73af commit eb7648a

14 files changed

+74
-75
lines changed

build.gradle

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -74,7 +74,7 @@ ext {
7474
springBootVersion = '2.6.7' // docs module
7575
springDataVersion = '2022.0.0-M5'
7676
springRetryVersion = '1.3.3'
77-
springVersion = '6.0.0-M5'
77+
springVersion = '6.0.0-SNAPSHOT'
7878
zookeeperVersion = '3.6.3'
7979

8080
idPrefix = 'kafka'

spring-kafka/src/main/java/org/springframework/kafka/listener/ConcurrentMessageListenerContainer.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,7 @@
3333

3434
import org.springframework.context.ApplicationContext;
3535
import org.springframework.context.ApplicationEventPublisher;
36-
import org.springframework.core.task.AsyncListenableTaskExecutor;
36+
import org.springframework.core.task.AsyncTaskExecutor;
3737
import org.springframework.core.task.SimpleAsyncTaskExecutor;
3838
import org.springframework.kafka.core.ConsumerFactory;
3939
import org.springframework.kafka.support.TopicPartitionOffset;
@@ -62,7 +62,7 @@ public class ConcurrentMessageListenerContainer<K, V> extends AbstractMessageLis
6262

6363
private final List<KafkaMessageListenerContainer<K, V>> containers = new ArrayList<>();
6464

65-
private final List<AsyncListenableTaskExecutor> executors = new ArrayList<>();
65+
private final List<AsyncTaskExecutor> executors = new ArrayList<>();
6666

6767
private int concurrency = 1;
6868

@@ -237,7 +237,7 @@ private void configureChildContainer(int index, KafkaMessageListenerContainer<K,
237237
stopAbnormally(() -> {
238238
});
239239
});
240-
AsyncListenableTaskExecutor exec = container.getContainerProperties().getConsumerTaskExecutor();
240+
AsyncTaskExecutor exec = container.getContainerProperties().getListenerTaskExecutor();
241241
if (exec == null) {
242242
if ((this.executors.size() > index)) {
243243
exec = this.executors.get(index);
@@ -246,7 +246,7 @@ private void configureChildContainer(int index, KafkaMessageListenerContainer<K,
246246
exec = new SimpleAsyncTaskExecutor(beanName + "-C-");
247247
this.executors.add(exec);
248248
}
249-
container.getContainerProperties().setConsumerTaskExecutor(exec);
249+
container.getContainerProperties().setListenerTaskExecutor(exec);
250250
}
251251
}
252252

spring-kafka/src/main/java/org/springframework/kafka/listener/ContainerProperties.java

Lines changed: 10 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@
3030
import org.springframework.aop.framework.Advised;
3131
import org.springframework.aop.framework.ProxyFactory;
3232
import org.springframework.aop.support.AopUtils;
33-
import org.springframework.core.task.AsyncListenableTaskExecutor;
33+
import org.springframework.core.task.AsyncTaskExecutor;
3434
import org.springframework.kafka.support.TopicPartitionOffset;
3535
import org.springframework.lang.Nullable;
3636
import org.springframework.scheduling.TaskScheduler;
@@ -231,7 +231,7 @@ public enum EOSMode {
231231
/**
232232
* The executor for threads that poll the consumer.
233233
*/
234-
private AsyncListenableTaskExecutor consumerTaskExecutor;
234+
private AsyncTaskExecutor listenerTaskExecutor;
235235

236236
/**
237237
* The timeout for shutting down the container. This is the maximum amount of
@@ -380,10 +380,11 @@ public void setAckTime(long ackTime) {
380380

381381
/**
382382
* Set the executor for threads that poll the consumer.
383-
* @param consumerTaskExecutor the executor
383+
* @param listenerTaskExecutor the executor
384+
* @since 2.8.9
384385
*/
385-
public void setConsumerTaskExecutor(@Nullable AsyncListenableTaskExecutor consumerTaskExecutor) {
386-
this.consumerTaskExecutor = consumerTaskExecutor;
386+
public void setListenerTaskExecutor(@Nullable AsyncTaskExecutor listenerTaskExecutor) {
387+
this.listenerTaskExecutor = listenerTaskExecutor;
387388
}
388389

389390
/**
@@ -466,8 +467,8 @@ public Object getMessageListener() {
466467
* @return the executor.
467468
*/
468469
@Nullable
469-
public AsyncListenableTaskExecutor getConsumerTaskExecutor() {
470-
return this.consumerTaskExecutor;
470+
public AsyncTaskExecutor getListenerTaskExecutor() {
471+
return this.listenerTaskExecutor;
471472
}
472473

473474
public long getShutdownTimeout() {
@@ -919,8 +920,8 @@ public String toString() {
919920
+ "\n ackCount=" + this.ackCount
920921
+ "\n ackTime=" + this.ackTime
921922
+ "\n messageListener=" + this.messageListener
922-
+ (this.consumerTaskExecutor != null
923-
? "\n consumerTaskExecutor=" + this.consumerTaskExecutor
923+
+ (this.listenerTaskExecutor != null
924+
? "\n listenerTaskExecutor=" + this.listenerTaskExecutor
924925
: "")
925926
+ "\n shutdownTimeout=" + this.shutdownTimeout
926927
+ "\n idleEventInterval="

spring-kafka/src/main/java/org/springframework/kafka/listener/KafkaMessageListenerContainer.java

Lines changed: 24 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -35,12 +35,14 @@
3535
import java.util.Properties;
3636
import java.util.Set;
3737
import java.util.concurrent.BlockingQueue;
38+
import java.util.concurrent.CompletableFuture;
3839
import java.util.concurrent.ConcurrentHashMap;
3940
import java.util.concurrent.CountDownLatch;
4041
import java.util.concurrent.LinkedBlockingQueue;
4142
import java.util.concurrent.ScheduledFuture;
4243
import java.util.concurrent.TimeUnit;
4344
import java.util.concurrent.atomic.AtomicBoolean;
45+
import java.util.function.BiConsumer;
4446
import java.util.regex.Pattern;
4547
import java.util.stream.Collectors;
4648

@@ -72,7 +74,7 @@
7274
import org.springframework.context.ApplicationContext;
7375
import org.springframework.context.ApplicationEventPublisher;
7476
import org.springframework.core.log.LogAccessor;
75-
import org.springframework.core.task.AsyncListenableTaskExecutor;
77+
import org.springframework.core.task.AsyncTaskExecutor;
7678
import org.springframework.core.task.SimpleAsyncTaskExecutor;
7779
import org.springframework.kafka.KafkaException;
7880
import org.springframework.kafka.core.ConsumerFactory;
@@ -123,8 +125,6 @@
123125
import org.springframework.util.ClassUtils;
124126
import org.springframework.util.CollectionUtils;
125127
import org.springframework.util.StringUtils;
126-
import org.springframework.util.concurrent.ListenableFuture;
127-
import org.springframework.util.concurrent.ListenableFutureCallback;
128128

129129

130130
/**
@@ -175,7 +175,7 @@ public class KafkaMessageListenerContainer<K, V> // NOSONAR line count
175175

176176
private volatile ListenerConsumer listenerConsumer;
177177

178-
private volatile ListenableFuture<?> listenerConsumerFuture;
178+
private volatile CompletableFuture<Void> listenerConsumerFuture;
179179

180180
private volatile CountDownLatch startLatch = new CountDownLatch(1);
181181

@@ -350,21 +350,22 @@ protected void doStart() {
350350
checkAckMode(containerProperties);
351351

352352
Object messageListener = containerProperties.getMessageListener();
353-
AsyncListenableTaskExecutor consumerExecutor = containerProperties.getConsumerTaskExecutor();
353+
AsyncTaskExecutor consumerExecutor = containerProperties.getListenerTaskExecutor();
354354
if (consumerExecutor == null) {
355355
consumerExecutor = new SimpleAsyncTaskExecutor(
356356
(getBeanName() == null ? "" : getBeanName()) + "-C-");
357-
containerProperties.setConsumerTaskExecutor(consumerExecutor);
357+
containerProperties.setListenerTaskExecutor(consumerExecutor);
358358
}
359359
GenericMessageListener<?> listener = (GenericMessageListener<?>) messageListener;
360360
ListenerType listenerType = determineListenerType(listener);
361361
this.listenerConsumer = new ListenerConsumer(listener, listenerType);
362362
setRunning(true);
363363
this.startLatch = new CountDownLatch(1);
364-
this.listenerConsumerFuture = consumerExecutor
365-
.submitListenable(this.listenerConsumer);
364+
this.listenerConsumerFuture = consumerExecutor.submitCompletable(this.listenerConsumer);
366365
try {
367-
if (!this.startLatch.await(containerProperties.getConsumerStartTimeout().toMillis(), TimeUnit.MILLISECONDS)) {
366+
if (!this.startLatch.await(containerProperties.getConsumerStartTimeout().toMillis(),
367+
TimeUnit.MILLISECONDS)) {
368+
368369
this.logger.error("Consumer thread failed to start - does the configured task executor "
369370
+ "have enough threads to support all containers and concurrency?");
370371
publishConsumerFailedToStart();
@@ -403,7 +404,7 @@ private ListenerType determineListenerType(GenericMessageListener<?> listener) {
403404
@Override
404405
protected void doStop(final Runnable callback, boolean normal) {
405406
if (isRunning()) {
406-
this.listenerConsumerFuture.addCallback(new StopCallback(callback));
407+
this.listenerConsumerFuture.whenComplete(new StopCallback(callback));
407408
setRunning(false);
408409
this.listenerConsumer.wakeIfNecessaryForStop();
409410
setStoppedNormally(normal);
@@ -3712,7 +3713,7 @@ private static final class OffsetMetadata {
37123713

37133714
}
37143715

3715-
private class StopCallback implements ListenableFutureCallback<Object> {
3716+
private class StopCallback implements BiConsumer<Object, Throwable> {
37163717

37173718
private final Runnable callback;
37183719

@@ -3721,20 +3722,19 @@ private class StopCallback implements ListenableFutureCallback<Object> {
37213722
}
37223723

37233724
@Override
3724-
public void onFailure(Throwable e) {
3725-
KafkaMessageListenerContainer.this.logger
3726-
.error(e, "Error while stopping the container: ");
3727-
if (this.callback != null) {
3728-
this.callback.run();
3725+
public void accept(Object result, @Nullable Throwable throwable) {
3726+
if (throwable != null) {
3727+
KafkaMessageListenerContainer.this.logger.error(throwable, "Error while stopping the container");
3728+
if (this.callback != null) {
3729+
this.callback.run();
3730+
}
37293731
}
3730-
}
3731-
3732-
@Override
3733-
public void onSuccess(Object result) {
3734-
KafkaMessageListenerContainer.this.logger
3735-
.debug(() -> KafkaMessageListenerContainer.this + " stopped normally");
3736-
if (this.callback != null) {
3737-
this.callback.run();
3732+
else {
3733+
KafkaMessageListenerContainer.this.logger
3734+
.debug(() -> KafkaMessageListenerContainer.this + " stopped normally");
3735+
if (this.callback != null) {
3736+
this.callback.run();
3737+
}
37383738
}
37393739
}
37403740

spring-kafka/src/test/java/org/springframework/kafka/core/DefaultKafkaProducerFactoryTests.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,7 @@
3636
import java.util.List;
3737
import java.util.Map;
3838
import java.util.Queue;
39+
import java.util.concurrent.CompletableFuture;
3940
import java.util.concurrent.atomic.AtomicBoolean;
4041
import java.util.concurrent.atomic.AtomicInteger;
4142

@@ -58,7 +59,6 @@
5859
import org.springframework.kafka.transaction.KafkaTransactionManager;
5960
import org.springframework.transaction.CannotCreateTransactionException;
6061
import org.springframework.transaction.support.TransactionTemplate;
61-
import org.springframework.util.concurrent.SettableListenableFuture;
6262

6363
/**
6464
* @author Gary Russell
@@ -71,7 +71,7 @@ public class DefaultKafkaProducerFactoryTests {
7171
@Test
7272
void testProducerClosedAfterBadTransition() throws Exception {
7373
final Producer producer = mock(Producer.class);
74-
given(producer.send(any(), any())).willReturn(new SettableListenableFuture<>());
74+
given(producer.send(any(), any())).willReturn(new CompletableFuture());
7575
DefaultKafkaProducerFactory pf = new DefaultKafkaProducerFactory(new HashMap<>()) {
7676

7777
@Override

spring-kafka/src/test/java/org/springframework/kafka/core/KafkaTemplateTests.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -96,7 +96,6 @@
9696
import org.springframework.kafka.test.utils.KafkaTestUtils;
9797
import org.springframework.messaging.Message;
9898
import org.springframework.messaging.support.MessageBuilder;
99-
import org.springframework.util.concurrent.SettableListenableFuture;
10099

101100
/**
102101
* @author Gary Russell
@@ -422,7 +421,7 @@ void testWithCallbackFailure() throws Exception {
422421
willAnswer(inv -> {
423422
Callback callback = inv.getArgument(1);
424423
callback.onCompletion(null, new RuntimeException("test"));
425-
return new SettableListenableFuture<RecordMetadata>();
424+
return new CompletableFuture<RecordMetadata>();
426425
}).given(producer).send(any(), any());
427426
ProducerFactory<Integer, String> pf = mock(ProducerFactory.class);
428427
given(pf.createProducer()).willReturn(producer);
@@ -449,7 +448,7 @@ void testWithCallbackFailureFunctional() throws Exception {
449448
willAnswer(inv -> {
450449
Callback callback = inv.getArgument(1);
451450
callback.onCompletion(null, new RuntimeException("test"));
452-
return new SettableListenableFuture<RecordMetadata>();
451+
return new CompletableFuture<RecordMetadata>();
453452
}).given(producer).send(any(), any());
454453
ProducerFactory<Integer, String> pf = mock(ProducerFactory.class);
455454
given(pf.createProducer()).willReturn(producer);

spring-kafka/src/test/java/org/springframework/kafka/core/KafkaTemplateTransactionTests.java

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,7 @@
3939
import java.util.Iterator;
4040
import java.util.Map;
4141
import java.util.concurrent.BlockingQueue;
42+
import java.util.concurrent.CompletableFuture;
4243
import java.util.concurrent.LinkedBlockingDeque;
4344
import java.util.concurrent.atomic.AtomicBoolean;
4445

@@ -79,7 +80,6 @@
7980
import org.springframework.transaction.support.AbstractPlatformTransactionManager;
8081
import org.springframework.transaction.support.DefaultTransactionStatus;
8182
import org.springframework.transaction.support.TransactionTemplate;
82-
import org.springframework.util.concurrent.SettableListenableFuture;
8383

8484
/**
8585
* @author Gary Russell
@@ -316,10 +316,10 @@ public void testTransactionSynchronizationExceptionOnCommit() {
316316
public void testDeadLetterPublisherWhileTransactionActive() {
317317
@SuppressWarnings("unchecked")
318318
Producer<Object, Object> producer1 = mock(Producer.class);
319-
given(producer1.send(any(), any())).willReturn(new SettableListenableFuture<>());
319+
given(producer1.send(any(), any())).willReturn(new CompletableFuture<>());
320320
@SuppressWarnings("unchecked")
321321
Producer<Object, Object> producer2 = mock(Producer.class);
322-
given(producer2.send(any(), any())).willReturn(new SettableListenableFuture<>());
322+
given(producer2.send(any(), any())).willReturn(new CompletableFuture<>());
323323
producer1.initTransactions();
324324

325325
@SuppressWarnings("unchecked")
@@ -503,10 +503,10 @@ public void testAbort() {
503503
public void testExecuteInTransactionNewInnerTx() {
504504
@SuppressWarnings("unchecked")
505505
Producer<Object, Object> producer1 = mock(Producer.class);
506-
given(producer1.send(any(), any())).willReturn(new SettableListenableFuture<>());
506+
given(producer1.send(any(), any())).willReturn(new CompletableFuture<>());
507507
@SuppressWarnings("unchecked")
508508
Producer<Object, Object> producer2 = mock(Producer.class);
509-
given(producer2.send(any(), any())).willReturn(new SettableListenableFuture<>());
509+
given(producer2.send(any(), any())).willReturn(new CompletableFuture<>());
510510
producer1.initTransactions();
511511
AtomicBoolean first = new AtomicBoolean(true);
512512

@@ -596,15 +596,15 @@ public static class DeclarativeConfig {
596596
@Bean
597597
public Producer producer1() {
598598
Producer mock = mock(Producer.class);
599-
given(mock.send(any(), any())).willReturn(new SettableListenableFuture<>());
599+
given(mock.send(any(), any())).willReturn(new CompletableFuture<>());
600600
return mock;
601601
}
602602

603603
@SuppressWarnings({ "rawtypes", "unchecked" })
604604
@Bean
605605
public Producer producer2() {
606606
Producer mock = mock(Producer.class);
607-
given(mock.send(any(), any())).willReturn(new SettableListenableFuture<>());
607+
given(mock.send(any(), any())).willReturn(new CompletableFuture<>());
608608
return mock;
609609
}
610610

spring-kafka/src/test/java/org/springframework/kafka/core/RoutingKafkaTemplateTests.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,5 @@
11
/*
2-
* Copyright 2020 the original author or authors.
2+
* Copyright 2020-2022 the original author or authors.
33
*
44
* Licensed under the Apache License, Version 2.0 (the "License");
55
* you may not use this file except in compliance with the License.
@@ -25,13 +25,13 @@
2525

2626
import java.util.LinkedHashMap;
2727
import java.util.Map;
28+
import java.util.concurrent.CompletableFuture;
2829
import java.util.regex.Pattern;
2930

3031
import org.apache.kafka.clients.producer.Producer;
3132
import org.junit.jupiter.api.Test;
3233

3334
import org.springframework.kafka.test.utils.KafkaTestUtils;
34-
import org.springframework.util.concurrent.SettableListenableFuture;
3535

3636
/**
3737
* @author Gary Russell
@@ -44,9 +44,9 @@ public class RoutingKafkaTemplateTests {
4444
@Test
4545
public void routing() {
4646
Producer<Object, Object> p1 = mock(Producer.class);
47-
given(p1.send(any(), any())).willReturn(new SettableListenableFuture<>());
47+
given(p1.send(any(), any())).willReturn(new CompletableFuture<>());
4848
Producer<Object, Object> p2 = mock(Producer.class);
49-
given(p2.send(any(), any())).willReturn(new SettableListenableFuture<>());
49+
given(p2.send(any(), any())).willReturn(new CompletableFuture<>());
5050
ProducerFactory<Object, Object> pf1 = mock(ProducerFactory.class);
5151
ProducerFactory<Object, Object> pf2 = mock(ProducerFactory.class);
5252
given(pf1.createProducer()).willReturn(p1);

spring-kafka/src/test/java/org/springframework/kafka/listener/ConcurrentMessageListenerContainerMockTests.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -101,7 +101,7 @@ void testThreadStarvation() throws InterruptedException {
101101
ThreadPoolTaskExecutor exec = new ThreadPoolTaskExecutor();
102102
exec.setCorePoolSize(1);
103103
exec.afterPropertiesSet();
104-
containerProperties.setConsumerTaskExecutor(exec);
104+
containerProperties.setListenerTaskExecutor(exec);
105105
containerProperties.setConsumerStartTimeout(Duration.ofMillis(50));
106106
ConcurrentMessageListenerContainer container = new ConcurrentMessageListenerContainer<>(consumerFactory,
107107
containerProperties);

spring-kafka/src/test/java/org/springframework/kafka/listener/KafkaMessageListenerContainerTests.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -227,7 +227,7 @@ public void testDelegateType() throws Exception {
227227
ThreadPoolTaskScheduler scheduler = new ThreadPoolTaskScheduler();
228228
scheduler.setPoolSize(10);
229229
scheduler.initialize();
230-
containerProps.setConsumerTaskExecutor(scheduler);
230+
containerProps.setListenerTaskExecutor(scheduler);
231231
KafkaMessageListenerContainer<Integer, String> container =
232232
new KafkaMessageListenerContainer<>(cf, containerProps);
233233
container.setBeanName("delegate");

0 commit comments

Comments
 (0)