Support `@Scheduled` fixedDelay/fixedRate on Publisher-returning methods
This commit adds support for `@Scheduled` annotation on reactive methods and Kotlin suspending functions. Reactive methods are methods that return a `Publisher` or a subclass of `Publisher`. The `ReactiveAdapterRegistry` is used to support many implementations, such as `Flux`, `Mono`, `Flow`, `Single`, etc. Methods should not take any argument and published values will be ignored, as they are already with synchronous support. This is implemented in `ScheduledAnnotationReactiveSupport`, which "converts" Publishers to `Runnable`. This strategy keeps track of active Subscriptions in the `ScheduledAnnotationBeanPostProcessor`, in order to cancel them all in case of shutdown. The existing scheduling support for tasks is reused, aligning the triggering behavior with the existing support: cron, fixedDelay and fixedRate are all supported strategies. If the `Publisher` errors, the exception is logged at warn level and otherwise ignored. As a result new `Runnable` instances will be created for each execution and scheduling will continue. The only difference with synchronous support is that error signals will not be thrown by those `Runnable` tasks and will not be made available to the `org.springframework.util.ErrorHandler` contract. This is due to the asynchronous and lazy nature of Publishers. Closes gh-23533 Closes gh-28515
This commit is contained in:
parent
53f891226e
commit
35052f2113
|
@ -393,6 +393,117 @@ container and once through the `@Configurable` aspect), with the consequence of
|
|||
`@Scheduled` method being invoked twice.
|
||||
====
|
||||
|
||||
[[scheduling-annotation-support-scheduled-reactive]]
|
||||
=== The `@Scheduled` annotation on Reactive methods or Kotlin suspending functions
|
||||
|
||||
As of Spring Framework 6.1, `@Scheduled` methods are also supported on several types
|
||||
of reactive methods:
|
||||
|
||||
- methods with a `Publisher` return type (or any concrete implementation of `Publisher`)
|
||||
like in the following example:
|
||||
|
||||
[source,java,indent=0,subs="verbatim,quotes"]
|
||||
----
|
||||
@Scheduled(fixedDelay = 500)
|
||||
public Publisher<Void> reactiveSomething() {
|
||||
// return an instance of Publisher
|
||||
}
|
||||
----
|
||||
|
||||
- methods with a return type that can be adapted to `Publisher` via the shared instance
|
||||
of the `ReactiveAdapterRegistry`, provided the type supports _deferred subscription_ like
|
||||
in the following example:
|
||||
|
||||
[source,java,indent=0,subs="verbatim,quotes"]
|
||||
----
|
||||
@Scheduled(fixedDelay = 500)
|
||||
public Single<String> rxjavaNonPublisher() {
|
||||
return Single.just("example");
|
||||
}
|
||||
----
|
||||
|
||||
[NOTE]
|
||||
====
|
||||
The `CompletableFuture` class is an example of a type that can typically be adapted
|
||||
to `Publisher` but doesn't support deferred subscription. Its `ReactiveAdapter` in the
|
||||
registry denotes that by having the `getDescriptor().isDeferred()` method return `false`.
|
||||
====
|
||||
|
||||
|
||||
- Kotlin suspending functions, like in the following example:
|
||||
|
||||
[source,kotlin,indent=0,subs="verbatim,quotes"]
|
||||
----
|
||||
@Scheduled(fixedDelay = 500)
|
||||
suspend fun something() {
|
||||
// do something asynchronous
|
||||
}
|
||||
----
|
||||
|
||||
- methods that return a Kotlin `Flow` or `Deferred` instance, like in the following example:
|
||||
|
||||
[source,kotlin,indent=0,subs="verbatim,quotes"]
|
||||
----
|
||||
@Scheduled(fixedDelay = 500)
|
||||
fun something(): Flow<Void> {
|
||||
flow {
|
||||
// do something asynchronous
|
||||
}
|
||||
}
|
||||
----
|
||||
|
||||
All these types of methods must be declared without any arguments. In the case of Kotlin
|
||||
suspending functions the `kotlinx.coroutines.reactor` bridge must also be present to allow
|
||||
the framework to invoke a suspending function as a `Publisher`.
|
||||
|
||||
The Spring Framework will obtain a `Publisher` out of the annotated method once and will
|
||||
schedule a `Runnable` in which it subscribes to said `Publisher`. These inner regular
|
||||
subscriptions happen according to the `cron`/fixedDelay`/`fixedRate` configuration.
|
||||
|
||||
If the `Publisher` emits `onNext` signal(s), these are ignored and discarded (the same way
|
||||
return values from synchronous `@Scheduled` methods are ignored).
|
||||
|
||||
In the following example, the `Flux` emits `onNext("Hello"), onNext("World")` every 5
|
||||
seconds, but these values are unused:
|
||||
|
||||
[source,java,indent=0,subs="verbatim,quotes"]
|
||||
----
|
||||
@Scheduled(initialDelay = 5000, fixedRate = 5000)
|
||||
public Flux<String> reactiveSomething() {
|
||||
return Flux.just("Hello", "World");
|
||||
}
|
||||
----
|
||||
|
||||
If the `Publisher` emits an `onError` signal, it is logged at WARN level and recovered.
|
||||
As a result, further scheduled subscription do happen despite the error.
|
||||
|
||||
In the following example, the `Mono` subscription fails twice in the first five seconds
|
||||
then subscriptions start succeeding, printing a message to the standard output every five
|
||||
seconds:
|
||||
|
||||
[source,java,indent=0,subs="verbatim,quotes"]
|
||||
----
|
||||
@Scheduled(initialDelay = 0, fixedRate = 5000)
|
||||
public Mono<Void> reactiveSomething() {
|
||||
AtomicInteger countdown = new AtomicInteger(2);
|
||||
|
||||
return Mono.defer(() -> {
|
||||
if (countDown.get() == 0 || countDown.decrementAndGet() == 0) {
|
||||
return Mono.fromRunnable(() -> System.out.println("Message"));
|
||||
}
|
||||
return Mono.error(new IllegalStateException("Cannot deliver message"));
|
||||
})
|
||||
}
|
||||
----
|
||||
|
||||
[NOTE]
|
||||
====
|
||||
When destroying the annotated bean or closing the application context Spring Framework cancels
|
||||
scheduled tasks, which includes the next scheduled subscription to the `Publisher` as well
|
||||
as any past subscription that is still currently active (e.g. for long-running publishers,
|
||||
or even infinite publishers).
|
||||
====
|
||||
|
||||
|
||||
[[scheduling-annotation-support-async]]
|
||||
=== The `@Async` annotation
|
||||
|
|
|
@ -27,6 +27,7 @@ dependencies {
|
|||
optional("org.jetbrains.kotlin:kotlin-reflect")
|
||||
optional("org.jetbrains.kotlin:kotlin-stdlib")
|
||||
optional("org.reactivestreams:reactive-streams")
|
||||
optional("io.projectreactor:reactor-core")
|
||||
testImplementation(project(":spring-core-test"))
|
||||
testImplementation(testFixtures(project(":spring-aop")))
|
||||
testImplementation(testFixtures(project(":spring-beans")))
|
||||
|
@ -38,6 +39,8 @@ dependencies {
|
|||
testImplementation("org.awaitility:awaitility")
|
||||
testImplementation("jakarta.inject:jakarta.inject-tck")
|
||||
testImplementation("org.jetbrains.kotlinx:kotlinx-coroutines-core")
|
||||
testImplementation("org.jetbrains.kotlinx:kotlinx-coroutines-reactor")
|
||||
testImplementation("io.reactivex.rxjava3:rxjava")
|
||||
testRuntimeOnly("jakarta.xml.bind:jakarta.xml.bind-api")
|
||||
testRuntimeOnly("org.glassfish:jakarta.el")
|
||||
// Substitute for javax.management:jmxremote_optional:1.0.1_04 (not available on Maven Central)
|
||||
|
|
|
@ -36,6 +36,20 @@ import org.springframework.scheduling.config.ScheduledTaskRegistrar;
|
|||
* a {@code void} return type; if not, the returned value will be ignored
|
||||
* when called through the scheduler.
|
||||
*
|
||||
* <p>Methods that return a reactive {@code Publisher} or a type which can be adapted
|
||||
* to {@code Publisher} by the default {@code ReactiveAdapterRegistry} are supported.
|
||||
* The {@code Publisher} MUST support multiple subsequent subscriptions (i.e. be cold).
|
||||
* The returned Publisher is only produced once, and the scheduling infrastructure
|
||||
* then periodically {@code subscribe()} to it according to configuration.
|
||||
* Values emitted by the publisher are ignored. Errors are logged at WARN level, which
|
||||
* doesn't prevent further iterations. If a {@code fixed delay} is configured, the
|
||||
* subscription is blocked upon in order to respect the fixed delay semantics.
|
||||
*
|
||||
* <p>Kotlin suspending functions are also supported, provided the coroutine-reactor
|
||||
* bridge ({@code kotlinx.coroutine.reactor}) is present at runtime. This bridge is
|
||||
* used to adapt the suspending function into a {@code Publisher} which is treated
|
||||
* the same way as in the reactive method case (see above).
|
||||
*
|
||||
* <p>Processing of {@code @Scheduled} annotations is performed by
|
||||
* registering a {@link ScheduledAnnotationBeanPostProcessor}. This can be
|
||||
* done manually or, more conveniently, through the {@code <task:annotation-driven/>}
|
||||
|
|
|
@ -28,6 +28,7 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.TimeZone;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
@ -98,6 +99,7 @@ import org.springframework.util.StringValueResolver;
|
|||
* @author Elizabeth Chatman
|
||||
* @author Victor Brown
|
||||
* @author Sam Brannen
|
||||
* @author Simon Baslé
|
||||
* @since 3.0
|
||||
* @see Scheduled
|
||||
* @see EnableScheduling
|
||||
|
@ -143,6 +145,8 @@ public class ScheduledAnnotationBeanPostProcessor
|
|||
|
||||
private final Map<Object, Set<ScheduledTask>> scheduledTasks = new IdentityHashMap<>(16);
|
||||
|
||||
private final Map<Object, List<Runnable>> reactiveSubscriptions = new IdentityHashMap<>(16);
|
||||
|
||||
|
||||
/**
|
||||
* Create a default {@code ScheduledAnnotationBeanPostProcessor}.
|
||||
|
@ -385,15 +389,33 @@ public class ScheduledAnnotationBeanPostProcessor
|
|||
}
|
||||
|
||||
/**
|
||||
* Process the given {@code @Scheduled} method declaration on the given bean.
|
||||
* Process the given {@code @Scheduled} method declaration on the given bean,
|
||||
* attempting to distinguish {@link #processScheduledAsync(Scheduled, Method, Object) reactive}
|
||||
* methods from {@link #processScheduledSync(Scheduled, Method, Object) synchronous} methods.
|
||||
* @param scheduled the {@code @Scheduled} annotation
|
||||
* @param method the method that the annotation has been declared on
|
||||
* @param bean the target bean instance
|
||||
* @see #createRunnable(Object, Method)
|
||||
* @see #processScheduledSync(Scheduled, Method, Object)
|
||||
* @see #processScheduledAsync(Scheduled, Method, Object)
|
||||
*/
|
||||
protected void processScheduled(Scheduled scheduled, Method method, Object bean) {
|
||||
// Is method a Kotlin suspending function? Throws if true but reactor bridge isn't on the classpath.
|
||||
// Is method returning a reactive type? Throws if true, but it isn't a deferred Publisher type.
|
||||
if (ScheduledAnnotationReactiveSupport.isReactive(method)) {
|
||||
processScheduledAsync(scheduled, method, bean);
|
||||
return;
|
||||
}
|
||||
processScheduledSync(scheduled, method, bean);
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse the {@code Scheduled} annotation and schedule the provided {@code Runnable}
|
||||
* accordingly. The Runnable can represent either a synchronous method invocation
|
||||
* (see {@link #processScheduledSync(Scheduled, Method, Object)}) or an asynchronous
|
||||
* one (see {@link #processScheduledAsync(Scheduled, Method, Object)}).
|
||||
*/
|
||||
protected void processScheduledTask(Scheduled scheduled, Runnable runnable, Method method, Object bean) {
|
||||
try {
|
||||
Runnable runnable = createRunnable(bean, method);
|
||||
boolean processedSchedule = false;
|
||||
String errorMessage =
|
||||
"Exactly one of the 'cron', 'fixedDelay(String)', or 'fixedRate(String)' attributes is required";
|
||||
|
@ -516,6 +538,53 @@ public class ScheduledAnnotationBeanPostProcessor
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the given {@code @Scheduled} method declaration on the given bean,
|
||||
* as a synchronous method. The method MUST take no arguments. Its return value
|
||||
* is ignored (if any) and the scheduled invocations of the method take place
|
||||
* using the underlying {@link TaskScheduler} infrastructure.
|
||||
* @param scheduled the {@code @Scheduled} annotation
|
||||
* @param method the method that the annotation has been declared on
|
||||
* @param bean the target bean instance
|
||||
* @see #createRunnable(Object, Method)
|
||||
*/
|
||||
protected void processScheduledSync(Scheduled scheduled, Method method, Object bean) {
|
||||
Runnable task;
|
||||
try {
|
||||
task = createRunnable(bean, method);
|
||||
}
|
||||
catch (IllegalArgumentException ex) {
|
||||
throw new IllegalStateException("Could not create recurring task for @Scheduled method '" + method.getName() + "': " + ex.getMessage());
|
||||
}
|
||||
processScheduledTask(scheduled, task, method, bean);
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the given {@code @Scheduled} bean method declaration which returns
|
||||
* a {@code Publisher}, or the given Kotlin suspending function converted to a
|
||||
* Publisher. A {@code Runnable} which subscribes to that publisher is then repeatedly
|
||||
* scheduled according to the annotation configuration.
|
||||
* <p>Note that for fixed delay configuration, the subscription is turned into a blocking
|
||||
* call instead. Types for which a {@code ReactiveAdapter} is registered but which cannot
|
||||
* be deferred (i.e. not a {@code Publisher}) are not supported.
|
||||
* @param scheduled the {@code @Scheduled} annotation
|
||||
* @param method the method that the annotation has been declared on, which
|
||||
* MUST either return a Publisher-adaptable type or be a Kotlin suspending function
|
||||
* @param bean the target bean instance
|
||||
* @see ScheduledAnnotationReactiveSupport
|
||||
*/
|
||||
protected void processScheduledAsync(Scheduled scheduled, Method method, Object bean) {
|
||||
Runnable task;
|
||||
try {
|
||||
task = ScheduledAnnotationReactiveSupport.createSubscriptionRunnable(method, bean, scheduled,
|
||||
this.reactiveSubscriptions.computeIfAbsent(bean, k -> new CopyOnWriteArrayList<>()));
|
||||
}
|
||||
catch (IllegalArgumentException ex) {
|
||||
throw new IllegalStateException("Could not create recurring task for @Scheduled method '" + method.getName() + "': " + ex.getMessage());
|
||||
}
|
||||
processScheduledTask(scheduled, task, method, bean);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a {@link Runnable} for the given bean instance,
|
||||
* calling the specified scheduled method.
|
||||
|
@ -554,6 +623,8 @@ public class ScheduledAnnotationBeanPostProcessor
|
|||
/**
|
||||
* Return all currently scheduled tasks, from {@link Scheduled} methods
|
||||
* as well as from programmatic {@link SchedulingConfigurer} interaction.
|
||||
* <p>Note this includes upcoming scheduled subscriptions for reactive methods,
|
||||
* but doesn't cover any currently active subscription for such methods.
|
||||
* @since 5.0.2
|
||||
*/
|
||||
@Override
|
||||
|
@ -572,20 +643,27 @@ public class ScheduledAnnotationBeanPostProcessor
|
|||
@Override
|
||||
public void postProcessBeforeDestruction(Object bean, String beanName) {
|
||||
Set<ScheduledTask> tasks;
|
||||
List<Runnable> liveSubscriptions;
|
||||
synchronized (this.scheduledTasks) {
|
||||
tasks = this.scheduledTasks.remove(bean);
|
||||
liveSubscriptions = this.reactiveSubscriptions.remove(bean);
|
||||
}
|
||||
if (tasks != null) {
|
||||
for (ScheduledTask task : tasks) {
|
||||
task.cancel();
|
||||
}
|
||||
}
|
||||
if (liveSubscriptions != null) {
|
||||
for (Runnable subscription : liveSubscriptions) {
|
||||
subscription.run(); // equivalent to cancelling the subscription
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean requiresDestruction(Object bean) {
|
||||
synchronized (this.scheduledTasks) {
|
||||
return this.scheduledTasks.containsKey(bean);
|
||||
return this.scheduledTasks.containsKey(bean) || this.reactiveSubscriptions.containsKey(bean);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -599,6 +677,12 @@ public class ScheduledAnnotationBeanPostProcessor
|
|||
}
|
||||
}
|
||||
this.scheduledTasks.clear();
|
||||
Collection<List<Runnable>> allLiveSubscriptions = this.reactiveSubscriptions.values();
|
||||
for (List<Runnable> liveSubscriptions : allLiveSubscriptions) {
|
||||
for (Runnable liveSubscription : liveSubscriptions) {
|
||||
liveSubscription.run(); //equivalent to cancelling the subscription
|
||||
}
|
||||
}
|
||||
}
|
||||
this.registrar.destroy();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,268 @@
|
|||
/*
|
||||
* Copyright 2002-2023 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.scheduling.annotation;
|
||||
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.reactivestreams.Publisher;
|
||||
import org.reactivestreams.Subscriber;
|
||||
import org.reactivestreams.Subscription;
|
||||
import reactor.core.publisher.Flux;
|
||||
|
||||
import org.springframework.aop.support.AopUtils;
|
||||
import org.springframework.core.CoroutinesUtils;
|
||||
import org.springframework.core.KotlinDetector;
|
||||
import org.springframework.core.ReactiveAdapter;
|
||||
import org.springframework.core.ReactiveAdapterRegistry;
|
||||
import org.springframework.lang.Nullable;
|
||||
import org.springframework.util.Assert;
|
||||
import org.springframework.util.ClassUtils;
|
||||
import org.springframework.util.ReflectionUtils;
|
||||
import org.springframework.util.StringUtils;
|
||||
|
||||
/**
|
||||
* Helper class for @{@link ScheduledAnnotationBeanPostProcessor} to support reactive cases
|
||||
* without a dependency on optional classes.
|
||||
* @author Simon Baslé
|
||||
* @since 6.1.0
|
||||
*/
|
||||
abstract class ScheduledAnnotationReactiveSupport {
|
||||
|
||||
static final boolean reactorPresent = ClassUtils.isPresent(
|
||||
"reactor.core.publisher.Flux", ScheduledAnnotationReactiveSupport.class.getClassLoader());
|
||||
|
||||
static final boolean coroutinesReactorPresent = ClassUtils.isPresent(
|
||||
"kotlinx.coroutines.reactor.MonoKt", ScheduledAnnotationReactiveSupport.class.getClassLoader());
|
||||
|
||||
private static final Log LOGGER = LogFactory.getLog(ScheduledAnnotationReactiveSupport.class);
|
||||
|
||||
/**
|
||||
* Checks that if the method is reactive, it can be scheduled. Methods are considered
|
||||
* eligible for reactive scheduling if they either return an instance of a type that
|
||||
* can be converted to {@code Publisher} or are a Kotlin Suspending Function.
|
||||
* If the method isn't matching these criteria then this check returns {@code false}.
|
||||
* <p>For scheduling of Kotlin Suspending Functions, the Coroutine-Reactor bridge
|
||||
* {@code kotlinx.coroutines.reactor} MUST be present at runtime (in order to invoke
|
||||
* suspending functions as a {@code Publisher}).
|
||||
* Provided that is the case, this method returns {@code true}. Otherwise, it throws
|
||||
* an {@code IllegalStateException}.
|
||||
* @throws IllegalStateException if the method is reactive but Reactor and/or the
|
||||
* Kotlin coroutines bridge are not present at runtime
|
||||
*/
|
||||
static boolean isReactive(Method method) {
|
||||
if (KotlinDetector.isKotlinPresent() && KotlinDetector.isSuspendingFunction(method)) {
|
||||
//Note that suspending functions declared without args have a single Continuation parameter in reflective inspection
|
||||
Assert.isTrue(method.getParameterCount() == 1,"Kotlin suspending functions may only be"
|
||||
+ " annotated with @Scheduled if declared without arguments");
|
||||
Assert.isTrue(coroutinesReactorPresent, "Kotlin suspending functions may only be annotated with"
|
||||
+ " @Scheduled if the Coroutine-Reactor bridge (kotlinx.coroutines.reactor) is present at runtime");
|
||||
return true;
|
||||
}
|
||||
ReactiveAdapterRegistry registry = ReactiveAdapterRegistry.getSharedInstance();
|
||||
if (!registry.hasAdapters()) {
|
||||
return false;
|
||||
}
|
||||
Class<?> returnType = method.getReturnType();
|
||||
ReactiveAdapter candidateAdapter = registry.getAdapter(returnType);
|
||||
if (candidateAdapter == null) {
|
||||
return false;
|
||||
}
|
||||
Assert.isTrue(method.getParameterCount() == 0, "Reactive methods may only be annotated with"
|
||||
+ " @Scheduled if declared without arguments");
|
||||
Assert.isTrue(candidateAdapter.getDescriptor().isDeferred(), "Reactive methods may only be annotated with"
|
||||
+ " @Scheduled if the return type supports deferred execution");
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Turn the invocation of the provided {@code Method} into a {@code Publisher},
|
||||
* either by reflectively invoking it and converting the result to a {@code Publisher}
|
||||
* via {@link ReactiveAdapterRegistry} or by converting a Kotlin suspending function
|
||||
* into a {@code Publisher} via {@link CoroutinesUtils}.
|
||||
* The {@link #isReactive(Method)} check is a precondition to calling this method.
|
||||
* If Reactor is present at runtime, the Publisher is additionally converted to a {@code Flux}
|
||||
* with a checkpoint String, allowing for better debugging.
|
||||
*/
|
||||
static Publisher<?> getPublisherFor(Method method, Object bean) {
|
||||
if (KotlinDetector.isKotlinPresent() && KotlinDetector.isSuspendingFunction(method)) {
|
||||
return CoroutinesUtils.invokeSuspendingFunction(method, bean, (Object[]) method.getParameters());
|
||||
}
|
||||
|
||||
ReactiveAdapterRegistry registry = ReactiveAdapterRegistry.getSharedInstance();
|
||||
Class<?> returnType = method.getReturnType();
|
||||
ReactiveAdapter adapter = registry.getAdapter(returnType);
|
||||
if (adapter == null) {
|
||||
throw new IllegalArgumentException("Cannot convert the @Scheduled reactive method return type to Publisher");
|
||||
}
|
||||
if (!adapter.getDescriptor().isDeferred()) {
|
||||
throw new IllegalArgumentException("Cannot convert the @Scheduled reactive method return type to Publisher, "
|
||||
+ returnType.getSimpleName() + " is not a deferred reactive type");
|
||||
}
|
||||
Method invocableMethod = AopUtils.selectInvocableMethod(method, bean.getClass());
|
||||
try {
|
||||
ReflectionUtils.makeAccessible(invocableMethod);
|
||||
Object r = invocableMethod.invoke(bean);
|
||||
|
||||
Publisher<?> publisher = adapter.toPublisher(r);
|
||||
//if Reactor is on the classpath, we could benefit from having a checkpoint for debuggability
|
||||
if (reactorPresent) {
|
||||
final String checkpoint = "@Scheduled '"+ method.getName() + "()' in bean '"
|
||||
+ method.getDeclaringClass().getName() + "'";
|
||||
return Flux.from(publisher).checkpoint(checkpoint);
|
||||
}
|
||||
else {
|
||||
return publisher;
|
||||
}
|
||||
}
|
||||
catch (InvocationTargetException ex) {
|
||||
throw new IllegalArgumentException("Cannot obtain a Publisher-convertible value from the @Scheduled reactive method", ex.getTargetException());
|
||||
}
|
||||
catch (IllegalAccessException ex) {
|
||||
throw new IllegalArgumentException("Cannot obtain a Publisher-convertible value from the @Scheduled reactive method", ex);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a {@link Runnable} for the Scheduled infrastructure, allowing for scheduled
|
||||
* subscription to the publisher produced by a reactive method.
|
||||
* <p>Note that the reactive method is invoked once, but the resulting {@code Publisher}
|
||||
* is subscribed to repeatedly, once per each invocation of the {@code Runnable}.
|
||||
* <p>In the case of a {@code fixed delay} configuration, the subscription inside the
|
||||
* Runnable is turned into a blocking call in order to maintain fixedDelay semantics
|
||||
* (i.e. the task blocks until completion of the Publisher, then the delay is applied
|
||||
* until next iteration).
|
||||
*/
|
||||
static Runnable createSubscriptionRunnable(Method method, Object targetBean, Scheduled scheduled,
|
||||
List<Runnable> subscriptionTrackerRegistry) {
|
||||
boolean shouldBlock = scheduled.fixedDelay() > 0 || StringUtils.hasText(scheduled.fixedDelayString());
|
||||
final Publisher<?> publisher = getPublisherFor(method, targetBean);
|
||||
return new SubscribingRunnable(publisher, shouldBlock, subscriptionTrackerRegistry);
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility implementation of {@code Runnable} that subscribes to a {@code Publisher}
|
||||
* or subscribes-then-blocks if {@code shouldBlock} is set to {@code true}.
|
||||
*/
|
||||
static final class SubscribingRunnable implements Runnable {
|
||||
|
||||
final Publisher<?> publisher;
|
||||
final boolean shouldBlock;
|
||||
final List<Runnable> subscriptionTrackerRegistry;
|
||||
|
||||
SubscribingRunnable(Publisher<?> publisher, boolean shouldBlock, List<Runnable> subscriptionTrackerRegistry) {
|
||||
this.publisher = publisher;
|
||||
this.shouldBlock = shouldBlock;
|
||||
this.subscriptionTrackerRegistry = subscriptionTrackerRegistry;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
if (this.shouldBlock) {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
TrackingSubscriber subscriber = new TrackingSubscriber(this.subscriptionTrackerRegistry, latch);
|
||||
this.subscriptionTrackerRegistry.add(subscriber);
|
||||
this.publisher.subscribe(subscriber);
|
||||
try {
|
||||
latch.await();
|
||||
}
|
||||
catch (InterruptedException ex) {
|
||||
throw new RuntimeException(ex);
|
||||
}
|
||||
}
|
||||
else {
|
||||
final TrackingSubscriber subscriber = new TrackingSubscriber(this.subscriptionTrackerRegistry);
|
||||
this.subscriptionTrackerRegistry.add(subscriber);
|
||||
this.publisher.subscribe(subscriber);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A {@code Subscriber} which keeps track of its {@code Subscription} and exposes the
|
||||
* capacity to cancel the subscription as a {@code Runnable}. Can optionally support
|
||||
* blocking if a {@code CountDownLatch} is passed at construction.
|
||||
*/
|
||||
private static final class TrackingSubscriber implements Subscriber<Object>, Runnable {
|
||||
|
||||
private final List<Runnable> subscriptionTrackerRegistry;
|
||||
|
||||
@Nullable
|
||||
private final CountDownLatch blockingLatch;
|
||||
|
||||
/*
|
||||
Implementation note: since this is created last minute when subscribing,
|
||||
there shouldn't be a way to cancel the tracker externally from the
|
||||
ScheduledAnnotationBeanProcessor before the #setSubscription(Subscription)
|
||||
method is called.
|
||||
*/
|
||||
@Nullable
|
||||
private Subscription s;
|
||||
|
||||
TrackingSubscriber(List<Runnable> subscriptionTrackerRegistry) {
|
||||
this(subscriptionTrackerRegistry, null);
|
||||
}
|
||||
|
||||
TrackingSubscriber(List<Runnable> subscriptionTrackerRegistry, @Nullable CountDownLatch latch) {
|
||||
this.subscriptionTrackerRegistry = subscriptionTrackerRegistry;
|
||||
this.blockingLatch = latch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
if (this.s != null) {
|
||||
this.s.cancel();
|
||||
}
|
||||
if (this.blockingLatch != null) {
|
||||
this.blockingLatch.countDown();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onSubscribe(Subscription s) {
|
||||
this.s = s;
|
||||
s.request(Integer.MAX_VALUE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onNext(Object o) {
|
||||
// NO-OP
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onError(Throwable ex) {
|
||||
this.subscriptionTrackerRegistry.remove(this);
|
||||
LOGGER.warn("Unexpected error occurred in scheduled reactive task", ex);
|
||||
if (this.blockingLatch != null) {
|
||||
this.blockingLatch.countDown();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onComplete() {
|
||||
this.subscriptionTrackerRegistry.remove(this);
|
||||
if (this.blockingLatch != null) {
|
||||
this.blockingLatch.countDown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,238 @@
|
|||
/*
|
||||
* Copyright 2002-2023 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.scheduling.annotation;
|
||||
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import io.reactivex.rxjava3.core.Completable;
|
||||
import io.reactivex.rxjava3.core.Flowable;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
import org.reactivestreams.Publisher;
|
||||
import reactor.core.publisher.Flux;
|
||||
import reactor.core.publisher.Mono;
|
||||
|
||||
import org.springframework.core.annotation.AnnotationUtils;
|
||||
import org.springframework.util.ReflectionUtils;
|
||||
|
||||
import static org.assertj.core.api.Assertions.assertThat;
|
||||
import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException;
|
||||
import static org.springframework.scheduling.annotation.ScheduledAnnotationReactiveSupport.createSubscriptionRunnable;
|
||||
import static org.springframework.scheduling.annotation.ScheduledAnnotationReactiveSupport.getPublisherFor;
|
||||
import static org.springframework.scheduling.annotation.ScheduledAnnotationReactiveSupport.isReactive;
|
||||
|
||||
class ScheduledAnnotationReactiveSupportTests {
|
||||
|
||||
@Test
|
||||
void ensureReactor() {
|
||||
assertThat(ScheduledAnnotationReactiveSupport.reactorPresent).isTrue();
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = { "mono", "flux", "monoString", "fluxString", "publisherMono",
|
||||
"publisherString", "monoThrows", "flowable", "completable" }) //note: monoWithParams can't be found by this test
|
||||
void checkIsReactive(String method) {
|
||||
Method m = ReflectionUtils.findMethod(ReactiveMethods.class, method);
|
||||
assertThat(isReactive(m)).as(m.getName()).isTrue();
|
||||
}
|
||||
|
||||
@Test
|
||||
void checkNotReactive() {
|
||||
Method string = ReflectionUtils.findMethod(ReactiveMethods.class, "oops");
|
||||
|
||||
assertThat(isReactive(string))
|
||||
.as("String-returning").isFalse();
|
||||
}
|
||||
|
||||
@Test
|
||||
void rejectReactiveAdaptableButNotDeferred() {
|
||||
Method future = ReflectionUtils.findMethod(ReactiveMethods.class, "future");
|
||||
|
||||
assertThatIllegalArgumentException().isThrownBy(() -> isReactive(future))
|
||||
.withMessage("Reactive methods may only be annotated with @Scheduled if the return type supports deferred execution");
|
||||
}
|
||||
|
||||
@Test
|
||||
void isReactiveRejectsWithParams() {
|
||||
Method m = ReflectionUtils.findMethod(ReactiveMethods.class, "monoWithParam", String.class);
|
||||
|
||||
//isReactive rejects with context
|
||||
assertThatIllegalArgumentException().isThrownBy(() -> isReactive(m))
|
||||
.withMessage("Reactive methods may only be annotated with @Scheduled if declared without arguments")
|
||||
.withNoCause();
|
||||
}
|
||||
|
||||
@Test
|
||||
void rejectCantProducePublisher() {
|
||||
ReactiveMethods target = new ReactiveMethods();
|
||||
Method m = ReflectionUtils.findMethod(ReactiveMethods.class, "monoThrows");
|
||||
|
||||
//static helper method
|
||||
assertThatIllegalArgumentException().isThrownBy(() -> getPublisherFor(m, target))
|
||||
.withMessage("Cannot obtain a Publisher-convertible value from the @Scheduled reactive method")
|
||||
.withCause(new IllegalStateException("expected"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void rejectCantAccessMethod() {
|
||||
ReactiveMethods target = new ReactiveMethods();
|
||||
Method m = ReflectionUtils.findMethod(ReactiveMethods.class, "monoThrowsIllegalAccess");
|
||||
|
||||
//static helper method
|
||||
assertThatIllegalArgumentException().isThrownBy(() -> getPublisherFor(m, target))
|
||||
.withMessage("Cannot obtain a Publisher-convertible value from the @Scheduled reactive method")
|
||||
.withCause(new IllegalAccessException("expected"));
|
||||
}
|
||||
|
||||
@Test
|
||||
void fixedDelayIsBlocking() {
|
||||
ReactiveMethods target = new ReactiveMethods();
|
||||
Method m = ReflectionUtils.findMethod(ReactiveMethods.class, "mono");
|
||||
Scheduled fixedDelayString = AnnotationUtils.synthesizeAnnotation(Map.of("fixedDelayString", "123"), Scheduled.class, null);
|
||||
Scheduled fixedDelayLong = AnnotationUtils.synthesizeAnnotation(Map.of("fixedDelay", 123L), Scheduled.class, null);
|
||||
List<Runnable> tracker = new ArrayList<>();
|
||||
|
||||
assertThat(createSubscriptionRunnable(m, target, fixedDelayString, tracker))
|
||||
.isInstanceOfSatisfying(ScheduledAnnotationReactiveSupport.SubscribingRunnable.class, sr ->
|
||||
assertThat(sr.shouldBlock).as("fixedDelayString.shouldBlock").isTrue()
|
||||
);
|
||||
|
||||
assertThat(createSubscriptionRunnable(m, target, fixedDelayLong, tracker))
|
||||
.isInstanceOfSatisfying(ScheduledAnnotationReactiveSupport.SubscribingRunnable.class, sr ->
|
||||
assertThat(sr.shouldBlock).as("fixedDelayLong.shouldBlock").isTrue()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void fixedRateIsNotBlocking() {
|
||||
ReactiveMethods target = new ReactiveMethods();
|
||||
Method m = ReflectionUtils.findMethod(ReactiveMethods.class, "mono");
|
||||
Scheduled fixedRateString = AnnotationUtils.synthesizeAnnotation(Map.of("fixedRateString", "123"), Scheduled.class, null);
|
||||
Scheduled fixedRateLong = AnnotationUtils.synthesizeAnnotation(Map.of("fixedRate", 123L), Scheduled.class, null);
|
||||
List<Runnable> tracker = new ArrayList<>();
|
||||
|
||||
assertThat(createSubscriptionRunnable(m, target, fixedRateString, tracker))
|
||||
.isInstanceOfSatisfying(ScheduledAnnotationReactiveSupport.SubscribingRunnable.class, sr ->
|
||||
assertThat(sr.shouldBlock).as("fixedRateString.shouldBlock").isFalse()
|
||||
);
|
||||
|
||||
assertThat(createSubscriptionRunnable(m, target, fixedRateLong, tracker))
|
||||
.isInstanceOfSatisfying(ScheduledAnnotationReactiveSupport.SubscribingRunnable.class, sr ->
|
||||
assertThat(sr.shouldBlock).as("fixedRateLong.shouldBlock").isFalse()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void cronIsNotBlocking() {
|
||||
ReactiveMethods target = new ReactiveMethods();
|
||||
Method m = ReflectionUtils.findMethod(ReactiveMethods.class, "mono");
|
||||
Scheduled cron = AnnotationUtils.synthesizeAnnotation(Map.of("cron", "-"), Scheduled.class, null);
|
||||
List<Runnable> tracker = new ArrayList<>();
|
||||
|
||||
assertThat(createSubscriptionRunnable(m, target, cron, tracker))
|
||||
.isInstanceOfSatisfying(ScheduledAnnotationReactiveSupport.SubscribingRunnable.class, sr ->
|
||||
assertThat(sr.shouldBlock).as("cron.shouldBlock").isFalse()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
void hasCheckpointToString() {
|
||||
ReactiveMethods target = new ReactiveMethods();
|
||||
Method m = ReflectionUtils.findMethod(ReactiveMethods.class, "mono");
|
||||
Publisher<?> p = getPublisherFor(m, target);
|
||||
|
||||
assertThat(p.getClass().getName())
|
||||
.as("checkpoint class")
|
||||
.isEqualTo("reactor.core.publisher.FluxOnAssembly");
|
||||
|
||||
assertThat(p).hasToString("checkpoint(\"@Scheduled 'mono()' in bean 'org.springframework.scheduling.annotation.ScheduledAnnotationReactiveSupportTests$ReactiveMethods'\")");
|
||||
}
|
||||
|
||||
static class ReactiveMethods {
|
||||
|
||||
public String oops() {
|
||||
return "oops";
|
||||
}
|
||||
|
||||
public Mono<Void> mono() {
|
||||
return Mono.empty();
|
||||
}
|
||||
|
||||
public Flux<Void> flux() {
|
||||
return Flux.empty();
|
||||
}
|
||||
|
||||
public Mono<String> monoString() {
|
||||
return Mono.just("example");
|
||||
}
|
||||
|
||||
public Flux<String> fluxString() {
|
||||
return Flux.just("example");
|
||||
}
|
||||
|
||||
public Publisher<Void> publisherMono() {
|
||||
return Mono.empty();
|
||||
}
|
||||
|
||||
public Publisher<String> publisherString() {
|
||||
return fluxString();
|
||||
}
|
||||
|
||||
public CompletableFuture<String> future() {
|
||||
return CompletableFuture.completedFuture("example");
|
||||
}
|
||||
|
||||
public Mono<Void> monoWithParam(String param) {
|
||||
return Mono.just(param).then();
|
||||
}
|
||||
|
||||
public Mono<Void> monoThrows() {
|
||||
throw new IllegalStateException("expected");
|
||||
}
|
||||
|
||||
public Mono<Void> monoThrowsIllegalAccess() throws IllegalAccessException {
|
||||
//simulate a reflection issue
|
||||
throw new IllegalAccessException("expected");
|
||||
}
|
||||
|
||||
public Flowable<Void> flowable() {
|
||||
return Flowable.empty();
|
||||
}
|
||||
|
||||
public Completable completable() {
|
||||
return Completable.complete();
|
||||
}
|
||||
|
||||
AtomicInteger subscription = new AtomicInteger();
|
||||
|
||||
public Mono<Void> trackingMono() {
|
||||
return Mono.<Void>empty()
|
||||
.doOnSubscribe(s -> subscription.incrementAndGet());
|
||||
}
|
||||
|
||||
public Mono<Void> monoError() {
|
||||
return Mono.error(new IllegalStateException("expected"));
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,155 @@
|
|||
/*
|
||||
* Copyright 2002-2023 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.scheduling.annotation
|
||||
|
||||
import kotlinx.coroutines.CompletableDeferred
|
||||
import kotlinx.coroutines.Deferred
|
||||
import kotlinx.coroutines.flow.Flow
|
||||
import kotlinx.coroutines.flow.flowOf
|
||||
import org.assertj.core.api.Assertions
|
||||
import org.assertj.core.api.Assertions.assertThat
|
||||
import org.junit.jupiter.api.BeforeEach
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
import org.springframework.scheduling.annotation.ScheduledAnnotationReactiveSupport.getPublisherFor
|
||||
import org.springframework.scheduling.annotation.ScheduledAnnotationReactiveSupport.isReactive
|
||||
import org.springframework.util.ReflectionUtils
|
||||
import reactor.core.publisher.Mono
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import kotlin.coroutines.Continuation
|
||||
|
||||
class KotlinScheduledAnnotationReactiveSupportTests {
|
||||
|
||||
@Test
|
||||
fun ensureReactor() {
|
||||
assertThat(ScheduledAnnotationReactiveSupport.reactorPresent).isTrue
|
||||
}
|
||||
|
||||
@Test
|
||||
fun ensureKotlinCoroutineReactorBridge() {
|
||||
assertThat(ScheduledAnnotationReactiveSupport.coroutinesReactorPresent).isTrue
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = ["suspending", "suspendingReturns"])
|
||||
fun isReactiveSuspending(methodName: String) {
|
||||
val method = ReflectionUtils.findMethod(SuspendingFunctions::class.java, methodName, Continuation::class.java)!!
|
||||
assertThat(isReactive(method)).isTrue
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = ["flow", "deferred"])
|
||||
fun isReactiveKotlinType(methodName: String) {
|
||||
val method = ReflectionUtils.findMethod(SuspendingFunctions::class.java, methodName)!!
|
||||
assertThat(isReactive(method)).isTrue
|
||||
}
|
||||
|
||||
@Test
|
||||
fun isNotReactive() {
|
||||
val method = ReflectionUtils.findMethod(SuspendingFunctions::class.java, "notSuspending")!!
|
||||
assertThat(isReactive(method)).isFalse
|
||||
}
|
||||
|
||||
internal class SuspendingFunctions {
|
||||
suspend fun suspending() {
|
||||
}
|
||||
|
||||
suspend fun suspendingReturns(): String = "suspended"
|
||||
|
||||
suspend fun withParam(param: String): String {
|
||||
return param
|
||||
}
|
||||
|
||||
suspend fun throwsIllegalState() {
|
||||
throw IllegalStateException("expected")
|
||||
}
|
||||
|
||||
var subscription = AtomicInteger()
|
||||
suspend fun suspendingTracking() {
|
||||
subscription.incrementAndGet()
|
||||
}
|
||||
|
||||
fun notSuspending() { }
|
||||
|
||||
fun flow(): Flow<Void> {
|
||||
return flowOf()
|
||||
}
|
||||
|
||||
fun deferred(): Deferred<Void> {
|
||||
return CompletableDeferred()
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private var target: SuspendingFunctions? = null
|
||||
|
||||
@BeforeEach
|
||||
fun init() {
|
||||
target = SuspendingFunctions()
|
||||
}
|
||||
|
||||
@Test
|
||||
fun checkKotlinRuntimeIfNeeded() {
|
||||
val suspendingMethod = ReflectionUtils.findMethod(SuspendingFunctions::class.java, "suspending", Continuation::class.java)!!
|
||||
val notSuspendingMethod = ReflectionUtils.findMethod(SuspendingFunctions::class.java, "notSuspending")!!
|
||||
|
||||
assertThat(isReactive(suspendingMethod)).describedAs("suspending").isTrue()
|
||||
assertThat(isReactive(notSuspendingMethod)).describedAs("not suspending").isFalse()
|
||||
}
|
||||
|
||||
@Test
|
||||
fun isReactiveRejectsWithParams() {
|
||||
val m = ReflectionUtils.findMethod(SuspendingFunctions::class.java, "withParam", String::class.java, Continuation::class.java)!!
|
||||
|
||||
//isReactive rejects with some context
|
||||
Assertions.assertThatIllegalArgumentException().isThrownBy { isReactive(m) }
|
||||
.withMessage("Kotlin suspending functions may only be annotated with @Scheduled if declared without arguments")
|
||||
.withNoCause()
|
||||
}
|
||||
|
||||
@Test
|
||||
fun rejectNotSuspending() {
|
||||
val m = ReflectionUtils.findMethod(SuspendingFunctions::class.java, "notSuspending")
|
||||
|
||||
//static helper method
|
||||
Assertions.assertThatIllegalArgumentException().isThrownBy { getPublisherFor(m!!, target!!) }
|
||||
.withMessage("Cannot convert the @Scheduled reactive method return type to Publisher")
|
||||
.withNoCause()
|
||||
}
|
||||
|
||||
@Test
|
||||
fun suspendingThrowIsTurnedToMonoError() {
|
||||
val m = ReflectionUtils.findMethod(SuspendingFunctions::class.java, "throwsIllegalState", Continuation::class.java)
|
||||
|
||||
val mono = Mono.from(getPublisherFor(m!!, target!!))
|
||||
|
||||
Assertions.assertThatIllegalStateException().isThrownBy { mono.block() }
|
||||
.withMessage("expected")
|
||||
.withNoCause()
|
||||
}
|
||||
|
||||
@Test
|
||||
fun turningSuspendingFunctionToMonoDoesntExecuteTheMethod() {
|
||||
val m = ReflectionUtils.findMethod(SuspendingFunctions::class.java, "suspendingTracking", Continuation::class.java)
|
||||
val mono = Mono.from(getPublisherFor(m!!, target!!))
|
||||
|
||||
assertThat(target!!.subscription).hasValue(0)
|
||||
mono.block()
|
||||
assertThat(target!!.subscription).describedAs("after subscription").hasValue(1)
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue