From 1319ce045aeccdd8f32e2c93ddefa206f297e7c2 Mon Sep 17 00:00:00 2001 From: Stephane Maldini Date: Wed, 31 May 2017 16:01:24 -0700 Subject: [PATCH] Move io.projectreactor.addons:reactor-test to io.projectreactor:reactor-test Original history tagged as https://github.com/reactor/reactor-addons/tree/testExtractedToCore --- build.gradle | 14 + .../test/DefaultStepVerifierBuilder.java | 2157 +++++++++++++++++ .../main/java/reactor/test/StepVerifier.java | 994 ++++++++ .../reactor/test/StepVerifierOptions.java | 104 + .../test/publisher/DefaultTestPublisher.java | 374 +++ .../reactor/test/publisher/TestPublisher.java | 225 ++ .../test/scheduler/VirtualTimeScheduler.java | 535 ++++ .../test/DefaultStepVerifierBuilderTests.java | 180 ++ .../test/StepVerifierAssertionsTests.java | 536 ++++ .../java/reactor/test/StepVerifierTests.java | 1665 +++++++++++++ .../publisher/DefaultTestPublisherTests.java | 282 +++ .../scheduler/VirtualTimeSchedulerTests.java | 123 + settings.gradle | 2 +- 13 files changed, 7190 insertions(+), 1 deletion(-) create mode 100644 reactor-test/src/main/java/reactor/test/DefaultStepVerifierBuilder.java create mode 100644 reactor-test/src/main/java/reactor/test/StepVerifier.java create mode 100644 reactor-test/src/main/java/reactor/test/StepVerifierOptions.java create mode 100644 reactor-test/src/main/java/reactor/test/publisher/DefaultTestPublisher.java create mode 100644 reactor-test/src/main/java/reactor/test/publisher/TestPublisher.java create mode 100644 reactor-test/src/main/java/reactor/test/scheduler/VirtualTimeScheduler.java create mode 100644 reactor-test/src/test/java/reactor/test/DefaultStepVerifierBuilderTests.java create mode 100644 reactor-test/src/test/java/reactor/test/StepVerifierAssertionsTests.java create mode 100644 reactor-test/src/test/java/reactor/test/StepVerifierTests.java create mode 100644 reactor-test/src/test/java/reactor/test/publisher/DefaultTestPublisherTests.java create mode 100644 reactor-test/src/test/java/reactor/test/scheduler/VirtualTimeSchedulerTests.java diff --git a/build.gradle b/build.gradle index 9b52b2873a..3e31dfd224 100644 --- a/build.gradle +++ b/build.gradle @@ -261,4 +261,18 @@ project('reactor-core') { check.dependsOn jacocoTestReport } +project('reactor-test') { + description = 'Reactor Test support' + + dependencies { + compile project(":reactor-core") + + testCompile 'junit:junit:4.12' + + testCompile "org.hamcrest:hamcrest-library:1.3", + "org.assertj:assertj-core:$assertJVersion", + "org.mockito:mockito-core:$mockitoVersion" + } +} + assemble.dependsOn docsZip diff --git a/reactor-test/src/main/java/reactor/test/DefaultStepVerifierBuilder.java b/reactor-test/src/main/java/reactor/test/DefaultStepVerifierBuilder.java new file mode 100644 index 0000000000..f9d2719341 --- /dev/null +++ b/reactor-test/src/main/java/reactor/test/DefaultStepVerifierBuilder.java @@ -0,0 +1,2157 @@ +/* + * Copyright (c) 2011-2017 Pivotal Software Inc, All Rights Reserved. + * + * 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 + * + * http://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 reactor.test; + +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.function.BiFunction; +import java.util.function.BooleanSupplier; +import java.util.function.Consumer; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.reactivestreams.Publisher; +import org.reactivestreams.Subscriber; +import org.reactivestreams.Subscription; +import reactor.core.Exceptions; +import reactor.core.Fuseable; +import reactor.core.publisher.Hooks; +import reactor.core.publisher.Operators; +import reactor.core.publisher.Signal; +import reactor.test.StepVerifier.Step; +import reactor.test.scheduler.VirtualTimeScheduler; +import reactor.util.Logger; +import reactor.util.Loggers; +import reactor.util.function.Tuple2; +import reactor.util.function.Tuples; + +/** + * Default implementation of {@link StepVerifier.Step} and + * {@link StepVerifier.LastStep}. + * + * @author Arjen Poutsma + * @since 1.0 + */ +final class DefaultStepVerifierBuilder + implements StepVerifier.FirstStep { + + static void checkPositive(long n) { + if (n < 0) { + throw new IllegalArgumentException("'n' should be >= 0 but was " + n); + } + } + + static void checkStrictlyPositive(long n) { + if (n <= 0) { + throw new IllegalArgumentException("'n' should be > 0 but was " + n); + } + } + + static StepVerifier.FirstStep newVerifier(StepVerifierOptions options, + Supplier> scenarioSupplier) { + DefaultStepVerifierBuilder.checkPositive(options.getInitialRequest()); + Objects.requireNonNull(scenarioSupplier, "scenarioSupplier"); + + return new DefaultStepVerifierBuilder<>(options, scenarioSupplier); + } + + @SuppressWarnings("unchecked") + static SignalEvent defaultFirstStep() { + return (SignalEvent) DEFAULT_ONSUBSCRIBE_STEP; + } + + final List> script; + final long initialRequest; + final Supplier vtsLookup; + final Supplier> sourceSupplier; + private final StepVerifierOptions options; + + long hangCheckRequested; + int requestedFusionMode = -1; + int expectedFusionMode = -1; + + DefaultStepVerifierBuilder(StepVerifierOptions options, + Supplier> sourceSupplier) { + this.initialRequest = options.getInitialRequest(); + this.options = options; + this.vtsLookup = options.getVirtualTimeSchedulerSupplier(); + this.sourceSupplier = sourceSupplier; + this.script = new ArrayList<>(); + this.script.add(defaultFirstStep()); + + this.hangCheckRequested = initialRequest; + } + + @Override + public DefaultStepVerifierBuilder as(String description) { + this.script.add(new DescriptionEvent<>(description)); + return this; + } + + @Override + public DefaultStepVerifier consumeErrorWith(Consumer consumer) { + Objects.requireNonNull(consumer, "consumer"); + SignalEvent event = new SignalEvent<>((signal, se) -> { + if (!signal.isOnError()) { + return fail(se, "expected: onError(); actual: %s", signal); + } + else { + consumer.accept(signal.getThrowable()); + return Optional.empty(); + } + }, "consumeErrorWith"); + this.script.add(event); + return build(); + } + + @Override + public DefaultStepVerifierBuilder assertNext(Consumer consumer) { + return consumeNextWith(consumer, "assertNext"); + } + + @Override + public DefaultStepVerifierBuilder consumeNextWith( + Consumer consumer) { + return consumeNextWith(consumer, "consumeNextWith"); + } + + private DefaultStepVerifierBuilder consumeNextWith(Consumer consumer, String description) { + Objects.requireNonNull(consumer, "consumer"); + checkPotentialHang(1, description); + SignalEvent event = new SignalEvent<>((signal, se) -> { + if (!signal.isOnNext()) { + return fail(se, "expected: onNext(); actual: %s", signal); + } + else { + consumer.accept(signal.get()); + return Optional.empty(); + } + }, description); + this.script.add(event); + return this; + } + + @Override + public DefaultStepVerifierBuilder consumeRecordedWith( + Consumer> consumer) { + Objects.requireNonNull(consumer, "consumer"); + this.script.add(new CollectEvent<>(consumer, "consumeRecordedWith")); + return this; + } + + @Override + public DefaultStepVerifierBuilder consumeSubscriptionWith( + Consumer consumer) { + Objects.requireNonNull(consumer, "consumer"); + if(script.isEmpty() || (script.size() == 1 && script.get(0) == DEFAULT_ONSUBSCRIBE_STEP)) { + this.script.set(0, new SignalEvent<>((signal, se) -> { + if (!signal.isOnSubscribe()) { + return fail(se, "expected: onSubscribe(); actual: %s", signal); + } + else { + consumer.accept(signal.getSubscription()); + return Optional.empty(); + } + }, "consumeSubscriptionWith")); + } + else { + this.script.add(new SubscriptionConsumerEvent<>(consumer, + "consumeSubscriptionWith")); + } + return this; + } + + @Override + public DefaultStepVerifier expectComplete() { + SignalEvent event = new SignalEvent<>((signal, se) -> { + if (!signal.isOnComplete()) { + return fail(se, "expected: onComplete(); actual: %s", signal); + } + else { + return Optional.empty(); + } + }, "expectComplete"); + this.script.add(event); + return build(); + } + + @Override + public DefaultStepVerifier expectError() { + SignalEvent event = new SignalEvent<>((signal, se) -> { + if (!signal.isOnError()) { + return fail(se, "expected: onError(); actual: %s", signal); + } + else { + return Optional.empty(); + } + }, "expectError()"); + this.script.add(event); + return build(); + + } + + @Override + public DefaultStepVerifier expectError(Class clazz) { + Objects.requireNonNull(clazz, "clazz"); + SignalEvent event = new SignalEvent<>((signal, se) -> { + if (!signal.isOnError()) { + return fail(se, "expected: onError(%s); actual: %s", + clazz.getSimpleName(), signal); + } + else if (!clazz.isInstance(signal.getThrowable())) { + return fail(se, "expected error of type: %s; actual type: %s", + clazz.getSimpleName(), signal.getThrowable()); + } + else { + return Optional.empty(); + } + }, "expectError(Class)"); + this.script.add(event); + return build(); + } + + @Override + public DefaultStepVerifier expectErrorMessage(String errorMessage) { + SignalEvent event = new SignalEvent<>((signal, se) -> { + if (!signal.isOnError()) { + return fail(se, "expected: onError(\"%s\"); actual: %s", + errorMessage, signal); + } + else if (!Objects.equals(errorMessage, + signal.getThrowable() + .getMessage())) { + return fail(se, "expected error message: \"%s\"; " + "actual " + "message: %s", + errorMessage, + signal.getThrowable() + .getMessage()); + } + else { + return Optional.empty(); + } + }, "expectErrorMessage"); + this.script.add(event); + return build(); + } + + @Override + public DefaultStepVerifier expectErrorMatches(Predicate predicate) { + Objects.requireNonNull(predicate, "predicate"); + SignalEvent event = new SignalEvent<>((signal, se) -> { + if (!signal.isOnError()) { + return fail(se, "expected: onError(); actual: %s", signal); + } + else if (!predicate.test(signal.getThrowable())) { + return fail(se, "predicate failed on exception: %s", signal.getThrowable()); + } + else { + return Optional.empty(); + } + }, "expectErrorMatches"); + this.script.add(event); + return build(); + } + + @Override + public DefaultStepVerifierBuilder expectNoFusionSupport() { + return expectFusion(Fuseable.ANY, Fuseable.NONE); + } + + @Override + public DefaultStepVerifierBuilder expectFusion() { + return expectFusion(Fuseable.ANY, Fuseable.ANY); + } + + @Override + public DefaultStepVerifierBuilder expectFusion(int requested) { + return expectFusion(requested, requested); + } + + @Override + public DefaultStepVerifierBuilder expectFusion(int requested, + int expected) { + checkPositive(requested); + checkPositive(expected); + requestedFusionMode = requested; + expectedFusionMode = expected; + return this; + } + + @Override + public final DefaultStepVerifierBuilder expectNext(T t) { + return addExpectedValues(new Object[] { t }); + } + + @Override + public final DefaultStepVerifierBuilder expectNext(T t1, T t2) { + return addExpectedValues(new Object[] { t1, t2 }); + } + + @Override + public final DefaultStepVerifierBuilder expectNext(T t1, T t2, T t3) { + return addExpectedValues(new Object[] { t1, t2, t3 }); + } + + @Override + public final DefaultStepVerifierBuilder expectNext(T t1, T t2, T t3, T t4) { + return addExpectedValues(new Object[] { t1, t2, t3, t4 }); + } + + @Override + public final DefaultStepVerifierBuilder expectNext(T t1, T t2, T t3, T t4, T t5) { + return addExpectedValues(new Object[] { t1, t2, t3, t4, t5 }); + } + + @Override + public final DefaultStepVerifierBuilder expectNext(T t1, T t2, T t3, T t4, T t5, T t6) { + return addExpectedValues(new Object[] { t1, t2, t3, t4, t5, t6 }); + } + + @Override + @SafeVarargs + public final DefaultStepVerifierBuilder expectNext(T... ts) { + Objects.requireNonNull(ts, "ts"); + Arrays.stream(ts).forEach(this::addExpectedValue); + return this; + } + + @SuppressWarnings("unchecked") // cast to a known type + private DefaultStepVerifierBuilder addExpectedValues(Object[] values) { + Arrays.stream(values).map(val -> (T) val).forEach(this::addExpectedValue); + return this; + } + + private void addExpectedValue(T value) { + String desc = String.format("expectNext(%s)", value); + checkPotentialHang(1, desc); + SignalEvent event = new SignalEvent<>((signal, se) -> { + if (!signal.isOnNext()) { + return fail(se, "expected: onNext(%s); actual: %s", value, signal); + } + else if (!Objects.equals(value, signal.get())) { + return fail(se, "expected value: %s; actual value: %s", value, signal.get()); + } + else { + return Optional.empty(); + } + }, desc); + this.script.add(event); + } + + @Override + public DefaultStepVerifierBuilder expectNextSequence( + Iterable iterable) { + Objects.requireNonNull(iterable, "iterable"); + if (iterable instanceof Collection) { + checkPotentialHang(((Collection) iterable).size(), "expectNextSequence"); + } + else { + //best effort + checkPotentialHang(-1, "expectNextSequence"); + } + this.script.add(new SignalSequenceEvent<>(iterable, "expectNextSequence")); + + return this; + } + + @Override + public DefaultStepVerifierBuilder expectNextCount(long count) { + checkPositive(count); + String desc = "expectNextCount(" + count + ")"; + checkPotentialHang(count, desc); + this.script.add(new SignalCountEvent<>(count, desc)); + return this; + } + + @Override + public DefaultStepVerifierBuilder expectNextMatches( + Predicate predicate) { + Objects.requireNonNull(predicate, "predicate"); + checkPotentialHang(1, "expectNextMatches"); + SignalEvent event = new SignalEvent<>((signal, se) -> { + if (!signal.isOnNext()) { + return fail(se, "expected: onNext(); actual: %s", signal); + } + else if (!predicate.test(signal.get())) { + return fail(se, "predicate failed on value: %s", signal.get()); + } + else { + return Optional.empty(); + } + }, "expectNextMatches"); + this.script.add(event); + return this; + } + + @Override + public DefaultStepVerifierBuilder expectRecordedMatches( + Predicate> predicate) { + Objects.requireNonNull(predicate, "predicate"); + this.script.add(new CollectEvent<>(predicate, "expectRecordedMatches")); + return this; + } + + @Override + public DefaultStepVerifierBuilder expectSubscription() { + if(this.script.get(0) instanceof NoEvent) { + this.script.add(defaultFirstStep()); + } + else{ + this.script.set(0, newOnSubscribeStep("expectSubscription")); + } + return this; + } + + @Override + public DefaultStepVerifierBuilder expectSubscriptionMatches( + Predicate predicate) { + Objects.requireNonNull(predicate, "predicate"); + this.script.set(0, new SignalEvent<>((signal, se) -> { + if (!signal.isOnSubscribe()) { + return fail(se, "expected: onSubscribe(); actual: %s", signal); + } + else if (!predicate.test(signal.getSubscription())) { + return fail(se, "predicate failed on subscription: %s", + signal.getSubscription()); + } + else { + return Optional.empty(); + } + }, "expectSubscriptionMatches")); + return this; + } + + @Override + public DefaultStepVerifierBuilder expectNoEvent(Duration duration) { + Objects.requireNonNull(duration, "duration"); + if(this.script.size() == 1 && this.script.get(0) == defaultFirstStep()){ + this.script.set(0, new NoEvent<>(duration, "expectNoEvent")); + } + else { + this.script.add(new NoEvent<>(duration, "expectNoEvent")); + } + return this; + } + + @Override + public DefaultStepVerifierBuilder recordWith(Supplier> supplier) { + Objects.requireNonNull(supplier, "supplier"); + this.script.add(new CollectEvent<>(supplier, "recordWith")); + return this; + } + + @Override + public DefaultStepVerifierBuilder then(Runnable task) { + Objects.requireNonNull(task, "task"); + this.script.add(new TaskEvent<>(task, "then")); + return this; + } + + @Override + public DefaultStepVerifier thenCancel() { + this.script.add(new SubscriptionEvent<>("thenCancel")); + return build(); + } + + @Override + public Duration verifyError() { + return expectError().verify(); + } + + @Override + public Duration verifyError(Class clazz) { + return expectError(clazz).verify(); + } + + @Override + public Duration verifyErrorMessage(String errorMessage) { + return expectErrorMessage(errorMessage).verify(); + } + + @Override + public Duration verifyErrorMatches(Predicate predicate) { + return expectErrorMatches(predicate).verify(); + } + + @Override + public Duration verifyComplete() { + return expectComplete().verify(); + } + + @Override + public DefaultStepVerifierBuilder thenRequest(long n) { + checkStrictlyPositive(n); + this.script.add(new RequestEvent(n, "thenRequest")); + this.hangCheckRequested = Operators.addCap(hangCheckRequested, n); + return this; + } + + @Override + public DefaultStepVerifierBuilder thenAwait() { + return thenAwait(Duration.ZERO); + } + + @Override + public DefaultStepVerifierBuilder thenAwait(Duration timeshift) { + Objects.requireNonNull(timeshift, "timeshift"); + this.script.add(new WaitEvent<>(timeshift, "thenAwait")); + return this; + } + + @Override + public DefaultStepVerifierBuilder thenConsumeWhile(Predicate predicate) { + return thenConsumeWhile(predicate, t -> {}); + } + + @Override + public DefaultStepVerifierBuilder thenConsumeWhile(Predicate predicate, + Consumer consumer) { + Objects.requireNonNull(predicate, "predicate"); + checkPotentialHang(-1, "thenConsumeWhile"); + this.script.add(new SignalConsumeWhileEvent<>(predicate, consumer, "thenConsumeWhile")); + return this; + } + + private void checkPotentialHang(long expectedAmount, String stepDescription) { + if (!options.isCheckUnderRequesting()) { + return; + } + + boolean bestEffort = false; + if (expectedAmount == -1) { + bestEffort = true; + expectedAmount = 1; + } + if (this.hangCheckRequested < expectedAmount) { + StringBuilder message = new StringBuilder() + .append("The scenario will hang at ") + .append(stepDescription) + .append(" due to too little request being performed for the expectations to finish; ") + .append("request remaining since last step: ") + .append(hangCheckRequested) + .append(", expected: "); + if (bestEffort) { + message.append("at least ") + .append(expectedAmount) + .append(" (best effort estimation)"); + } else { + message.append(expectedAmount); + } + throw new IllegalArgumentException(message.toString()); + } + else { + this.hangCheckRequested -= expectedAmount; + } + } + + final DefaultStepVerifier build() { + return new DefaultStepVerifier<>(this); + } + + interface Event { + + boolean setDescription(String description); + + String getDescription(); + + } + + final static class DefaultStepVerifier implements StepVerifier { + + private final DefaultStepVerifierBuilder parent; + private final int requestedFusionMode; + private final int expectedFusionMode; + + private boolean debugEnabled; + + DefaultStepVerifier(DefaultStepVerifierBuilder parent) { + this.parent = parent; + this.requestedFusionMode = parent.requestedFusionMode; + this.expectedFusionMode = parent.expectedFusionMode == -1 ? parent.requestedFusionMode : parent.expectedFusionMode; + } + + @Override + public DefaultStepVerifier log() { + this.debugEnabled = true; + return this; + } + + @Override + public Assertions verifyThenAssertThat() { + //plug in the correct hooks + Queue droppedElements = new ConcurrentLinkedQueue<>(); + Queue droppedErrors = new ConcurrentLinkedQueue<>(); + Queue> operatorErrors = new ConcurrentLinkedQueue<>(); + Hooks.onErrorDropped(droppedErrors::offer); + Hooks.onNextDropped(droppedElements::offer); + Hooks.onOperatorError((t, d) -> { + operatorErrors.offer(Tuples.of(t, d)); + return t; + }); + + try { + //trigger the verify + Duration time = verify(); + + //return the assertion API + return new DefaultStepVerifierAssertions(droppedElements, droppedErrors, operatorErrors, time); + } + finally { + //unplug the hooks + Hooks.resetOnNextDropped(); + Hooks.resetOnErrorDropped(); + Hooks.resetOnOperatorError(); + } + } + + @Override + public Duration verify() { + return verify(Duration.ZERO); + } + + @Override + public Duration verify(Duration duration) { + Objects.requireNonNull(duration, "duration"); + if (parent.sourceSupplier != null) { + VirtualTimeScheduler vts = null; + if (parent.vtsLookup != null) { + vts = parent.vtsLookup.get(); + //this works even for the default case where StepVerifier has created + // a vts through enable(false), because the CURRENT will already be that vts + VirtualTimeScheduler.set(vts); + } + try { + Publisher publisher = parent.sourceSupplier.get(); + Instant now = Instant.now(); + + DefaultVerifySubscriber newVerifier = new DefaultVerifySubscriber<>( + this.parent.script, + this.parent.initialRequest, + this.requestedFusionMode, + this.expectedFusionMode, + this.debugEnabled, + vts); + + publisher.subscribe(newVerifier); + newVerifier.verify(duration); + + return Duration.between(now, Instant.now()); + } + finally { + if (vts != null) { + vts.dispose(); + //explicitly reset the factory, rather than rely on vts shutdown doing so + // because it could have been eagerly shut down in a test. + VirtualTimeScheduler.reset(); + } + } + } else { + return toSubscriber().verify(duration); + } + } + + /** + * Converts the {@link StepVerifier} to a {@link Subscriber}, leaving all the + * lifecycle management to the user. Most notably: + *
    + *
  • no subscription is performed + *
  • no {@link VirtualTimeScheduler} is registered in the Schedulers factories + *
+ *

+ * However if a {@link VirtualTimeScheduler} supplier was passed in originally + * it will be invoked and the resulting scheduler will be affected by time + * manipulation methods. That scheduler can be retrieved from the subscriber's + * {@link DefaultVerifySubscriber#virtualTimeScheduler() virtualTimeScheduler()} + * method. + */ + DefaultVerifySubscriber toSubscriber() { + VirtualTimeScheduler vts = null; + if (parent.vtsLookup != null) { + vts = parent.vtsLookup.get(); + } + return new DefaultVerifySubscriber<>( + this.parent.script, + this.parent.initialRequest, + this.requestedFusionMode, + this.expectedFusionMode, + this.debugEnabled, + vts); + } + + } + + final static class DefaultVerifySubscriber + extends AtomicReference + implements StepVerifier, Subscriber { + + final CountDownLatch completeLatch; + final Queue> script; + final Queue> taskEvents; + final int requestedFusionMode; + final int expectedFusionMode; + final long initialRequest; + final VirtualTimeScheduler virtualTimeScheduler; + + Logger logger; + int establishedFusionMode; + Fuseable.QueueSubscription qs; + long produced; //used for request tracking + long unasserted; //used for expectNextXXX tracking + volatile long requested; + volatile boolean done; // async fusion + Iterator currentNextAs; + Collection currentCollector; + + static final AtomicLongFieldUpdater REQUESTED = + AtomicLongFieldUpdater.newUpdater(DefaultVerifySubscriber.class, "requested"); + + @SuppressWarnings("unused") + volatile int wip; + + @SuppressWarnings("unused") + volatile Throwable errors; + + volatile boolean monitorSignal; + + /** The constructor used for verification, where a VirtualTimeScheduler can be + * passed */ + @SuppressWarnings("unchecked") + DefaultVerifySubscriber(List> script, + long initialRequest, + int requestedFusionMode, + int expectedFusionMode, + boolean debugEnabled, + VirtualTimeScheduler vts) { + this.virtualTimeScheduler = vts; + this.requestedFusionMode = requestedFusionMode; + this.expectedFusionMode = expectedFusionMode; + this.initialRequest = initialRequest; + this.logger = debugEnabled ? Loggers.getLogger(StepVerifier.class) : null; + this.script = conflateScript(script, this.logger); + this.taskEvents = new ConcurrentLinkedQueue<>(); + Event event; + for (; ; ) { + event = this.script.peek(); + if (event instanceof TaskEvent) { + taskEvents.add((TaskEvent) this.script.poll()); + } + else { + break; + } + } + this.monitorSignal = taskEvents.peek() instanceof NoEvent; + this.produced = 0L; + this.unasserted = 0L; + this.completeLatch = new CountDownLatch(1); + this.requested = initialRequest; + } + + static Queue> conflateScript(List> script, Logger logger) { + ConcurrentLinkedQueue> queue = new ConcurrentLinkedQueue<>(script); + ConcurrentLinkedQueue> conflated = new ConcurrentLinkedQueue<>(); + + Event event; + while ((event = queue.peek()) != null) { + if (event instanceof TaskEvent) { + conflated.add(queue.poll()); + while (queue.peek() instanceof SubscriptionEvent) { + conflated.add(new SubscriptionTaskEvent<>((SubscriptionEvent) queue.poll())); + } + } else { + conflated.add(queue.poll()); + } + } + + Iterator> iterator = conflated.iterator(); + Event previous = null; + while(iterator.hasNext()) { + Event current = iterator.next(); + if (previous != null && current instanceof DescriptionEvent) { + String newDescription = current.getDescription(); + String oldDescription = previous.getDescription(); + boolean applied = previous.setDescription(newDescription); + if (logger != null && applied) { + logger.debug("expectation <{}> now described as <{}>", + oldDescription, newDescription); + } + } + previous = current; + } + + queue.clear(); + queue.addAll(conflated.stream() + .filter(ev -> !(ev instanceof DescriptionEvent)) + .collect(Collectors.toList())); + conflated = queue; + + if (logger != null) { + logger.debug("Scenario:"); + for (Event current : conflated) { + logger.debug("\t<{}>", current.getDescription()); + } + } + //TODO simplified whole algo, remove DescriptionTasks + + return conflated; + } + + /** + * @return the {@link VirtualTimeScheduler} this verifier will manipulate when + * using {@link #thenAwait(Duration)} methods, or null if real time is used + */ + VirtualTimeScheduler virtualTimeScheduler() { + return this.virtualTimeScheduler; + } + + boolean isCancelled() { + return get() == Operators.cancelledSubscription(); + } + + boolean isTerminated() { + return completeLatch.getCount() == 0L; + } + + @Override + public void onComplete() { + if (establishedFusionMode != Fuseable.ASYNC) { + onExpectation(Signal.complete()); + this.completeLatch.countDown(); + } + else { + done = true; + serializeDrainAndSubscriptionEvent(); + } + } + + @Override + public void onError(Throwable t) { + onExpectation(Signal.error(t)); + this.completeLatch.countDown(); + } + + @Override + public void onNext(T t) { + if (establishedFusionMode == Fuseable.ASYNC) { + serializeDrainAndSubscriptionEvent(); + } + else { + produced++; + unasserted++; + if (currentCollector != null) { + currentCollector.add(t); + } + Signal signal = Signal.next(t); + if (!checkRequestOverflow(signal)) { + onExpectation(signal); + } + } + } + + @Override + public void onSubscribe(Subscription subscription) { + if (subscription == null) { + throw Exceptions.argumentIsNullException(); + } + + if (this.compareAndSet(null, subscription)) { + onExpectation(Signal.subscribe(subscription)); + if (requestedFusionMode >= Fuseable.NONE) { + startFusion(subscription); + } + else if (initialRequest != 0L) { + subscription.request(initialRequest); + } + } + else { + subscription.cancel(); + if (isCancelled()) { + setFailure(null, "an unexpected Subscription has been received: %s; actual: cancelled", + subscription); + } + else { + setFailure(null, "an unexpected Subscription has been received: %s; actual: ", + subscription, + this); + } + } + } + + void drainAsyncLoop(){ + T t; + long r = requested; + for( ; ;) { + boolean d = done; + if (d && qs.isEmpty()) { + if(get() == Operators.cancelledSubscription()){ + return; + } + if(errors != null){ + onExpectation(Signal.complete()); + } + this.completeLatch.countDown(); + return; + } + + if (r == 0L) { + return; + } + long p = 0L; + while (p != r) { + if(get() == Operators.cancelledSubscription()){ + return; + } + try { + t = qs.poll(); + if (t == null) { + break; + } + p++; + produced++; + unasserted++; + } + catch (Throwable e) { + Exceptions.throwIfFatal(e); + cancel(); + onError(Exceptions.unwrap(e)); + return; + } + if (currentCollector != null) { + currentCollector.add(t); + } + Signal signal = Signal.next(t); + if (!checkRequestOverflow(signal)) { + onExpectation(signal); + if (d && qs.isEmpty()) { + if(get() == Operators.cancelledSubscription()){ + return; + } + if(errors != null){ + onExpectation(Signal.complete()); + } + this.completeLatch.countDown(); + return; + } + } + else { + return; + } + } + + if (p != 0) { + r = Operators.addAndGet(REQUESTED, this, -p); + } + + if(r == 0L || qs.isEmpty()){ + break; + } + } + } + + @Override + public DefaultVerifySubscriber log() { + if (this.logger == null) { + this.logger = Loggers.getLogger(StepVerifier.class); + } + return this; + } + + @Override + public Assertions verifyThenAssertThat() { + //plug in the correct hooks + Queue droppedElements = new ConcurrentLinkedQueue<>(); + Queue droppedErrors = new ConcurrentLinkedQueue<>(); + Queue> operatorErrors = new ConcurrentLinkedQueue<>(); + Hooks.onErrorDropped(droppedErrors::offer); + Hooks.onNextDropped(droppedElements::offer); + Hooks.onOperatorError((t, d) -> { + operatorErrors.offer(Tuples.of(t, d)); + return t; + }); + + try { + //trigger the verify + Duration time = verify(); + + //return the assertion API + return new DefaultStepVerifierAssertions(droppedElements, droppedErrors, operatorErrors, time); + } + finally { + //unplug the hooks + Hooks.resetOnNextDropped(); + Hooks.resetOnErrorDropped(); + Hooks.resetOnOperatorError(); + } + } + + @Override + public Duration verify() { + return verify(Duration.ZERO); + } + + @Override + public Duration verify(Duration duration) { + Objects.requireNonNull(duration, "duration"); + Instant now = Instant.now(); + try { + pollTaskEventOrComplete(duration); + } + catch (InterruptedException ex) { + Thread.currentThread() + .interrupt(); + } + validate(); + return Duration.between(now, Instant.now()); + } + + /** + * Signal a failure, always cancelling this subscriber. If the actual signal + * causing the failure might be either onComplete or onError (for which cancel + * is prohibited), prefer using {@link #setFailure(Event, Signal, String, Object...)}. + * + * @param event the event that triggered the failure + * @param msg the message for the error + * @param arguments the optional formatter arguments to the message + */ + final void setFailure(Event event, String msg, Object... arguments) { + setFailure(event, null, msg, arguments); + } + + /** + * Signal a failure, potentially cancelling this subscriber if the actual signal + * is neither onComplete nor onError (for which cancel is prohibited). + * + * @param event the event that triggered the failure + * @param actualSignal the actual signal that triggered the failure (used to + * decide whether or not to cancel, passing null will cause cancellation) + * @param msg the message for the error + * @param arguments the optional formatter arguments to the message + */ + final void setFailure(Event event, Signal actualSignal, String msg, Object... arguments) { + Exceptions.addThrowable(ERRORS, this, fail(event, msg, arguments).get()); + maybeCancel(actualSignal); + this.completeLatch.countDown(); + } + + final void setFailurePrefix(String prefix, Signal actualSignal, String msg, Object... arguments) { + Exceptions.addThrowable(ERRORS, this, failPrefix(prefix, msg, arguments).get()); + maybeCancel(actualSignal); + this.completeLatch.countDown(); + } + + final Subscription cancel() { + Subscription s = + this.getAndSet(Operators.cancelledSubscription()); + if (s != null && s != Operators.cancelledSubscription()) { + s.cancel(); + if(establishedFusionMode == Fuseable.ASYNC) { + qs.clear(); + } + } + return s; + } + + /** Cancels this subscriber if the actual signal is null or not a complete/error */ + final void maybeCancel(Signal actualSignal) { + if (actualSignal == null || (!actualSignal.isOnComplete() && !actualSignal.isOnError())) { + cancel(); + } + } + + final Optional checkCountMismatch(SignalCountEvent event, Signal s) { + long expected = event.count; + if (!s.isOnNext()) { + return fail(event, "expected: count = %s; actual: " + "counted = %s; " + + "signal: %s", + expected, + unasserted, s); + } + else { + return Optional.empty(); + } + } + + /** Returns true if the requested amount was overflown by the given signal */ + final boolean checkRequestOverflow(Signal s) { + long r = requested; + if (!s.isOnNext() + || r < 0 || r == Long.MAX_VALUE //was Long.MAX from beginning or switched to unbounded + || (establishedFusionMode == Fuseable.ASYNC && r != 0L) + || r >= produced) { + return false; + } + else { + //not really an expectation failure so customize the message + setFailurePrefix("request overflow (", s, + "expected production of at most %s; produced: %s; request overflown by signal: %s", r, produced, s); + return true; + } + } + + boolean onCollect(Signal actualSignal) { + Collection c; + CollectEvent collectEvent = (CollectEvent) this.script.poll(); + if (collectEvent.supplier != null) { + c = collectEvent.get(); + this.currentCollector = c; + + if (c == null) { + setFailure(collectEvent, actualSignal, "expected collection; actual supplied is [null]"); + } + return true; + } + c = this.currentCollector; + + if (c == null) { + setFailure(collectEvent, actualSignal, "expected record collector; actual record is [null]"); + return true; + } + + Optional error = collectEvent.test(c); + if (error.isPresent()) { + Exceptions.addThrowable(ERRORS, this, error.get()); + maybeCancel(actualSignal); + this.completeLatch.countDown(); + return true; + } + return true; + } + + @SuppressWarnings("unchecked") + final void onExpectation(Signal actualSignal) { + if (monitorSignal) { + setFailure(null, actualSignal, "expected no event: %s", actualSignal); + return; + } + try { + Event event = this.script.peek(); + if (event == null) { + setFailure(null, actualSignal, "did not expect: %s", actualSignal); + return; + } + + onTaskEvent(); + if (event instanceof DefaultStepVerifierBuilder.SignalConsumeWhileEvent) { + if (consumeWhile(actualSignal, (SignalConsumeWhileEvent) event)) { + return; + } + //possibly re-evaluate the current onNext + event = this.script.peek(); + } + if (event instanceof SignalCountEvent) { + if (onSignalCount(actualSignal, (SignalCountEvent) event)) { + return; + } + } + else if (event instanceof CollectEvent) { + if (onCollect(actualSignal)) { + return; + } + } + else if (event instanceof SignalSequenceEvent) { + if (onSignalSequence(actualSignal, (SignalSequenceEvent) event)) { + return; + } + } + else if (event instanceof SignalEvent) { + if (onSignal(actualSignal)) { + return; + } + } + + event = this.script.peek(); + if (event == null || !(event instanceof EagerEvent)) { + return; + } + + for (; ; ) { + if (event == null || !(event instanceof EagerEvent)) { + break; + } + if (event instanceof SubscriptionEvent) { + if (serializeDrainAndSubscriptionEvent()) { + return; + } + } + else if (event instanceof CollectEvent) { + if (onCollect(actualSignal)) { + return; + } + } + else { + onTaskEvent(); + } + event = this.script.peek(); + } + } + catch (Throwable e) { + Exceptions.throwIfFatal(e); + if(e instanceof AssertionError){ + Exceptions.addThrowable(ERRORS, this, e); + } + else { + String msg = e.getMessage() != null ? e.getMessage() : ""; + AssertionError wrapFailure = fail(null, + "failed running expectation on signal [%s] " + "with " + "[%s]:\n%s", + actualSignal, + Exceptions.unwrap(e) + .getClass() + .getName(), + msg).get(); + wrapFailure.addSuppressed(e); + Exceptions.addThrowable(ERRORS, + this, wrapFailure); + } + maybeCancel(actualSignal); + completeLatch.countDown(); + } + } + + boolean onSignal(Signal actualSignal) { + SignalEvent signalEvent = (SignalEvent) this.script.poll(); + Optional error = signalEvent.test(actualSignal); + if (error.isPresent()) { + Exceptions.addThrowable(ERRORS, this, error.get()); + // #55 ensure the onError is added as a suppressed to the AssertionError + if(actualSignal.isOnError()) { + error.get().addSuppressed(actualSignal.getThrowable()); + } + maybeCancel(actualSignal); + this.completeLatch.countDown(); + return true; + } + if (actualSignal.isOnNext()) { + unasserted--; + } + return false; + } + + boolean onSignalSequence(Signal actualSignal, + SignalSequenceEvent sequenceEvent) { + Iterator currentNextAs = this.currentNextAs; + if (currentNextAs == null) { + currentNextAs = sequenceEvent.iterable.iterator(); + this.currentNextAs = currentNextAs; + } + + Optional error = + sequenceEvent.test(actualSignal, currentNextAs); + + if (error == EXPECT_MORE) { + if (actualSignal.isOnNext()) { + unasserted--; + } + return false; + } + if (!error.isPresent()) { + this.currentNextAs = null; + this.script.poll(); + if (actualSignal.isOnNext()) { + unasserted--; + } + } + else { + Exceptions.addThrowable(ERRORS, this, error.get()); + if(actualSignal.isOnError()) { + // #55 ensure the onError is added as a suppressed to the AssertionError + error.get().addSuppressed(actualSignal.getThrowable()); + } + maybeCancel(actualSignal); + this.completeLatch.countDown(); + return true; + } + return false; + } + + boolean consumeWhile(Signal actualSignal, SignalConsumeWhileEvent whileEvent) { + if (actualSignal.isOnNext()) { + if (whileEvent.test(actualSignal.get())) { + //the value matches, gobble it up + unasserted--; + if (this.logger != null) { + logger.debug("{} consumed {}", whileEvent.getDescription(), actualSignal); + } + return true; + } + } + if (this.logger != null) { + logger.debug("{} stopped at {}", whileEvent.getDescription(), actualSignal); + } + //stop evaluating the predicate + this.script.poll(); + return false; + } + + final boolean onSignalCount(Signal actualSignal, SignalCountEvent event) { + if (unasserted >= event.count) { + this.script.poll(); + unasserted -= event.count; + } + else { + if (event.count != 0) { + Optional error = + this.checkCountMismatch(event, actualSignal); + + if (error.isPresent()) { + Exceptions.addThrowable(ERRORS, this, error.get()); + if(actualSignal.isOnError()) { + // #55 ensure the onError is added as a suppressed to the AssertionError + error.get().addSuppressed(actualSignal.getThrowable()); + } + maybeCancel(actualSignal); + this.completeLatch.countDown(); + } + } + return true; + } + return false; + } + + void onTaskEvent() { + Event event; + for (; ; ) { + if (isCancelled()) { + return; + } + event = this.script.peek(); + if (!(event instanceof TaskEvent)) { + break; + } + event = this.script.poll(); + if (!(event instanceof TaskEvent)) { + return; + } + taskEvents.add((TaskEvent) event); + } + } + + boolean onSubscriptionLoop(){ + SubscriptionEvent subscriptionEvent; + if (this.script.peek() instanceof SubscriptionEvent) { + subscriptionEvent = (SubscriptionEvent) this.script.poll(); + if (subscriptionEvent instanceof RequestEvent) { + updateRequested(subscriptionEvent); + } + if (subscriptionEvent.isTerminal()) { + doCancel(); + return true; + } + subscriptionEvent.consume(get()); + } + return false; + } + + boolean serializeDrainAndSubscriptionEvent() { + int missed = WIP.incrementAndGet(this); + if (missed != 1) { + return true; + } + for (; ; ) { + if(onSubscriptionLoop()){ + return true; + } + if(establishedFusionMode == Fuseable.ASYNC) { + drainAsyncLoop(); + } + missed = WIP.addAndGet(this, -missed); + if (missed == 0) { + break; + } + } + return false; + } + + void doCancel() { + cancel(); + this.completeLatch.countDown(); + } + + @SuppressWarnings("unchecked") + final void pollTaskEventOrComplete(Duration timeout) throws InterruptedException { + Objects.requireNonNull(timeout, "timeout"); + Event event; + Instant stop = Instant.now() + .plus(timeout); + + boolean skip = true; + for (; ; ) { + while ((event = taskEvents.poll()) != null) { + try { + skip = false; + if (event instanceof SubscriptionTaskEvent) { + updateRequested(event); + } + ((TaskEvent) event).run(this); + } + catch (Throwable t) { + Exceptions.throwIfFatal(t); + cancel(); + if(t instanceof AssertionError){ + throw (AssertionError)t; + } + throw Exceptions.propagate(t); + } + } + if (!skip) { + event = script.peek(); + if (event instanceof SubscriptionEvent) { + serializeDrainAndSubscriptionEvent(); + } + } + if (this.completeLatch.await(10, TimeUnit.NANOSECONDS)) { + break; + } + if (timeout != Duration.ZERO && stop.isBefore(Instant.now())) { + if (get() == null) { + throw new IllegalStateException( + "VerifySubscriber has not been subscribed"); + } + else { + throw new AssertionError("VerifySubscriber timed out on " + get()); + } + } + } + } + + private void updateRequested(Event event) { + RequestEvent requestEvent = null; + if (event instanceof RequestEvent) requestEvent = (RequestEvent) event; + else if (event instanceof SubscriptionTaskEvent) { + SubscriptionTaskEvent ste = (SubscriptionTaskEvent) event; + if (ste.delegate instanceof RequestEvent) { + requestEvent = (RequestEvent) ste.delegate; + } + } + + if (requestEvent == null) { + return; + } + else if (requestEvent.isBounded()) { + Operators.addAndGet(REQUESTED, this, requestEvent.getRequestAmount()); + + } + else { + REQUESTED.set(this, Long.MAX_VALUE); + } + } + + final void startFusion(Subscription s) { + if (s instanceof Fuseable.QueueSubscription) { + @SuppressWarnings("unchecked") Fuseable.QueueSubscription qs = + (Fuseable.QueueSubscription) s; + + this.qs = qs; + + int m = qs.requestFusion(requestedFusionMode); + if (expectedFusionMode == Fuseable.NONE && m != Fuseable.NONE) { + setFailure(null, + "expected no fusion; actual: %s", + formatFusionMode(m)); + return; + } + if (expectedFusionMode != Fuseable.NONE && m == Fuseable.NONE) { + setFailure(null, + "expected fusion: %s; actual does not support " + "fusion", + formatFusionMode(expectedFusionMode)); + return; + } + if ((m & expectedFusionMode) != m) { + setFailure(null, "expected fusion mode: %s; actual: %s", + formatFusionMode(expectedFusionMode), + formatFusionMode(m)); + return; + } + + this.establishedFusionMode = m; + + if (m == Fuseable.SYNC) { + T v; + for (; ; ) { + if(get() == Operators.cancelledSubscription()){ + return; + } + try { + v = qs.poll(); + } + catch (Throwable e) { + Exceptions.throwIfFatal(e); + cancel(); + onError(Exceptions.unwrap(e)); + return; + } + if (v == null) { + onComplete(); + break; + } + + onNext(v); + } + } + else if (initialRequest != 0) { + s.request(initialRequest); + } + } + else if (expectedFusionMode != Fuseable.NONE) { + setFailure(null, + "expected fuseable source but actual Subscription " + "is " + + "not: %s", + expectedFusionMode, + s); + } + else if (initialRequest != 0L) { + s.request(initialRequest); + } + } + + @SuppressWarnings("unchecked") + final void validate() { + if (get() == null) { + throw new IllegalStateException( + "VerifySubscriber has not been subscribed"); + } + Throwable errors = this.errors; + + if (errors == null) { + return; + } + + if(errors instanceof AssertionError){ + throw (AssertionError)errors; + } + + List flat = new ArrayList<>(); + flat.add(errors); + flat.addAll(Arrays.asList(errors.getSuppressed())); + + StringBuilder messageBuilder = new StringBuilder("Expectation failure(s):\n"); + flat.stream() + .flatMap(error -> Stream.of(" - ", error, "\n")) + .forEach(messageBuilder::append); + + messageBuilder.delete(messageBuilder.length() - 1, messageBuilder.length()); + throw new AssertionError(messageBuilder.toString(), errors); + } + + } + + static class DefaultStepVerifierAssertions implements StepVerifier.Assertions { + + private final Queue droppedElements; + private final Queue droppedErrors; + private final Queue> operatorErrors; + private final Duration duration; + + DefaultStepVerifierAssertions(Queue droppedElements, + Queue droppedErrors, + Queue> operatorErrors, + Duration duration) { + this.droppedElements = droppedElements; + this.droppedErrors = droppedErrors; + this.operatorErrors = operatorErrors; + this.duration = duration; + } + + private StepVerifier.Assertions satisfies(BooleanSupplier check, Supplier message) { + if (!check.getAsBoolean()) { + throw new AssertionError(message.get()); + } + return this; + } + + @Override + public StepVerifier.Assertions hasDroppedElements() { + return satisfies(() -> !droppedElements.isEmpty(), () -> "Expected dropped elements, none found."); + } + + @Override + public StepVerifier.Assertions hasDropped(Object... values) { + satisfies(() -> values != null && values.length > 0, () -> "Require non-empty values"); + List valuesList = Arrays.asList(values); + return satisfies(() -> droppedElements.containsAll(valuesList), + () -> String.format("Expected dropped elements to contain <%s>, was <%s>.", valuesList, droppedElements)); + } + + @Override + public StepVerifier.Assertions hasDroppedExactly(Object... values) { + satisfies(() -> values != null && values.length > 0, () -> "Require non-empty values"); + List valuesList = Arrays.asList(values); + return satisfies(() -> droppedElements.containsAll(valuesList) + && droppedElements.size() == valuesList.size(), + () -> String.format("Expected dropped elements to contain exactly <%s>, was <%s>.", valuesList, droppedElements)); + } + + @Override + public StepVerifier.Assertions hasDroppedErrors() { + return satisfies(() -> !droppedErrors.isEmpty(), + () -> "Expected at least 1 dropped error, none found."); + } + @Override + public StepVerifier.Assertions hasDroppedErrors(int size) { + return satisfies(() -> droppedErrors.size() == size, + () -> String.format("Expected exactly %d dropped errors, %d found.", size, droppedErrors.size())); + } + + @Override + public StepVerifier.Assertions hasDroppedErrorOfType(Class clazz) { + satisfies(() -> clazz != null, () -> "Require non-null clazz"); + hasDroppedErrors(1); + return satisfies(() -> clazz.isInstance(droppedErrors.peek()), + () -> String.format("Expected dropped error to be of type %s, was %s.", clazz.getCanonicalName(), droppedErrors.peek().getClass().getCanonicalName())); + } + + @Override + public StepVerifier.Assertions hasDroppedErrorMatching(Predicate matcher) { + satisfies(() -> matcher != null, () -> "Require non-null matcher"); + hasDroppedErrors(1); + return satisfies(() -> matcher.test(droppedErrors.peek()), + () -> String.format("Expected dropped error matching the given predicate, did not match: <%s>.", droppedErrors.peek())); + } + + @Override + public StepVerifier.Assertions hasDroppedErrorWithMessage(String message) { + satisfies(() -> message != null, () -> "Require non-null message"); + hasDroppedErrors(1); + String actual = droppedErrors.peek().getMessage(); + return satisfies(() -> message.equals(actual), + () -> String.format("Expected dropped error with message <\"%s\">, was <\"%s\">.", message, actual)); + } + + @Override + public StepVerifier.Assertions hasDroppedErrorWithMessageContaining( + String messagePart) { + satisfies(() -> messagePart != null, () -> "Require non-null messagePart"); + hasDroppedErrors(1); + String actual = droppedErrors.peek().getMessage(); + return satisfies(() -> actual != null && actual.contains(messagePart), + () -> String.format("Expected dropped error with message containing <\"%s\">, was <\"%s\">.", messagePart, actual)); + } + + @Override + public StepVerifier.Assertions hasDroppedErrorsMatching(Predicate> matcher) { + satisfies(() -> matcher != null, () -> "Require non-null matcher"); + hasDroppedErrors(); + return satisfies(() -> matcher.test(droppedErrors), + () -> String.format("Expected collection of dropped errors matching the given predicate, did not match: <%s>.", droppedErrors)); + } + + @Override + public StepVerifier.Assertions hasDroppedErrorsSatisfying(Consumer> asserter) { + satisfies(() -> asserter != null, () -> "Require non-null asserter"); + hasDroppedErrors(); + asserter.accept(droppedErrors); + return this; + } + + @Override + public StepVerifier.Assertions hasOperatorErrors() { + return satisfies(() -> !operatorErrors.isEmpty(), + () -> "Expected at least 1 operator error, none found."); + } + @Override + public StepVerifier.Assertions hasOperatorErrors(int size) { + return satisfies(() -> operatorErrors.size() == size, + () -> String.format("Expected exactly %d operator errors, %d found.", size, operatorErrors.size())); + } + + StepVerifier.Assertions hasOneOperatorErrorWithError() { + satisfies(() -> operatorErrors.size() == 1, + () -> String.format("Expected exactly one operator error, %d found.", operatorErrors.size())); + satisfies(() -> operatorErrors.peek().getT1() != null, + () -> "Expected exactly one operator error with an actual throwable content, no throwable found."); + return this; + } + + @Override + public StepVerifier.Assertions hasOperatorErrorOfType(Class clazz) { + satisfies(() -> clazz != null, () -> "Require non-null clazz"); + hasOneOperatorErrorWithError(); + return satisfies(() -> clazz.isInstance(operatorErrors.peek().getT1()), + () -> String.format("Expected operator error to be of type %s, was %s.", + clazz.getCanonicalName(), operatorErrors.peek().getT1().getClass().getCanonicalName())); + } + + @Override + public StepVerifier.Assertions hasOperatorErrorMatching(Predicate matcher) { + satisfies(() -> matcher != null, () -> "Require non-null matcher"); + hasOneOperatorErrorWithError(); + return satisfies(() -> matcher.test(operatorErrors.peek().getT1()), + () -> String.format("Expected operator error matching the given predicate, did not match: <%s>.", operatorErrors.peek())); + } + + @Override + public StepVerifier.Assertions hasOperatorErrorWithMessage(String message) { + satisfies(() -> message != null, () -> "Require non-null message"); + hasOneOperatorErrorWithError(); + String actual = operatorErrors.peek().getT1().getMessage(); + return satisfies(() -> message.equals(actual), + () -> String.format("Expected operator error with message <\"%s\">, was <\"%s\">.", message, actual)); + } + + @Override + public StepVerifier.Assertions hasOperatorErrorWithMessageContaining( + String messagePart) { + satisfies(() -> messagePart != null, () -> "Require non-null messagePart"); + hasOneOperatorErrorWithError(); + String actual = operatorErrors.peek().getT1().getMessage(); + return satisfies(() -> actual != null && actual.contains(messagePart), + () -> String.format("Expected operator error with message containing <\"%s\">, was <\"%s\">.", messagePart, actual)); + } + + @Override + public StepVerifier.Assertions hasOperatorErrorsMatching(Predicate>> matcher) { + satisfies(() -> matcher != null, () -> "Require non-null matcher"); + hasOperatorErrors(); + return satisfies(() -> matcher.test(operatorErrors), + () -> String.format("Expected collection of operator errors matching the given predicate, did not match: <%s>.", operatorErrors)); + } + + @Override + public StepVerifier.Assertions hasOperatorErrorsSatisfying(Consumer>> asserter) { + satisfies(() -> asserter != null, () -> "Require non-null asserter"); + hasOperatorErrors(); + asserter.accept(operatorErrors); + return this; + } + + @Override + public StepVerifier.Assertions tookLessThan(Duration d) { + return satisfies(() -> duration.compareTo(d) <= 0, + () -> String.format("Expected scenario to be verified in less than %sms, took %sms.", + d.toMillis(), duration.toMillis())); + } + + @Override + public StepVerifier.Assertions tookMoreThan(Duration d) { + return satisfies(() -> duration.compareTo(d) >= 0, + () -> String.format("Expected scenario to be verified in more than %sms, took %sms.", + d.toMillis(), duration.toMillis())); + } + } + + + + interface EagerEvent extends Event { + + } + + static abstract class AbstractEagerEvent implements EagerEvent { + + String description = ""; + + public AbstractEagerEvent(String description) { + this.description = description; + } + + public boolean setDescription(String description) { + this.description = description; + return true; + } + + public String getDescription() { + return description; + } + } + + static class SubscriptionEvent extends AbstractEagerEvent { + + final Consumer consumer; + + SubscriptionEvent(String desc) { + this(null, desc); + } + + SubscriptionEvent(Consumer consumer, String desc) { + super(desc); + this.consumer = consumer; + } + + void consume(Subscription subscription) { + if (consumer != null) { + this.consumer.accept(subscription); + } + } + + boolean isTerminal() { + return consumer == null; + } + } + + static final class RequestEvent extends SubscriptionEvent { + + final long requestAmount; + + RequestEvent(long n, String desc) { + super(s -> s.request(n), desc); + this.requestAmount = n; + } + + public long getRequestAmount() { + return requestAmount; + } + + public boolean isBounded() { + return requestAmount >= 0 && requestAmount < Long.MAX_VALUE; + } + } + + static abstract class AbstractSignalEvent implements Event { + + String description; + + public AbstractSignalEvent(String description) { + this.description = description; + } + + public boolean setDescription(String description) { + this.description = description; + return true; + } + + public String getDescription() { + return description; + } + } + + static final class SignalEvent extends AbstractSignalEvent { + + final BiFunction, SignalEvent, Optional> function; + + SignalEvent(BiFunction, SignalEvent, Optional> function, + String desc) { + super(desc); + this.function = function; + } + + Optional test(Signal signal) { + return this.function.apply(signal, this); + } + + } + + static final class SignalCountEvent extends AbstractSignalEvent { + + final long count; + + SignalCountEvent(long count, String desc) { + super(desc); + this.count = count; + } + + } + + static final class CollectEvent extends AbstractEagerEvent { + + final Supplier> supplier; + + final Predicate> predicate; + + final Consumer> consumer; + + CollectEvent(Supplier> supplier, String desc) { + super(desc); + this.supplier = supplier; + this.predicate = null; + this.consumer = null; + } + + CollectEvent(Consumer> consumer, String desc) { + super(desc); + this.supplier = null; + this.predicate = null; + this.consumer = consumer; + } + + CollectEvent(Predicate> predicate, String desc) { + super(desc); + this.supplier = null; + this.predicate = predicate; + this.consumer = null; + } + + Collection get() { + return supplier != null ? supplier.get() : null; + } + + Optional test(Collection collection) { + if (predicate != null) { + if (!predicate.test(collection)) { + return fail(this, "expected collection predicate match; actual: %s", + collection); + } + else { + return Optional.empty(); + } + } + else if (consumer != null) { + consumer.accept(collection); + } + return Optional.empty(); + } + + } + + static class TaskEvent extends AbstractEagerEvent { + + final Runnable task; + + TaskEvent(Runnable task, String desc) { + super(desc); + this.task = task; + } + + void run(DefaultVerifySubscriber parent) throws Exception { + task.run(); + } + } + + static final class SubscriptionConsumerEvent extends TaskEvent { + + final Consumer task; + + SubscriptionConsumerEvent(Consumer task, String desc) { + super(null, desc); + this.task = task; + } + + @Override + void run(DefaultVerifySubscriber parent) throws Exception { + task.accept(parent.get()); + } + } + + static void virtualOrRealWait(Duration duration, DefaultVerifySubscriber s) + throws Exception { + if (s.virtualTimeScheduler == null) { + s.completeLatch.await(duration.toMillis(), TimeUnit.MILLISECONDS); + } + else { + s.virtualTimeScheduler.advanceTimeBy(duration); + } + } + + static final class NoEvent extends TaskEvent { + + final Duration duration; + + NoEvent(Duration duration, String desc) { + super(null, desc); + this.duration = duration; + } + + @Override + void run(DefaultVerifySubscriber parent) throws Exception { + if(parent.virtualTimeScheduler != null) { + parent.monitorSignal = true; + virtualOrRealWait(duration.minus(Duration.ofNanos(1)), parent); + parent.monitorSignal = false; + if(parent.isTerminated() && !parent.isCancelled()){ + throw new AssertionError("unexpected end during a no-event expectation"); + } + virtualOrRealWait(Duration.ofNanos(1), parent); + } + else{ + parent.monitorSignal = true; + virtualOrRealWait(duration, parent); + parent.monitorSignal = false; + if(parent.isTerminated() && !parent.isCancelled()){ + throw new AssertionError("unexpected end during a no-event expectation"); + } + } + } + } + + static final class WaitEvent extends TaskEvent { + + final Duration duration; + + WaitEvent(Duration duration, String desc) { + super(null, desc); + this.duration = duration; + } + + @Override + void run(DefaultVerifySubscriber s) throws Exception { + virtualOrRealWait(duration, s); + } + + } + + /** + * A lazy cancellation task that will only trigger cancellation after all previous + * tasks have been processed (avoiding short-circuiting of time manipulating tasks). + */ + static final class SubscriptionTaskEvent extends TaskEvent { + + final SubscriptionEvent delegate; + + SubscriptionTaskEvent(SubscriptionEvent subscriptionEvent) { + super(null, subscriptionEvent.getDescription()); + this.delegate = subscriptionEvent; + } + + @Override + void run(DefaultVerifySubscriber parent) throws Exception { + if (delegate.isTerminal()) { + parent.doCancel(); + } else { + delegate.consume(parent.get()); + } + } + } + + static final class SignalSequenceEvent extends AbstractSignalEvent { + + final Iterable iterable; + + SignalSequenceEvent(Iterable iterable, String desc) { + super(desc); + this.iterable = iterable; + } + + Optional test(Signal signal, Iterator iterator) { + if (signal.isOnNext()) { + if (!iterator.hasNext()) { + return Optional.empty(); + } + T d2 = iterator.next(); + if (!Objects.equals(signal.get(), d2)) { + return fail(this, "expected : onNext(%s); actual: %s; iterable: %s", + d2, + signal.get(), + iterable); + } + return iterator.hasNext() ? EXPECT_MORE : Optional.empty(); + + } + if (iterator != null && iterator.hasNext() || signal.isOnError()) { + return fail(this, "expected next value: %s; actual signal: %s; iterable: %s", + iterator != null && iterator.hasNext() ? iterator.next() : "none", + signal, iterable); + } + return Optional.empty(); + } + } + + static final class SignalConsumeWhileEvent extends AbstractSignalEvent { + + private final Predicate predicate; + private final Consumer consumer; + + SignalConsumeWhileEvent(Predicate predicate, Consumer consumer, String desc) { + super(desc); + this.predicate = predicate; + this.consumer = consumer; + } + + boolean test(T actual) { + if (predicate.test(actual)) { + consumer.accept(actual); + return true; + } + return false; + } + } + + static final class DescriptionEvent implements Event { + + final String description; + + public DescriptionEvent(String description) { + this.description = description; + } + + @Override + public boolean setDescription(String description) { + //NO OP + return false; + } + + @Override + public String getDescription() { + return description; + } + } + + static Optional fail(Event event, String msg, Object... args) { + String prefix = "expectation failed ("; + if (event != null && event.getDescription() != null) { + prefix = String.format("expectation \"%s\" failed (", event.getDescription()); + } + + return failPrefix(prefix, msg, args); + } + + static Optional failPrefix(String prefix, String msg, Object... args) { + return Optional.of(new AssertionError(prefix + String.format(msg, args) + ")")); + } + + static String formatFusionMode(int m) { + switch (m) { + case Fuseable.ANY: + return "(any)"; + case Fuseable.SYNC: + return "(sync)"; + case Fuseable.ASYNC: + return "(async)"; + case Fuseable.NONE: + return "none"; + case Fuseable.THREAD_BARRIER: + return "(thread-barrier)"; + } + return "" + m; + } + + static SignalEvent newOnSubscribeStep(String desc){ + return new SignalEvent<>((signal, se) -> { + if (!signal.isOnSubscribe()) { + return fail(se, "expected: onSubscribe(); actual: %s", signal); + } + else { + return Optional.empty(); + } + }, desc); + } + + static final SignalEvent DEFAULT_ONSUBSCRIBE_STEP = newOnSubscribeStep("defaultOnSubscribe"); + + static final AtomicReferenceFieldUpdater + ERRORS = + AtomicReferenceFieldUpdater.newUpdater(DefaultVerifySubscriber.class, + Throwable.class, + "errors"); + + static final AtomicIntegerFieldUpdater WIP = + AtomicIntegerFieldUpdater.newUpdater(DefaultVerifySubscriber.class, "wip"); + + static final Optional EXPECT_MORE = Optional.of(new AssertionError("EXPECT MORE")); + +} diff --git a/reactor-test/src/main/java/reactor/test/StepVerifier.java b/reactor-test/src/main/java/reactor/test/StepVerifier.java new file mode 100644 index 0000000000..476709017c --- /dev/null +++ b/reactor-test/src/main/java/reactor/test/StepVerifier.java @@ -0,0 +1,994 @@ +/* + * Copyright (c) 2011-2017 Pivotal Software Inc, All Rights Reserved. + * + * 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 + * + * http://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 reactor.test; + +import java.time.Duration; +import java.util.Collection; +import java.util.List; +import java.util.Objects; +import java.util.function.BiFunction; +import java.util.function.Consumer; +import java.util.function.Predicate; +import java.util.function.Supplier; + +import org.reactivestreams.Publisher; +import org.reactivestreams.Subscriber; +import org.reactivestreams.Subscription; +import reactor.core.Fuseable; +import reactor.core.publisher.Hooks; +import reactor.core.scheduler.Schedulers; +import reactor.test.scheduler.VirtualTimeScheduler; +import reactor.util.function.Tuple2; + +/** + * A {@link StepVerifier} provides a declarative way of creating a verifiable script for + * an async {@link Publisher} sequence, by expressing expectations about the events that + * will happen upon subscription. The verification must be triggered after the terminal + * expectations (completion, error, cancellation) have been declared, by calling one of + * the {@link #verify()} methods. + * + * + *
  • Create a {@code StepVerifier} around a {@link Publisher} using + * {@link #create create(Publisher)} or + * {@link #withVirtualTime withVirtualTime(Supplier<Publisher>)} + * (in which case you should lazily create the publisher inside the provided + * {@link Supplier lambda}).
  • + *
  • Set up individual value expectations using + * {@link Step#expectNext expectNext}, + * {@link Step#expectNextMatches(Predicate) expectNextMatches(Predicate)}, + * {@link Step#assertNext(Consumer) assertNext(Consumer)}, + * {@link Step#expectNextCount(long) expectNextCount(long)} or + * {@link Step#expectNextSequence(Iterable) expectNextSequence(Iterable)}.
  • + *
  • Trigger subscription actions during the verification using either + * {@link Step#thenRequest(long) thenRequest(long)} or {@link Step#thenCancel() thenCancel()}.
  • + *
  • Finalize the test scenario using a terminal expectation: + * {@link LastStep#expectComplete expectComplete()}, + * {@link LastStep#expectError expectError()}, + * {@link LastStep#expectError(Class) expectError(Class)}, + * {@link LastStep#expectErrorMatches(Predicate) expectErrorMatches(Predicate)}, or + * {@link LastStep#thenCancel thenCancel()}.
  • + *
  • Trigger the verification of the resulting {@code StepVerifier} on its {@code Publisher} + * using either {@link #verify()} or {@link #verify(Duration)}. (note some of the terminal + * expectations above have a "verify" prefixed alternative that both declare the + * expectation and trigger the verification).
  • + *
  • If any expectations failed, an {@link AssertionError} will be thrown indicating the + * failures.
  • + *
+ * + *

For example: + *

+ * StepVerifier.create(Flux.just("foo", "bar"))
+ *   .expectNext("foo")
+ *   .expectNext("bar")
+ *   .expectComplete()
+ *   .verify();
+ * 
+ * + * @author Arjen Poutsma + * @author Stephane Maldini + * @author Simon Baslé + */ +public interface StepVerifier { + + /** + * Prepare a new {@code StepVerifier} in an uncontrolled environment: + * {@link Step#thenAwait} will block in real time. + * Each {@link #verify()} will fully (re)play the scenario. + * + * @param publisher the publisher to subscribe to and verify + * + * @return a builder for expectation declaration and ultimately verification. + */ + static FirstStep create(Publisher publisher) { + return create(publisher, Long.MAX_VALUE); + } + + /** + * Prepare a new {@code StepVerifier} in an uncontrolled environment: + * {@link Step#thenAwait} will block in real time. + * Each {@link #verify()} will fully (re)play the scenario. + * The verification will request a specified amount of values. + * + * @param publisher the publisher to subscribe to and verify + * @param n the amount of items to request + * + * @return a builder for expectation declaration and ultimately verification. + */ + static FirstStep create(Publisher publisher, long n) { + return create(publisher, StepVerifierOptions.create().initialRequest(n)); + } + + /** + * Prepare a new {@code StepVerifier} in an uncontrolled environment: + * {@link Step#thenAwait} will block in real time. + * Each {@link #verify()} will fully (re)play the scenario. + * The verification will request a specified amount of values according to + * the {@link StepVerifierOptions options} passed. + * + * @param publisher the publisher to subscribe to + * @param options the options for the verification + * + * @return a builder for expectation declaration and ultimately verification. + */ + static FirstStep create(Publisher publisher, StepVerifierOptions options) { + return DefaultStepVerifierBuilder.newVerifier(options, () -> publisher); + } + + /** + * Prepare a new {@code StepVerifier} in a controlled environment using + * {@link VirtualTimeScheduler} to manipulate a virtual clock via + * {@link Step#thenAwait}. The scheduler is injected into all {@link Schedulers} factories, + * which means that any operator created within the lambda without a specific scheduler + * will use virtual time. + * Each {@link #verify()} will fully (re)play the scenario. + * The verification will request an unbounded amount of values. + * + * @param scenarioSupplier a mandatory supplier of the {@link Publisher} to subscribe + * to and verify. In order for operators to use virtual time, they must be invoked + * from within the lambda. + * @param the type of the subscriber + * + * @return a builder for expectation declaration and ultimately verification. + */ + static FirstStep withVirtualTime(Supplier> scenarioSupplier) { + return withVirtualTime(scenarioSupplier, Long.MAX_VALUE); + } + + /** + * Prepare a new {@code StepVerifier} in a controlled environment using + * {@link VirtualTimeScheduler} to manipulate a virtual clock via + * {@link Step#thenAwait}. The scheduler is injected into all {@link Schedulers} factories, + * which means that any operator created within the lambda without a specific scheduler + * will use virtual time. + * Each {@link #verify()} will fully (re)play the scenario. + * The verification will request a specified amount of values. + * + * @param scenarioSupplier a mandatory supplier of the {@link Publisher} to subscribe + * to and verify. In order for operators to use virtual time, they must be invoked + * from within the lambda. + * @param n the amount of items to request (must be >= 0) + * @param the type of the subscriber + * + * @return a builder for expectation declaration and ultimately verification. + */ + static FirstStep withVirtualTime(Supplier> scenarioSupplier, + long n) { + return withVirtualTime(scenarioSupplier, () -> VirtualTimeScheduler.getOrSet(), n); + } + + /** + * Prepare a new {@code StepVerifier} in a controlled environment using + * a user-provided {@link VirtualTimeScheduler} to manipulate a virtual clock via + * {@link Step#thenAwait}. The scheduler is injected into all {@link Schedulers} factories, + * which means that any operator created within the lambda without a specific scheduler + * will use virtual time. + * Each {@link #verify()} will fully (re)play the scenario. + * The verification will request a specified amount of values. + * + * @param scenarioSupplier a mandatory supplier of the {@link Publisher} to subscribe + * to and verify. In order for operators to use virtual time, they must be invoked + * from within the lambda. + * @param vtsLookup the supplier of the {@link VirtualTimeScheduler} to inject and + * manipulate during verification. + * @param n the amount of items to request (must be >= 0) + * @param the type of the subscriber + * + * @return a builder for expectation declaration and ultimately verification. + */ + static FirstStep withVirtualTime( + Supplier> scenarioSupplier, + Supplier vtsLookup, + long n) { + return withVirtualTime(scenarioSupplier, StepVerifierOptions.create() + .initialRequest(n) + .virtualTimeSchedulerSupplier(vtsLookup)); + } + + /** + * Prepare a new {@code StepVerifier} in a controlled environment using + * a user-provided {@link VirtualTimeScheduler} to manipulate a virtual clock via + * {@link Step#thenAwait}. The scheduler is injected into all {@link Schedulers} factories, + * which means that any operator created within the lambda without a specific scheduler + * will use virtual time. + * Each {@link #verify()} will fully (re)play the scenario. + * The verification will request a specified amount of values according to + * the provided {@link StepVerifierOptions options}. + * + * @param scenarioSupplier a mandatory supplier of the {@link Publisher} to subscribe + * to and verify. In order for operators to use virtual time, they must be invoked + * from within the lambda. + * @param options the verification options, including the supplier of the + * {@link VirtualTimeScheduler} to inject and manipulate during verification. + * @param the type of the subscriber + * + * @return a builder for expectation declaration and ultimately verification. + */ + static FirstStep withVirtualTime( + Supplier> scenarioSupplier, + StepVerifierOptions options) { + DefaultStepVerifierBuilder.checkPositive(options.getInitialRequest()); + Objects.requireNonNull(options.getVirtualTimeSchedulerSupplier(), "vtsLookup"); + Objects.requireNonNull(scenarioSupplier, "scenarioSupplier"); + + return DefaultStepVerifierBuilder.newVerifier(options, + scenarioSupplier); + } + + /** + * Activate debug logging of a description of the test scenario, as well as + * some details about certain verification steps. + * + * @return the verifier for final {@link #verify()} call + */ + StepVerifier log(); + + /** + * Verify the signals received by this subscriber. This method will + * block until the stream has been terminated (either + * through {@link Subscriber#onComplete()}, {@link Subscriber#onError(Throwable)} or + * {@link Subscription#cancel()}). Depending on the declared expectations and actions, + * notably in case of undersized manual requests, such a verification could also block + * indefinitely. + * + * @return the actual {@link Duration} the verification took. + * @throws AssertionError in case of expectation failures + */ + Duration verify() throws AssertionError; + + /** + * Verify the signals received by this subscriber. This method will + * block for up to the given duration or until the stream has been + * terminated (either through {@link Subscriber#onComplete()}, + * {@link Subscriber#onError(Throwable)} or {@link Subscription#cancel()}). + * + * @return the actual {@link Duration} the verification took. + * @throws AssertionError in case of expectation failures, or when the verification + * times out + */ + Duration verify(Duration duration) throws AssertionError; + + /** + * {@link #verify() Verifies} the signals received by this subscriber, then exposes + * various {@link Assertions assertion methods} on the final state. + *

+ * Note this method will block until the stream has been + * terminated (either through {@link Subscriber#onComplete()}, + * {@link Subscriber#onError(Throwable)} or {@link Subscription#cancel()}). + * Depending on the declared expectations and actions, notably in case of undersized + * manual requests, such a verification could also block indefinitely. + * + * @return the actual {@link Duration} the verification took. + * @throws AssertionError in case of expectation failures + */ + Assertions verifyThenAssertThat(); + + /** + * Define a builder for terminal states. + */ + interface LastStep { + + /** + * Expect an error and consume with the given consumer. Any + * {@code AssertionError}s thrown by the consumer will be rethrown + * during {@linkplain #verify() verification}. + * + * @param consumer the consumer for the exception + * + * @return the built verification + */ + StepVerifier consumeErrorWith(Consumer consumer); + + /** + * Expect an unspecified error. + * + * @return the built verification scenario, ready to be verified + * + * @see Subscriber#onError(Throwable) + */ + StepVerifier expectError(); + + /** + * Expect an error of the specified type. + * + * @param clazz the expected error type + * + * @return the built verification scenario, ready to be verified + * + * @see Subscriber#onError(Throwable) + */ + StepVerifier expectError(Class clazz); + + /** + * Expect an error with the specified message. + * + * @param errorMessage the expected error message + * + * @return the built verification scenario, ready to be verified + * + * @see Subscriber#onError(Throwable) + */ + StepVerifier expectErrorMessage(String errorMessage); + + /** + * Expect an error and evaluate with the given predicate. + * + * @param predicate the predicate to test on the next received error + * + * @return the built verification scenario, ready to be verified + * + * @see Subscriber#onError(Throwable) + */ + StepVerifier expectErrorMatches(Predicate predicate); + + /** + * Expect the completion signal. + * + * @return the built verification scenario, ready to be verified + * + * @see Subscriber#onComplete() + */ + StepVerifier expectComplete(); + + /** + * Cancel the underlying subscription. + * + * @return the built verification scenario, ready to be verified + * + * @see Subscription#cancel() + */ + StepVerifier thenCancel(); + + /** + * Trigger the {@link #verify() verification}, expecting an unspecified error + * as terminal event. + *

+ * This is a convenience method that calls {@link #verify()} in addition to the + * expectation. Explicitly use the expect method and verification method + * separately if you need something more specific (like activating logging or + * putting a timeout). + * + * @return the actual {@link Duration} the verification took. + * + * @see #expectError() + * @see #verify() + * @see Subscriber#onError(Throwable) + */ + Duration verifyError(); + + /** + * Trigger the {@link #verify() verification}, expecting an error of the specified + * type as terminal event. + *

+ * This is a convenience method that calls {@link #verify()} in addition to the + * expectation. Explicitly use the expect method and verification method + * separately if you need something more specific (like activating logging or + * putting a timeout). + * + * @param clazz the expected error type + * @return the actual {@link Duration} the verification took. + * + * @see #expectError(Class) + * @see #verify() + * @see Subscriber#onError(Throwable) + */ + Duration verifyError(Class clazz); + + /** + * Trigger the {@link #verify() verification}, expecting an error with the + * specified message as terminal event. + *

+ * This is a convenience method that calls {@link #verify()} in addition to the + * expectation. Explicitly use the expect method and verification method + * separately if you need something more specific (like activating logging or + * putting a timeout). + * + * @param errorMessage the expected error message + * @return the actual {@link Duration} the verification took. + * + * @see #expectErrorMessage(String) + * @see #verify() + * @see Subscriber#onError(Throwable) + */ + Duration verifyErrorMessage(String errorMessage); + + /** + * Trigger the {@link #verify() verification}, expecting an error that matches + * the given predicate as terminal event. + *

+ * This is a convenience method that calls {@link #verify()} in addition to the + * expectation. Explicitly use the expect method and verification method + * separately if you need something more specific (like activating logging or + * putting a timeout). + * + * @param predicate the predicate to test on the next received error + * @return the actual {@link Duration} the verification took. + * + * @see #expectErrorMatches(Predicate) + * @see #verify() + * @see Subscriber#onError(Throwable) + */ + Duration verifyErrorMatches(Predicate predicate); + + /** + * Trigger the {@link #verify() verification}, expecting a completion signal + * as terminal event. + *

+ * This is a convenience method that calls {@link #verify()} in addition to the + * expectation. Explicitly use the expect method and verification method + * separately if you need something more specific (like activating logging or + * putting a timeout). + * + * @return the actual {@link Duration} the verification took. + * + * @see #expectComplete() + * @see #verify() + * @see Subscriber#onComplete() + * + */ + Duration verifyComplete(); + + } + + /** + * Define a builder for expecting main sequence individual signals. + * + * @param the type of values that the subscriber contains + */ + interface Step extends LastStep { + + /** + * Set a description for the previous verification step. Choosing + * a unique and descriptive name can make assertion errors easier to + * resolve. + *

+ * Note that calling this several times in a row will only take the + * first description into account. + * + * @param description the description for the previous verification step + * @return this builder + */ + Step as(String description); + + /** + * Expect an element and consume with the given consumer.Any {@code + * AssertionError}s thrown by the consumer will be rethrown during {@linkplain + * #verify() verification}. + * + * @param consumer the consumer for the value + * + * @return this builder + */ + Step consumeNextWith(Consumer consumer); + + /** + * Expect an element and consume it with the given consumer, usually performing + * assertions on it (eg. using Hamcrest, AssertJ or JUnit assertion methods). + * Alias for {@link #consumeNextWith(Consumer)} for better discoverability of + * that use case. + *

+ * Any {@code AssertionError}s thrown by the consumer will be rethrown during + * {@linkplain #verify() verification}. + * + * @param assertionConsumer the consumer for the value, performing assertions + * @return this builder + */ + default Step assertNext(Consumer assertionConsumer) { + return consumeNextWith(assertionConsumer); + } + + /** + * Expect a recording session started via {@link #recordWith}, end it and verify + * it by applying the given consumer. + * Any {@code AssertionError}s thrown by the consumer will be rethrown during + * {@linkplain #verify() verification}. + * + * @param consumer the consumer used to apply assertions on the recorded session + * + * @return this builder + */ + Step consumeRecordedWith(Consumer> consumer); + + /** + * Expect the next element received to be equal to the given value. + * + * @param t the value to expect + * + * @return this builder + * + * @see Subscriber#onNext(Object) + */ + Step expectNext(T t); + + /** + * Expect the next elements received to be equal to the given values. + * + * @param t1 the first value to expect + * @param t2 the second value to expect + * + * @return this builder + * + * @see Subscriber#onNext(Object) + */ + Step expectNext(T t1, T t2); + + /** + * Expect the next elements received to be equal to the given values. + * + * @param t1 the first value to expect + * @param t2 the second value to expect + * @param t3 the third value to expect + * + * @return this builder + * + * @see Subscriber#onNext(Object) + */ + Step expectNext(T t1, T t2, T t3); + + /** + * Expect the next elements received to be equal to the given values. + * + * @param t1 the first value to expect + * @param t2 the second value to expect + * @param t3 the third value to expect + * @param t4 the fourth value to expect + * + * @return this builder + * + * @see Subscriber#onNext(Object) + */ + Step expectNext(T t1, T t2, T t3, T t4); + + /** + * Expect the next elements received to be equal to the given values. + * + * @param t1 the first value to expect + * @param t2 the second value to expect + * @param t3 the third value to expect + * @param t4 the fourth value to expect + * @param t5 the fifth value to expect + * + * @return this builder + * + * @see Subscriber#onNext(Object) + */ + Step expectNext(T t1, T t2, T t3, T t4, T t5); + + /** + * Expect the next elements received to be equal to the given values. + * + * @param t1 the first value to expect + * @param t2 the second value to expect + * @param t3 the third value to expect + * @param t4 the fourth value to expect + * @param t5 the fifth value to expect + * @param t6 the sixth value to expect + * + * @return this builder + * + * @see Subscriber#onNext(Object) + */ + Step expectNext(T t1, T t2, T t3, T t4, T t5, T t6); + + /** + * Expect the next elements received to be equal to the given values. + * + * @param ts the values to expect + * + * @return this builder + * + * @see Subscriber#onNext(Object) + */ + Step expectNext(T... ts); + + /** + * Expect to received {@code count} elements, starting from the previous + * expectation or onSubscribe. + * + * @param count the number of emitted items to expect. + * + * @return this builder + * + * @see Subscriber#onNext(Object) + */ + Step expectNextCount(long count); + + /** + * Expect the next elements to match the given {@link Iterable} until its + * iterator depletes. + * + * @param iterable the iterable containing the next expected values + * + * @return this builder + * + * @see Subscriber#onNext(Object) + */ + Step expectNextSequence(Iterable iterable); + + /** + * Expect an element and evaluate with the given predicate. + * + * @param predicate the predicate to test on the next received value + * + * @return this builder + * + * @see Subscriber#onNext(Object) + */ + Step expectNextMatches(Predicate predicate); + + /** + * Expect a {@link Subscription} and consume with the given consumer. Any {@code + * AssertionError}s thrown by the consumer will be rethrown during {@linkplain + * #verify() verification}. + * + * @param consumer the consumer for the {@link Subscription} + * + * @return this builder + * + * @see Subscriber#onSubscribe(Subscription) + */ + Step consumeSubscriptionWith(Consumer consumer); + + /** + * Expect that no event has been observed by the verifier for the length of + * the provided {@link Duration}. If virtual time is used, this duration is + * verified using the virtual clock. + * + * @param duration the duration for which to observe no event has been received + * + * @return this builder + * + * @see Subscriber + */ + Step expectNoEvent(Duration duration); + + /** + * Expect a recording session started via {@link #recordWith}, end it and verify + * it by ensuring the provided predicate matches. + * + * @param predicate the predicate to test on the recorded session + * + * @return this builder + * + * @see Subscriber#onNext(Object) + */ + Step expectRecordedMatches(Predicate> predicate); + + /** + * Start a recording session storing {@link Subscriber#onNext(Object)} values in + * the supplied {@link Collection}. Further steps + * {@link #expectRecordedMatches(Predicate)} and + * {@link #consumeRecordedWith(Consumer)} can consume and assert the session. + *

If an existing recording session hasn't not been declaratively consumed, this step + * will override the current session. + * + * @param supplier the supplier for the {@link Collection} to use for recording. + * + * @return this builder + */ + Step recordWith(Supplier> supplier); + + /** + * Run an arbitrary task scheduled after previous expectations or tasks. + * + * @param task the task to run + * + * @return this builder + */ + Step then(Runnable task); + + /** + * Mark a Pause in the expectation evaluation. + * If a {@link VirtualTimeScheduler} has been configured, + * {@link VirtualTimeScheduler#advanceTime()} will be used and the + * pause will not block testing or {@link Publisher} thread. + * + * @return this builder + */ + default Step thenAwait() { + return thenAwait(Duration.ZERO); + } + + /** + * Pause the expectation evaluation for a given {@link Duration}. + * If a {@link VirtualTimeScheduler} has been configured, + * {@link VirtualTimeScheduler#advanceTimeBy(Duration)} will be used and the + * pause will not block testing or {@link Publisher} thread. + * + * @param timeshift a pause {@link Duration} + * + * @return this builder + */ + Step thenAwait(Duration timeshift); + + /** + * Consume further onNext signals as long as they match a predicate. + * + * @param predicate the condition to continue consuming onNext + * + * @return this builder + */ + Step thenConsumeWhile(Predicate predicate); + + /** + * Consume further onNext signals using a provided {@link Consumer} as long as + * they match a {@link Predicate}. You can use the consumer to apply assertions + * on each value. + * + * @param predicate the condition to continue consuming onNext + * @param consumer the consumer to use to consume the data, when the predicate + * matches + * + * @return this builder + */ + Step thenConsumeWhile(Predicate predicate, Consumer consumer); + + /** + * Request the given amount of elements from the upstream {@code Publisher}. This + * is in addition to the initial number of elements requested by an + * initial passed demand like with {@link StepVerifier#create(Publisher, long)}. + * + * @param n the number of elements to request + * + * @return this builder + * + * @see Subscription#request(long) + */ + Step thenRequest(long n); + } + + /** + * Define a builder for explicitly expecting an initializing {@link Subscription} as + * first signal. + *

+ * If {@link FirstStep} expectations are not used, the produced + * {@link StepVerifier} keeps a first expectation that will be checking if + * the first signal is a + * {@link Subscription}. + * + * @param the type of values that the subscriber contains + */ + interface FirstStep extends Step { + + /** + * Expect the source {@link Publisher} to run with Reactor Fusion flow + * optimization. It will be requesting {@link Fuseable#ANY} fusion mode. + * + * @return this builder + * + * @see Fuseable + */ + Step expectFusion(); + + /** + * Expect the source {@link Publisher} to run the requested Reactor Fusion mode + * from any of these modes : + * {@link Fuseable#NONE}, {@link Fuseable#SYNC}, {@link Fuseable#ASYNC}, + * {@link Fuseable#ANY}, {@link Fuseable#THREAD_BARRIER}. + * + * @param requested the requested and expected fusion mode + * + * @return this builder + * + * @see Fuseable + */ + Step expectFusion(int requested); + + /** + * Expect the source {@link Publisher} to run with Reactor Fusion flow + * optimization. + * Expect the source {@link Publisher} to run the requested Reactor Fusion mode + * from any of these modes : + * {@link Fuseable#NONE}, {@link Fuseable#SYNC}, {@link Fuseable#ASYNC}, + * {@link Fuseable#ANY}, {@link Fuseable#THREAD_BARRIER}. + * + * @param requested the requested fusion mode + * @param expected the expected fusion mode + * + * @return this builder + * + * @see Fuseable + */ + Step expectFusion(int requested, int expected); + + /** + * Expect the source {@link Publisher} to NOT run with Reactor Fusion flow + * optimization. It will check if publisher is {@link Fuseable} or + * subscription is a {@link Fuseable.QueueSubscription}. + * + * @return this builder + * + * @see Fuseable + */ + Step expectNoFusionSupport(); + + /** + * Expect no event and no Subscription has been observed by the verifier for the + * length of the provided {@link Duration}. If virtual time is used, this duration + * is verified using the virtual clock. + *

+ * Note that you should only use this method as the first expectation if you + * actually don't expect a subscription to happen. Use + * {@link FirstStep#expectSubscription()} combined with {@link Step#expectNoEvent(Duration)} + * to work around that. + * + * @param duration the duration for which to observe no event has been received + * + * @return this builder + */ + @Override + FirstStep expectNoEvent(Duration duration); + + /** + * Expect a {@link Subscription}. + * Effectively behave as the default implicit {@link Subscription} expectation. + * + * @return this builder + * + * @see Subscriber#onSubscribe(Subscription) + */ + Step expectSubscription(); + + /** + * Expect a {@link Subscription} and evaluate with the given predicate. + * + * @param predicate the predicate to test on the received {@link Subscription} + * + * @return this builder + * + * @see Subscriber#onSubscribe(Subscription) + */ + Step expectSubscriptionMatches(Predicate predicate); + } + + /** + * Exposes post-verification state assertions. + */ + interface Assertions { + + /** + * Assert that the tested publisher has dropped at least one element to the + * {@link Hooks#onNextDropped(Consumer)} hook. + */ + Assertions hasDroppedElements(); + + /** + * Assert that the tested publisher has dropped at least all of the provided + * elements to the {@link Hooks#onNextDropped(Consumer)} hook, in any order. + */ + Assertions hasDropped(Object... values); + + /** + * Assert that the tested publisher has dropped all of the provided elements to + * the {@link Hooks#onNextDropped(Consumer)} hook, in any order, and that no + * other elements were dropped. + */ + Assertions hasDroppedExactly(Object... values); + + /** + * Assert that the tested publisher has dropped at least one error to the + * {@link Hooks#onErrorDropped(Consumer)} hook. + */ + Assertions hasDroppedErrors(); + + /** + * Assert that the tested publisher has dropped exactly n errors to the + * {@link Hooks#onErrorDropped(Consumer)} hook. + */ + Assertions hasDroppedErrors(int n); + + /** + * Assert that the tested publisher has dropped exactly one error of the given type + * to the {@link Hooks#onErrorDropped(Consumer)} hook. + */ + Assertions hasDroppedErrorOfType(Class clazz); + + /** + * Assert that the tested publisher has dropped exactly one error matching the given + * predicate to the {@link Hooks#onErrorDropped(Consumer)} hook. + */ + Assertions hasDroppedErrorMatching(Predicate matcher); + + /** + * Assert that the tested publisher has dropped exactly one error with the exact provided + * message to the {@link Hooks#onErrorDropped(Consumer)} hook. + */ + Assertions hasDroppedErrorWithMessage(String message); + + /** + * Assert that the tested publisher has dropped exactly one error with a message containing + * the provided string to the {@link Hooks#onErrorDropped(Consumer)} hook. + */ + Assertions hasDroppedErrorWithMessageContaining(String messagePart); + + /** + * Assert that the tested publisher has dropped one or more errors to the + * {@link Hooks#onErrorDropped(Consumer)} hook, and assert them as a collection. + */ + Assertions hasDroppedErrorsSatisfying(Consumer> errorsConsumer); + + /** + * Assert that the tested publisher has dropped one or more errors to the + * {@link Hooks#onErrorDropped(Consumer)} hook, and check that the collection of + * errors matches a predicate. + */ + Assertions hasDroppedErrorsMatching(Predicate> errorsConsumer); + + /** + * Assert that the tested publisher has triggered the {@link Hooks#onOperatorError(BiFunction) onOperatorError} hook + * at least once. + */ + Assertions hasOperatorErrors(); + + /** + * Assert that the tested publisher has triggered the {@link Hooks#onOperatorError(BiFunction) onOperatorError} hook + * exactly n times. + */ + Assertions hasOperatorErrors(int n); + + /** + * Assert that the tested publisher has triggered the {@link Hooks#onOperatorError(BiFunction) onOperatorError} hook + * exactly once and the error is of the given type. + */ + Assertions hasOperatorErrorOfType(Class clazz); + + /** + * Assert that the tested publisher has triggered the {@link Hooks#onOperatorError(BiFunction) onOperatorError} hook + * exactly once and the error matches the given predicate. + */ + Assertions hasOperatorErrorMatching(Predicate matcher); + + /** + * Assert that the tested publisher has triggered the {@link Hooks#onOperatorError(BiFunction) onOperatorError} hook + * exactly once and the error has the exact provided message. + */ + Assertions hasOperatorErrorWithMessage(String message); + + /** + * Assert that the tested publisher has triggered the {@link Hooks#onOperatorError(BiFunction) onOperatorError} hook + * exactly once, with the error message containing the provided string. + */ + Assertions hasOperatorErrorWithMessageContaining(String messagePart); + + /** + * Assert that the tested publisher has triggered the {@link Hooks#onOperatorError(BiFunction) onOperatorError} hook + * once or more, and assert the errors and optionally associated data as a collection. + */ + Assertions hasOperatorErrorsSatisfying(Consumer>> errorsConsumer); + + /** + * Assert that the tested publisher has triggered the {@link Hooks#onOperatorError(BiFunction) onOperatorError} hook + * once or more, and check that the collection of errors and their optionally + * associated data matches a predicate. + */ + Assertions hasOperatorErrorsMatching(Predicate>> errorsConsumer); + + /** + * Assert that the whole verification took strictly less than the provided + * duration to execute. + * @param d the expected maximum duration of the verification + */ + Assertions tookLessThan(Duration d); + + /** + * Assert that the whole verification took strictly more than the provided + * duration to execute. + * @param d the expected minimum duration of the verification + */ + Assertions tookMoreThan(Duration d); + } + +} diff --git a/reactor-test/src/main/java/reactor/test/StepVerifierOptions.java b/reactor-test/src/main/java/reactor/test/StepVerifierOptions.java new file mode 100644 index 0000000000..6cb1812eb1 --- /dev/null +++ b/reactor-test/src/main/java/reactor/test/StepVerifierOptions.java @@ -0,0 +1,104 @@ +/* + * Copyright (c) 2011-2017 Pivotal Software Inc, All Rights Reserved. + * + * 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 + * + * http://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 reactor.test; + +import java.util.function.Supplier; + +import reactor.test.scheduler.VirtualTimeScheduler; + +/** + * Options for a {@link StepVerifier}, including the initial request amount, + * {@link VirtualTimeScheduler} supplier and toggles for some checks. + * + * @author Simon Basle + */ +public class StepVerifierOptions { + + private boolean checkUnderRequesting = true; + private long initialRequest = Long.MAX_VALUE; + private Supplier vtsLookup = null; + + /** + * Create a new default set of options for a {@link StepVerifier} that can be tuned + * using the various available non-getter methods (which can be chained). + */ + public static StepVerifierOptions create() { + return new StepVerifierOptions(); + } + + private StepVerifierOptions() { } //disable constructor + + /** + * Activate or deactivate the {@link StepVerifier} check of request amount + * being too low. Defauts to true. + * + * @param enabled true if the check should be enabled. + * @return this instance, to continue setting the options. + */ + public StepVerifierOptions checkUnderRequesting(boolean enabled) { + this.checkUnderRequesting = enabled; + return this; + } + + /** + * @return true if the {@link StepVerifier} receiving these options should activate + * the check of request amount being too low. + */ + public boolean isCheckUnderRequesting() { + return this.checkUnderRequesting; + } + + /** + * Set the amount the {@link StepVerifier} should request initially. Defaults to + * unbounded request ({@code Long.MAX_VALUE}). + * + * @param initialRequest the initial request amount. + * @return this instance, to continue setting the options. + */ + public StepVerifierOptions initialRequest(long initialRequest) { + this.initialRequest = initialRequest; + return this; + } + + /** + * @return the initial request amount to be made by the {@link StepVerifier} + * receiving these options. + */ + public long getInitialRequest() { + return this.initialRequest; + } + + /** + * Set a supplier for a {@link VirtualTimeScheduler}, which is mandatory for a + * {@link StepVerifier} to work with virtual time. Defaults to null. + * + * @param vtsLookup the supplier of {@link VirtualTimeScheduler} to use. + * @return this instance, to continue setting the options. + */ + public StepVerifierOptions virtualTimeSchedulerSupplier(Supplier vtsLookup) { + this.vtsLookup = vtsLookup; + return this; + } + + /** + * @return the supplier of {@link VirtualTimeScheduler} to be used by the + * {@link StepVerifier} receiving these options. + * + */ + public Supplier getVirtualTimeSchedulerSupplier() { + return vtsLookup; + } +} diff --git a/reactor-test/src/main/java/reactor/test/publisher/DefaultTestPublisher.java b/reactor-test/src/main/java/reactor/test/publisher/DefaultTestPublisher.java new file mode 100644 index 0000000000..01d65bd60d --- /dev/null +++ b/reactor-test/src/main/java/reactor/test/publisher/DefaultTestPublisher.java @@ -0,0 +1,374 @@ +/* + * Copyright (c) 2011-2017 Pivotal Software Inc, All Rights Reserved. + * + * 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 + * + * http://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 reactor.test.publisher; + +import java.util.EnumSet; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.stream.Stream; + +import org.reactivestreams.Subscriber; +import org.reactivestreams.Subscription; +import reactor.core.Fuseable; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; +import reactor.core.publisher.Operators; + +/** + * A default implementation of a {@link TestPublisher}. + * + * @author Simon Basle + * @author Stephane Maldini + */ +class DefaultTestPublisher extends TestPublisher { + + @SuppressWarnings("rawtypes") + private static final TestPublisherSubscription[] EMPTY = new TestPublisherSubscription[0]; + + @SuppressWarnings("rawtypes") + private static final TestPublisherSubscription[] TERMINATED = new TestPublisherSubscription[0]; + + volatile int cancelCount; + + static final AtomicIntegerFieldUpdater CANCEL_COUNT = + AtomicIntegerFieldUpdater.newUpdater(DefaultTestPublisher.class, "cancelCount"); + + Throwable error; + + volatile boolean hasOverflown; + + final EnumSet violations; + + @SuppressWarnings("unchecked") + volatile TestPublisherSubscription[] subscribers = EMPTY; + + @SuppressWarnings("rawtypes") + static final AtomicReferenceFieldUpdater SUBSCRIBERS = + AtomicReferenceFieldUpdater.newUpdater(DefaultTestPublisher.class, TestPublisherSubscription[].class, "subscribers"); + + DefaultTestPublisher(Violation first, Violation... rest) { + this.violations = EnumSet.of(first, rest); + } + + DefaultTestPublisher() { + this.violations = EnumSet.noneOf(Violation.class); + } + + @Override + public void subscribe(Subscriber s) { + Objects.requireNonNull(s, "s"); + + TestPublisherSubscription + p = new TestPublisherSubscription<>(s, this); + s.onSubscribe(p); + + if (add(p)) { + if (p.cancelled) { + remove(p); + } + } else { + Throwable e = error; + if (e != null) { + s.onError(e); + } else { + s.onComplete(); + } + } + } + + boolean add(TestPublisherSubscription s) { + TestPublisherSubscription[] a = subscribers; + if (a == TERMINATED) { + return false; + } + + synchronized (this) { + a = subscribers; + if (a == TERMINATED) { + return false; + } + int len = a.length; + + @SuppressWarnings("unchecked") TestPublisherSubscription[] b = new TestPublisherSubscription[len + 1]; + System.arraycopy(a, 0, b, 0, len); + b[len] = s; + + subscribers = b; + + return true; + } + } + + @SuppressWarnings("unchecked") + void remove(TestPublisherSubscription s) { + TestPublisherSubscription[] a = subscribers; + + if (violations.contains(Violation.CLEANUP_ON_TERMINATE)) { + return; + } + + if (a == TERMINATED || a == EMPTY) { + return; + } + + synchronized (this) { + a = subscribers; + if (a == TERMINATED || a == EMPTY) { + return; + } + int len = a.length; + + int j = -1; + + for (int i = 0; i < len; i++) { + if (a[i] == s) { + j = i; + break; + } + } + if (j < 0) { + return; + } + if (len == 1) { + subscribers = EMPTY; + return; + } + + TestPublisherSubscription[] b = new TestPublisherSubscription[len - 1]; + System.arraycopy(a, 0, b, 0, j); + System.arraycopy(a, j + 1, b, j, len - j - 1); + + subscribers = b; + } + } + + static final class TestPublisherSubscription implements Subscription { + + final Subscriber actual; + final Fuseable.ConditionalSubscriber actualConditional; + + final DefaultTestPublisher parent; + + volatile boolean cancelled; + + volatile long requested; + + @SuppressWarnings("rawtypes") + static final AtomicLongFieldUpdater + REQUESTED = + AtomicLongFieldUpdater.newUpdater(TestPublisherSubscription.class, "requested"); + + @SuppressWarnings("unchecked") + TestPublisherSubscription(Subscriber actual, DefaultTestPublisher parent) { + this.actual = actual; + if(actual instanceof Fuseable.ConditionalSubscriber){ + this.actualConditional = (Fuseable.ConditionalSubscriber) actual; + } + else { + this.actualConditional = null; + } + this.parent = parent; + } + + @Override + public void request(long n) { + if (Operators.validate(n)) { + Operators.getAndAddCap(REQUESTED, this, n); + } + } + + @Override + public void cancel() { + if (!cancelled) { + DefaultTestPublisher.CANCEL_COUNT.incrementAndGet(parent); + if (parent.violations.contains(Violation.CLEANUP_ON_TERMINATE)) { + return; + } + cancelled = true; + parent.remove(this); + } + } + + void onNext(T value) { + long r = requested; + if (r != 0L || parent.violations.contains(Violation.REQUEST_OVERFLOW)) { + if (r == 0) { + parent.hasOverflown = true; + } + boolean sent; + if(actualConditional != null){ + sent = actualConditional.tryOnNext(value); + } + else { + sent = true; + actual.onNext(value); + } + if (sent && r != Long.MAX_VALUE) { + REQUESTED.decrementAndGet(this); + } + return; + } + parent.remove(this); + actual.onError(new IllegalStateException("Can't deliver value due to lack of requests")); + } + + void onError(Throwable e) { + actual.onError(e); + } + + void onComplete() { + actual.onComplete(); + } + } + + @Override + public Flux flux() { + return Flux.from(this); + } + + @Override + public Mono mono() { + return Mono.from(this); + } + + @Override + public DefaultTestPublisher assertMinRequested(long n) { + TestPublisherSubscription[] subs = subscribers; + long minRequest = Stream.of(subs) + .mapToLong(s -> s.requested) + .min() + .orElse(0); + if (minRequest < n) { + throw new AssertionError("Expected minimum request of " + n + "; got " + minRequest); + } + return this; + } + + @Override + public DefaultTestPublisher assertSubscribers() { + TestPublisherSubscription[] s = subscribers; + if (s == EMPTY || s == TERMINATED) { + throw new AssertionError("Expected subscribers"); + } + return this; + } + + @Override + public DefaultTestPublisher assertSubscribers(int n) { + int sl = subscribers.length; + if (sl != n) { + throw new AssertionError("Expected " + n + " subscribers, got " + sl); + } + return this; + } + + @Override + public DefaultTestPublisher assertNoSubscribers() { + int sl = subscribers.length; + if (sl != 0) { + throw new AssertionError("Expected no subscribers, got " + sl); + } + return this; + } + + @Override + public DefaultTestPublisher assertCancelled() { + if (cancelCount == 0) { + throw new AssertionError("Expected at least 1 cancellation"); + } + return this; + } + + @Override + public DefaultTestPublisher assertCancelled(int n) { + int cc = cancelCount; + if (cc != n) { + throw new AssertionError("Expected " + n + " cancellations, got " + cc); + } + return this; + } + + @Override + public DefaultTestPublisher assertNotCancelled() { + if (cancelCount != 0) { + throw new AssertionError("Expected no cancellation"); + } + return this; + } + + @Override + public DefaultTestPublisher assertRequestOverflow() { + if (!hasOverflown) { + throw new AssertionError("Expected some request overflow"); + } + return this; + } + + @Override + public DefaultTestPublisher assertNoRequestOverflow() { + if (hasOverflown) { + throw new AssertionError("Unexpected request overflow"); + } + return this; + } + + @Override + public DefaultTestPublisher next(T t) { + if (!violations.contains(Violation.ALLOW_NULL)) { + Objects.requireNonNull(t, "emitted values must be non-null"); + } + + for (TestPublisherSubscription s : subscribers) { + s.onNext(t); + } + + return this; + } + + @Override + public DefaultTestPublisher error(Throwable t) { + Objects.requireNonNull(t, "t"); + + error = t; + TestPublisherSubscription[] subs; + if (violations.contains(Violation.CLEANUP_ON_TERMINATE)) { + subs = subscribers; + } else { + subs = SUBSCRIBERS.getAndSet(this, TERMINATED); + } + for (TestPublisherSubscription s : subs) { + s.onError(t); + } + return this; + } + + @Override + public DefaultTestPublisher complete() { + TestPublisherSubscription[] subs; + if (violations.contains(Violation.CLEANUP_ON_TERMINATE)) { + subs = subscribers; + } else { + subs = SUBSCRIBERS.getAndSet(this, TERMINATED); + } + for (TestPublisherSubscription s : subs) { + s.onComplete(); + } + return this; + } + +} diff --git a/reactor-test/src/main/java/reactor/test/publisher/TestPublisher.java b/reactor-test/src/main/java/reactor/test/publisher/TestPublisher.java new file mode 100644 index 0000000000..c1952f67af --- /dev/null +++ b/reactor-test/src/main/java/reactor/test/publisher/TestPublisher.java @@ -0,0 +1,225 @@ +/* + * Copyright (c) 2011-2017 Pivotal Software Inc, All Rights Reserved. + * + * 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 + * + * http://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 reactor.test.publisher; + +import java.util.Objects; + +import org.reactivestreams.Publisher; +import org.reactivestreams.Subscriber; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +/** + * A {@link Publisher} that you can directly manipulate, triggering + * {@link #next(Object) onNext}, {@link #complete() onComplete} and + * {@link #error(Throwable) onError} events, for testing purposes. + * You can assert the state of the publisher using its {@code assertXXX} methods, + * usually inside a {@link reactor.test.StepVerifier}'s + * {@link reactor.test.StepVerifier.Step#then(Runnable) then} callback. + *

+ * The TestPublisher can also be made more lenient towards the RS spec + * and allow "spec violations", as enumerated in {@link Violation}. Use the + * {@link #createNoncompliant(Violation, Violation...)} factory method to create such + * a misbehaving publisher. + * + * @author Simon Basle + */ +public abstract class TestPublisher implements Publisher { + + /** + * Create a standard {@link TestPublisher}. + * + * @param the type of the publisher + * @return the new {@link TestPublisher} + */ + public static TestPublisher create() { + return new DefaultTestPublisher<>(); + } + + /** + * Create a {@link Violation noncompliant} {@link TestPublisher} + * with a given set of reactive streams spec violations that will be overlooked. + * + * @param first the first allowed {@link Violation} + * @param rest additional optional violations + * @param the type of the publisher + * @return the new noncompliant {@link TestPublisher} + */ + public static TestPublisher createNoncompliant(Violation first, Violation... rest) { + return new DefaultTestPublisher<>(first, rest); + } + + /** + * Convenience method to wrap this {@link TestPublisher} to a {@link Flux}. + */ + public abstract Flux flux(); + + /** + * Convenience method to wrap this {@link TestPublisher} to a {@link Mono}. + */ + public abstract Mono mono(); + + /** + * Assert that the current minimum request of all this publisher's subscribers + * is >= {@code n}. + * + * @param n the expected minimum request + * @return this {@link TestPublisher} for chaining. + */ + public abstract TestPublisher assertMinRequested(long n); + + /** + * Asserts that this publisher has subscribers. + * + * @return this {@link TestPublisher} for chaining. + */ + public abstract TestPublisher assertSubscribers(); + + /** + * Asserts that this publisher has exactly n subscribers. + * + * @param n the expected number of subscribers + * @return this {@link TestPublisher} for chaining. + */ + public abstract TestPublisher assertSubscribers(int n); + + /** + * Asserts that this publisher has no subscribers. + * + * @return this {@link TestPublisher} for chaining. + */ + public abstract TestPublisher assertNoSubscribers(); + + /** + * Asserts that this publisher has had at least one subscriber that has been cancelled. + * + * @return this {@link TestPublisher} for chaining. + */ + public abstract TestPublisher assertCancelled(); + + /** + * Asserts that this publisher has had at least n subscribers that have been cancelled. + * + * @param n the expected number of subscribers to have been cancelled. + * @return this {@link TestPublisher} for chaining. + */ + public abstract TestPublisher assertCancelled(int n); + + /** + * Asserts that this publisher has had no cancelled subscribers. + * + * @return this {@link TestPublisher} for chaining. + */ + public abstract TestPublisher assertNotCancelled(); + + /** + * Asserts that this publisher has had subscriber that saw request overflow, + * that is received an onNext event despite having a requested amount of 0 at + * the time. + * + * @return this {@link TestPublisher} for chaining. + */ + public abstract TestPublisher assertRequestOverflow(); + + /** + * Asserts that this publisher has had no subscriber with request overflow. + * Request overflow is receiving an onNext event despite having a requested amount + * of 0 at that time. + * + * @return this {@link TestPublisher} for chaining. + */ + public abstract TestPublisher assertNoRequestOverflow(); + + /** + * Send 1 {@link Subscriber#onNext(Object) onNext} signal to the subscribers. + * + * @param value the item to emit + * @return this {@link TestPublisher} for chaining. + */ + public abstract TestPublisher next(T value); + + /** + * Triggers an {@link Subscriber#onError(Throwable) error} signal to the subscribers. + * + * @param t the {@link Throwable} to trigger + * @return this {@link TestPublisher} for chaining. + */ + public abstract TestPublisher error(Throwable t); + + /** + * Triggers {@link Subscriber#onComplete() completion} of this publisher. + * + * @return this {@link TestPublisher} for chaining. + */ + public abstract TestPublisher complete(); + + /** + * Send 1-n {@link Subscriber#onNext(Object) onNext} signals to the subscribers. + * + * @param first the first item to emit + * @param rest the optional remaining items to emit + * @return this {@link TestPublisher} for chaining. + * @see #next(Object) next + */ + @SafeVarargs + public final TestPublisher next(T first, T... rest) { + Objects.requireNonNull(rest, "rest array is null, please cast to T if null T required"); + next(first); + for (T t : rest) { + next(t); + } + return this; + } + + /** + * Combine emitting items and completing this publisher. + * + * @param values the values to emit to subscribers + * @return this {@link TestPublisher} for chaining. + * @see #next(Object) next + * @see #complete() complete + */ + @SafeVarargs + public final TestPublisher emit(T... values) { + Objects.requireNonNull(values, "values array is null, please cast to T if null T required"); + for (T t : values) { + next(t); + } + return complete(); + } + + /** + * Possible misbehavior for a {@link TestPublisher}. + */ + public enum Violation { + /** + * Allow {@link TestPublisher#next(Object, Object[]) next} calls to be made + * despite insufficient request, without triggering an {@link IllegalStateException}. + */ + REQUEST_OVERFLOW, + /** + * Allow {@link TestPublisher#next(Object, Object[]) next} calls to be made + * with a {@code null} value without triggering a {@link NullPointerException} + */ + ALLOW_NULL, + /** + * Allow termination signals to be sent several times in a row. This includes + * {@link TestPublisher#complete()}, {@link TestPublisher#error(Throwable)} and + * {@link TestPublisher#emit(Object[])}. + */ + CLEANUP_ON_TERMINATE + } +} diff --git a/reactor-test/src/main/java/reactor/test/scheduler/VirtualTimeScheduler.java b/reactor-test/src/main/java/reactor/test/scheduler/VirtualTimeScheduler.java new file mode 100644 index 0000000000..b6c552f3a3 --- /dev/null +++ b/reactor-test/src/main/java/reactor/test/scheduler/VirtualTimeScheduler.java @@ -0,0 +1,535 @@ +/* + * Copyright (c) 2011-2017 Pivotal Software Inc, All Rights Reserved. + * + * 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 + * + * http://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 reactor.test.scheduler; + +import java.time.Duration; +import java.time.Instant; +import java.util.Queue; +import java.util.concurrent.PriorityBlockingQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLongFieldUpdater; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; + +import reactor.core.Disposable; +import reactor.core.Exceptions; +import reactor.core.scheduler.Scheduler; +import reactor.core.scheduler.Schedulers; +import reactor.util.concurrent.QueueSupplier; + +/** + * A {@link Scheduler} that uses a virtual clock, allowing to manipulate time + * (eg. in tests). Can replace the default reactor schedulers by using + * the {@link #getOrSet} / {@link #set(VirtualTimeScheduler)} methods. + * + * @author Stephane Maldini + */ +public class VirtualTimeScheduler implements Scheduler { + + /** + * Create a new {@link VirtualTimeScheduler} without enabling it. Call + * {@link #getOrSet(VirtualTimeScheduler)} to enable it on + * {@link Schedulers.Factory} factories. + * + * @return a new {@link VirtualTimeScheduler} intended for timed-only + * {@link Schedulers} factories. + */ + public static VirtualTimeScheduler create() { + return new VirtualTimeScheduler(); + } + + /** + * Assign a single newly created {@link VirtualTimeScheduler} to all {@link Schedulers.Factory} + * factories. While the method is thread safe, its usually advised to execute such + * wide-impact BEFORE all tested code runs (setup etc). The created scheduler is returned. + * + * @return the VirtualTimeScheduler that was created and set through the factory + */ + public static VirtualTimeScheduler getOrSet() { + return enable(VirtualTimeScheduler::new, false); + } + + /** + * Assign an externally created {@link VirtualTimeScheduler} to the relevant + * {@link Schedulers.Factory} factories, depending on how it was created (see + * {@link #create()} and {@link #create()}). Note that the returned scheduler + * should always be captured and used going forward, as the provided scheduler can be + * superseded by a matching scheduler that has already been enabled. + *

+ * While the method is thread safe, it's usually advised to execute such wide-impact + * BEFORE all tested code runs (setup etc). The actual enabled Scheduler is returned. + * + * @param scheduler the {@link VirtualTimeScheduler} to use in factories. + * @return the enabled VirtualTimeScheduler (can be different from the provided one) + */ + public static VirtualTimeScheduler getOrSet(VirtualTimeScheduler scheduler) { + return enable(() -> scheduler, false); + } + + /** + * Assign an externally created {@link VirtualTimeScheduler} to the relevant + * {@link Schedulers.Factory} factories, depending on how it was created (see + * {@link #create()} and {@link #create()}). Contrary to + * {@link #getOrSet(VirtualTimeScheduler)}, the provided scheduler is always used, even + * if a matching scheduler is currently enabled. + *

+ * While the method is thread safe, it's usually advised to execute such wide-impact + * BEFORE all tested code runs (setup etc). + * + * @param scheduler the {@link VirtualTimeScheduler} to use in factories. + * @return the enabled VirtualTimeScheduler (same as provided), for chaining + */ + public static VirtualTimeScheduler set(VirtualTimeScheduler scheduler) { + return enable(() -> scheduler, true); + } + + /** + * Common method to enable a {@link VirtualTimeScheduler} in {@link Schedulers} + * factories. The supplier is lazily called. Enabling the same scheduler twice is + * also idempotent. + * + * @param schedulerSupplier the supplier executed to obtain a fresh {@link VirtualTimeScheduler} + * @return the scheduler that is actually used after the operation. + */ + static VirtualTimeScheduler enable(Supplier schedulerSupplier) { + return enable(schedulerSupplier, false); + } + + /** + * Common method to enable a {@link VirtualTimeScheduler} in {@link Schedulers} + * factories. The supplier is lazily called. Enabling the same scheduler twice is + * also idempotent. + * + * @param schedulerSupplier the supplier executed to obtain a fresh {@link VirtualTimeScheduler} + * @param exact whether or not to force the use of the supplier, even if there's a matching scheduler + * @return the scheduler that is actually used after the operation. + */ + static VirtualTimeScheduler enable(Supplier + schedulerSupplier, boolean exact) { + for (; ; ) { + VirtualTimeScheduler s = CURRENT.get(); + if (s != null && !exact) { + return s; + } + VirtualTimeScheduler newS = schedulerSupplier.get(); + if (newS == CURRENT.get()) { + return newS; //nothing to do, it has already been set in the past + } + + if (CURRENT.compareAndSet(s, newS)) { + Schedulers.setFactory(new AllFactory(newS)); + if (CURRENT.get() == newS) { + return newS; + } + } + } + } + + /** + * The current {@link VirtualTimeScheduler} assigned in {@link Schedulers} + * @return current {@link VirtualTimeScheduler} assigned in {@link Schedulers} + * @throws IllegalStateException if no {@link VirtualTimeScheduler} has been found + */ + public static VirtualTimeScheduler get(){ + VirtualTimeScheduler s = CURRENT.get(); + if (s == null) { + throw new IllegalStateException( + "Check if VirtualTimeScheduler#enable has been invoked" + " first" + ": " + s); + } + return s; + } + + /** + * Return true if there is a {@link VirtualTimeScheduler} currently used by the + * {@link Schedulers} factory (ie it has been {@link #set(VirtualTimeScheduler) enabled}), + * false otherwise (ie it has been {@link #reset() reset}). + */ + public static boolean isFactoryEnabled() { + return CURRENT.get() != null; + } + + /** + * Re-assign the default Reactor Core {@link Schedulers} factories. + * While the method is thread safe, its usually advised to execute such wide-impact + * AFTER all tested code has been run (teardown etc). + */ + public static void reset() { + VirtualTimeScheduler s = CURRENT.get(); + if (s != null && CURRENT.compareAndSet(s, null)) { + Schedulers.resetFactory(); + } + } + + final Queue queue = + new PriorityBlockingQueue<>(QueueSupplier.XS_BUFFER_SIZE); + + @SuppressWarnings("unused") + volatile long counter; + + volatile long nanoTime; + + volatile boolean shutdown; + + protected VirtualTimeScheduler() { + } + + /** + * Triggers any tasks that have not yet been executed and that are scheduled to be + * executed at or before this {@link VirtualTimeScheduler}'s present time. + */ + public void advanceTime() { + advanceTimeBy(Duration.ZERO); + } + + /** + * Moves the {@link VirtualTimeScheduler}'s clock forward by a specified amount of time. + * + * @param delayTime the amount of time to move the {@link VirtualTimeScheduler}'s clock forward + */ + public void advanceTimeBy(Duration delayTime) { + advanceTime(nanoTime + delayTime.toNanos()); + } + + /** + * Moves the {@link VirtualTimeScheduler}'s clock to a particular moment in time. + * + * @param instant the point in time to move the {@link VirtualTimeScheduler}'s + * clock to + */ + public void advanceTimeTo(Instant instant) { + long targetTime = TimeUnit.NANOSECONDS.convert(instant.toEpochMilli(), + TimeUnit.MILLISECONDS); + advanceTime(targetTime); + } + + @Override + public VirtualTimeWorker createWorker() { + if (shutdown) { + throw new IllegalStateException("VirtualTimeScheduler is shutdown"); + } + return new VirtualTimeWorker(); + } + + @Override + public long now(TimeUnit unit) { + return unit.convert(nanoTime, TimeUnit.NANOSECONDS); + } + + @Override + public Disposable schedule(Runnable task) { + if (shutdown) { + return REJECTED; + } + return createWorker().schedule(task); + } + + @Override + public Disposable schedule(Runnable task, long delay, TimeUnit unit) { + if (shutdown) { + return REJECTED; + } + return createWorker().schedule(task, delay, unit); + } + + @Override + public boolean isDisposed() { + return shutdown; + } + + @Override + public void dispose() { + if (shutdown) { + return; + } + queue.clear(); + shutdown = true; + //TODO remove the below behavior? + VirtualTimeScheduler s = CURRENT.get(); + if (s != null && s == this && CURRENT.compareAndSet(s, null)) { + Schedulers.resetFactory(); + } + } + + @Override + public Disposable schedulePeriodically(Runnable task, + long initialDelay, + long period, TimeUnit unit) { + if (shutdown) { + return REJECTED; + } + + final Worker w = createWorker(); + + PeriodicDirectTask periodicTask = new PeriodicDirectTask(task, w); + + w.schedulePeriodically(periodicTask, initialDelay, period, unit); + + return periodicTask; + } + + final void advanceTime(long targetTimeInNanoseconds) { + while (!queue.isEmpty()) { + TimedRunnable current = queue.peek(); + if (current.time > targetTimeInNanoseconds) { + break; + } + // if scheduled time is 0 (immediate) use current virtual time + nanoTime = current.time == 0 ? nanoTime : current.time; + queue.remove(); + + // Only execute if not unsubscribed + if (!current.scheduler.shutdown) { + current.run.run(); + } + } + nanoTime = targetTimeInNanoseconds; + } + + static final class TimedRunnable implements Comparable { + + final long time; + final Runnable run; + final VirtualTimeWorker scheduler; + final long count; // for differentiating tasks at same time + + TimedRunnable(VirtualTimeWorker scheduler, long time, Runnable run, long count) { + this.time = time; + this.run = run; + this.scheduler = scheduler; + this.count = count; + } + + @Override + public int compareTo(TimedRunnable o) { + if (time == o.time) { + return compare(count, o.count); + } + return compare(time, o.time); + } + + static int compare(long a, long b){ + return a < b ? -1 : (a > b ? 1 : 0); + } + } + + static final class AllFactory implements Schedulers.Factory { + + final VirtualTimeScheduler s; + + AllFactory(VirtualTimeScheduler s) { + this.s = s; + } + + @Override + public Scheduler newElastic(int ttlSeconds, ThreadFactory threadFactory) { + return s; + } + + @Override + public Scheduler newParallel(int parallelism, ThreadFactory threadFactory) { + return s; + } + + @Override + public Scheduler newSingle(ThreadFactory threadFactory) { + return s; + } + } + + final class VirtualTimeWorker implements Worker { + + volatile boolean shutdown; + + @Override + public Disposable schedule(Runnable run) { + if (shutdown) { + return REJECTED; + } + final TimedRunnable timedTask = new TimedRunnable(this, + 0, + run, + COUNTER.getAndIncrement(VirtualTimeScheduler.this)); + queue.add(timedTask); + return () -> queue.remove(timedTask); + } + + @Override + public Disposable schedule(Runnable run, long delayTime, TimeUnit unit) { + if (shutdown) { + return REJECTED; + } + final TimedRunnable timedTask = new TimedRunnable(this, + nanoTime + unit.toNanos(delayTime), + run, + COUNTER.getAndIncrement(VirtualTimeScheduler.this)); + queue.add(timedTask); + + return () -> queue.remove(timedTask); + } + + @Override + public Disposable schedulePeriodically(Runnable task, + long initialDelay, + long period, + TimeUnit unit) { + final long periodInNanoseconds = unit.toNanos(period); + final long firstNowNanoseconds = now(TimeUnit.NANOSECONDS); + final long firstStartInNanoseconds = firstNowNanoseconds + unit.toNanos(initialDelay); + + PeriodicTask periodicTask = new PeriodicTask(firstStartInNanoseconds, task, + firstNowNanoseconds, + periodInNanoseconds); + + replace(periodicTask, schedule(periodicTask, initialDelay, unit)); + + return periodicTask; + } + + @Override + public void dispose() { + shutdown = true; + } + + @Override + public boolean isDisposed() { + return shutdown; + } + } + + static final Disposable CANCELLED = () -> { + }; + static final Disposable EMPTY = () -> { + }; + + final class PeriodicTask extends AtomicReference implements Runnable, + Disposable { + + final Runnable decoratedRun; + final long periodInNanoseconds; + long count; + long lastNowNanoseconds; + long startInNanoseconds; + + PeriodicTask(long firstStartInNanoseconds, + Runnable decoratedRun, + long firstNowNanoseconds, + long periodInNanoseconds) { + this.decoratedRun = decoratedRun; + this.periodInNanoseconds = periodInNanoseconds; + lastNowNanoseconds = firstNowNanoseconds; + startInNanoseconds = firstStartInNanoseconds; + lazySet(EMPTY); + } + + @Override + public void run() { + decoratedRun.run(); + + if (get() != CANCELLED) { + + long nextTick; + + long nowNanoseconds = now(TimeUnit.NANOSECONDS); + // If the clock moved in a direction quite a bit, rebase the repetition period + if (nowNanoseconds + CLOCK_DRIFT_TOLERANCE_NANOSECONDS < lastNowNanoseconds || nowNanoseconds >= lastNowNanoseconds + periodInNanoseconds + CLOCK_DRIFT_TOLERANCE_NANOSECONDS) { + nextTick = nowNanoseconds + periodInNanoseconds; + /* + * Shift the start point back by the drift as if the whole thing + * started count periods ago. + */ + startInNanoseconds = nextTick - (periodInNanoseconds * (++count)); + } + else { + nextTick = startInNanoseconds + (++count * periodInNanoseconds); + } + lastNowNanoseconds = nowNanoseconds; + + long delay = nextTick - nowNanoseconds; + replace(this, schedule(this, delay, TimeUnit.NANOSECONDS)); + } + } + + @Override + public void dispose() { + getAndSet(CANCELLED).dispose(); + } + } + + static boolean replace(AtomicReference ref, Disposable c) { + for (; ; ) { + Disposable current = ref.get(); + if (current == CANCELLED) { + if (c != null) { + c.dispose(); + } + return false; + } + if (ref.compareAndSet(current, c)) { + return true; + } + } + } + + static class PeriodicDirectTask implements Runnable, Disposable { + + final Runnable run; + + final Scheduler.Worker worker; + + volatile boolean disposed; + + PeriodicDirectTask(Runnable run, Worker worker) { + this.run = run; + this.worker = worker; + } + + @Override + public void run() { + if (!disposed) { + try { + run.run(); + } + catch (Throwable ex) { + Exceptions.throwIfFatal(ex); + worker.dispose(); + throw Exceptions.propagate(ex); + } + } + } + + @Override + public void dispose() { + disposed = true; + worker.dispose(); + } + } + + static final AtomicReference CURRENT = new AtomicReference<>(); + + static final AtomicLongFieldUpdater COUNTER = + AtomicLongFieldUpdater.newUpdater(VirtualTimeScheduler.class, "counter"); + static final long CLOCK_DRIFT_TOLERANCE_NANOSECONDS; + + static { + CLOCK_DRIFT_TOLERANCE_NANOSECONDS = TimeUnit.MINUTES.toNanos(Long.getLong( + "reactor.scheduler.drift-tolerance", + 15)); + } + + +} diff --git a/reactor-test/src/test/java/reactor/test/DefaultStepVerifierBuilderTests.java b/reactor-test/src/test/java/reactor/test/DefaultStepVerifierBuilderTests.java new file mode 100644 index 0000000000..67ea3899d0 --- /dev/null +++ b/reactor-test/src/test/java/reactor/test/DefaultStepVerifierBuilderTests.java @@ -0,0 +1,180 @@ +/* + * Copyright (c) 2011-2017 Pivotal Software Inc, All Rights Reserved. + * + * 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 + * + * http://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 reactor.test; + +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.Queue; + +import org.junit.Test; +import reactor.core.publisher.Flux; +import reactor.test.DefaultStepVerifierBuilder.DefaultVerifySubscriber; +import reactor.test.DefaultStepVerifierBuilder.DescriptionEvent; +import reactor.test.DefaultStepVerifierBuilder.Event; +import reactor.test.DefaultStepVerifierBuilder.SignalCountEvent; +import reactor.test.DefaultStepVerifierBuilder.SignalEvent; +import reactor.test.DefaultStepVerifierBuilder.SubscriptionEvent; +import reactor.test.DefaultStepVerifierBuilder.TaskEvent; +import reactor.test.DefaultStepVerifierBuilder.WaitEvent; +import reactor.test.scheduler.VirtualTimeScheduler; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + +/** + * @author Stephane Maldini + */ +public class DefaultStepVerifierBuilderTests { + + + @Test + public void subscribedTwice() { + Flux flux = Flux.just("foo", "bar"); + + DefaultVerifySubscriber s = + new DefaultStepVerifierBuilder(StepVerifierOptions.create().initialRequest(Long.MAX_VALUE), null) + .expectNext("foo", "bar") + .expectComplete() + .toSubscriber(); + + flux.subscribe(s); + flux.subscribe(s); + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(s::verify) + .withMessageStartingWith("expectation failed (an unexpected Subscription has been received"); + } + + @Test(timeout = 4000) + public void manuallyManagedVirtualTime() { + VirtualTimeScheduler vts = VirtualTimeScheduler.create(); + try { + VirtualTimeScheduler.getOrSet(vts); + assertThat(VirtualTimeScheduler.get()).isSameAs(vts); + + Flux flux = Flux.just("foo").delayElements(Duration.ofSeconds(4)); + + DefaultVerifySubscriber s = + new DefaultStepVerifierBuilder(StepVerifierOptions.create() + .initialRequest(Long.MAX_VALUE) + .virtualTimeSchedulerSupplier(() -> vts), + null)//important to avoid triggering of vts capture-and-enable + .thenAwait(Duration.ofSeconds(1)) + .expectNext("foo") + .expectComplete() + .toSubscriber(); + + flux.subscribe(s); + vts.advanceTimeBy(Duration.ofSeconds(3)); + s.verify(); + + assertThat(s.virtualTimeScheduler()).isSameAs(vts); + assertThat(VirtualTimeScheduler.get()).isSameAs(vts); + } + finally { + VirtualTimeScheduler.reset(); + } + } + + @Test + public void suppliedVirtualTimeButNoSourceDoesntEnableScheduler() { + VirtualTimeScheduler vts = VirtualTimeScheduler.create(); + + new DefaultStepVerifierBuilder(StepVerifierOptions.create() + .initialRequest(Long.MAX_VALUE) + .virtualTimeSchedulerSupplier(() -> vts), + null) //important to avoid triggering of vts capture-and-enable + .expectNoEvent(Duration.ofSeconds(4)) + .expectComplete() + .toSubscriber(); + + try { + //also test the side effect case where VTS has been enabled and not reset + VirtualTimeScheduler current = VirtualTimeScheduler.get(); + assertThat(current).isNotSameAs(vts); + } + catch (IllegalStateException e) { + assertThat(e).hasMessageContaining("VirtualTimeScheduler"); + } + } + + @Test + public void testConflateOnTaskThenSubscriptionEvents() { + List> script = Arrays.asList( + new TaskEvent(() -> {}, "A"), + new TaskEvent(() -> {}, "B"), + new WaitEvent(Duration.ofSeconds(5), "C"), + new SubscriptionEvent("D"), + new SubscriptionEvent(sub -> { }, "E") + ); + + Queue> queue = + DefaultVerifySubscriber.conflateScript(script, null); + + assertThat(queue) + .hasSize(5) + .extracting(e -> e.getClass().getName()) + .containsExactly( + TaskEvent.class.getName(), + TaskEvent.class.getName(), + WaitEvent.class.getName(), + DefaultStepVerifierBuilder.SubscriptionTaskEvent.class.getName(), + DefaultStepVerifierBuilder.SubscriptionTaskEvent.class.getName()); + } + + @Test + public void testNoConflateOnSignalThenSubscriptionEvents() { + List> script = Arrays.asList( + new TaskEvent(() -> {}, "A"), + new WaitEvent(Duration.ofSeconds(5), "B"), + new SignalCountEvent<>(3, "C"), + new SubscriptionEvent("D"), + new SubscriptionEvent(sub -> { }, "E") + ); + + Queue> queue = + DefaultVerifySubscriber.conflateScript(script, null); + + assertThat(queue) + .hasSize(5) + .extracting(e -> e.getClass().getName()) + .containsExactly( + TaskEvent.class.getName(), + WaitEvent.class.getName(), + SignalCountEvent.class.getName(), + SubscriptionEvent.class.getName(), + SubscriptionEvent.class.getName()); + } + + @Test + public void testConflateChangesDescriptionAndRemoveAs() { + List> script = Arrays.asList( + new SignalEvent((s,v) -> Optional.empty(), "A"), + new SignalEvent((s,v) -> Optional.empty(), "B"), + new DescriptionEvent("foo"), + new DescriptionEvent("bar"), + new SignalCountEvent(1, "C"), + new DescriptionEvent("baz") + ); + + Queue> queue = DefaultVerifySubscriber.conflateScript(script, null); + + assertThat(queue).hasSize(3) + .extracting(Event::getDescription) + .containsExactly("A", "foo", "baz"); + } +} diff --git a/reactor-test/src/test/java/reactor/test/StepVerifierAssertionsTests.java b/reactor-test/src/test/java/reactor/test/StepVerifierAssertionsTests.java new file mode 100644 index 0000000000..c1a66d0292 --- /dev/null +++ b/reactor-test/src/test/java/reactor/test/StepVerifierAssertionsTests.java @@ -0,0 +1,536 @@ +/* + * Copyright (c) 2011-2017 Pivotal Software Inc, All Rights Reserved. + * + * 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 + * + * http://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 reactor.test; + +import java.time.Duration; + +import org.junit.Test; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; +import reactor.core.publisher.Operators; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; + +public class StepVerifierAssertionsTests { + + @Test + public void assertDroppedElementsAllPass() { + StepVerifier.create(Flux.from(s -> { + s.onSubscribe(Operators.emptySubscription()); + s.onNext("foo"); + s.onComplete(); + s.onNext("bar"); + s.onNext("baz"); + }).take(3)) + .expectNext("foo") + .expectComplete() + .verifyThenAssertThat() + .hasDroppedElements() + .hasDropped("baz") + .hasDroppedExactly("baz", "bar"); + } + + @Test + public void assertDroppedElementsFailureNoDrop() { + try { + StepVerifier.create(Mono.empty()) + .expectComplete() + .verifyThenAssertThat() + .hasDroppedElements(); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected dropped elements, none found."); + } + } + + @Test + public void assertDroppedElementsFailureOneExtra() { + try { + StepVerifier.create(Flux.from(s -> { + s.onSubscribe(Operators.emptySubscription()); + s.onNext("foo"); + s.onComplete(); + s.onNext("bar"); + s.onNext("baz"); + }).take(3)) + .expectNext("foo") + .expectComplete() + .verifyThenAssertThat() + .hasDropped("foo"); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected dropped elements to contain <[foo]>, was <[bar, baz]>."); + } + } + + @Test + public void assertDroppedElementsFailureOneMissing() { + try { + StepVerifier.create(Flux.from(s -> { + s.onSubscribe(Operators.emptySubscription()); + s.onNext("foo"); + s.onComplete(); + s.onNext("bar"); + s.onNext("baz"); + }).take(3)) + .expectNext("foo") + .expectComplete() + .verifyThenAssertThat() + .hasDroppedExactly("baz"); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected dropped elements to contain exactly <[baz]>, was <[bar, baz]>."); + } + } + + @Test + public void assertDroppedErrorAllPass() { + Throwable err1 = new IllegalStateException("boom1"); + Throwable err2 = new IllegalStateException("boom2"); + StepVerifier.create(Flux.from(s -> { + s.onSubscribe(Operators.emptySubscription()); + s.onError(err1); + s.onError(err2); + }).buffer(1)) + .expectError() + .verifyThenAssertThat() + .hasDroppedErrors() + .hasDroppedErrors(1) + .hasDroppedErrorOfType(IllegalStateException.class) + .hasDroppedErrorWithMessageContaining("boom") + .hasDroppedErrorWithMessage("boom2") + .hasDroppedErrorMatching(t -> t instanceof IllegalStateException && "boom2".equals(t.getMessage())); + } + + @Test + public void assertDroppedErrorFailureNoDrop() { + try { + StepVerifier.create(Mono.empty()) + .expectComplete() + .verifyThenAssertThat() + .hasDroppedErrors(); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected at least 1 dropped error, none found."); + } + } + + @Test + public void assertDroppedErrorFailureWrongType() { + try { + Throwable err1 = new IllegalStateException("boom1"); + Throwable err2 = new IllegalStateException("boom2"); + StepVerifier.create(Flux.from(s -> { + s.onSubscribe(Operators.emptySubscription()); + s.onError(err1); + s.onError(err2); + }).buffer(1)) + .expectError() + .verifyThenAssertThat() + .hasDroppedErrorOfType(IllegalArgumentException.class); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected dropped error to be of type java.lang.IllegalArgumentException, was java.lang.IllegalStateException."); + } + } + + @Test + public void assertDroppedErrorFailureWrongContains() { + try { + Throwable err1 = new IllegalStateException("boom1"); + Throwable err2 = new IllegalStateException("boom2"); + StepVerifier.create(Flux.from(s -> { + s.onSubscribe(Operators.emptySubscription()); + s.onError(err1); + s.onError(err2); + }).buffer(1)) + .expectError() + .verifyThenAssertThat() + .hasDroppedErrorWithMessageContaining("foo"); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected dropped error with message containing <\"foo\">, was <\"boom2\">."); + } + } + + @Test + public void assertDroppedErrorFailureWrongMessage() { + try { + Throwable err1 = new IllegalStateException("boom1"); + Throwable err2 = new IllegalStateException("boom2"); + StepVerifier.create(Flux.from(s -> { + s.onSubscribe(Operators.emptySubscription()); + s.onError(err1); + s.onError(err2); + }).buffer(1)) + .expectError() + .verifyThenAssertThat() + .hasDroppedErrorWithMessage("boom1"); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected dropped error with message <\"boom1\">, was <\"boom2\">."); + } + } + + @Test + public void assertDroppedErrorFailureWrongMatch() { + try { + Throwable err1 = new IllegalStateException("boom1"); + Throwable err2 = new IllegalStateException("boom2"); + StepVerifier.create(Flux.from(s -> { + s.onSubscribe(Operators.emptySubscription()); + s.onError(err1); + s.onError(err2); + }).buffer(1)) + .expectError() + .verifyThenAssertThat() + .hasDroppedErrorMatching(t -> t instanceof IllegalStateException && "foo".equals(t.getMessage())); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected dropped error matching the given predicate, did not match: ."); + } + } + + @Test + public void assertDroppedErrorsFailureWrongCount() { + Throwable err1 = new IllegalStateException("boom1"); + Throwable err2 = new IllegalStateException("boom2"); + Throwable err3 = new IllegalStateException("boom3"); + try { + StepVerifier.create(Flux.from(s -> { + s.onSubscribe(Operators.emptySubscription()); + s.onError(err1); + s.onError(err2); + s.onError(err3); + }).buffer(1)) + .expectError() + .verifyThenAssertThat() + .hasDroppedErrors() + .hasDroppedErrors(3); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected exactly 3 dropped errors, 2 found."); + } + } + + @Test + public void assertDroppedErrorsNotSatisfying() { + Throwable err1 = new IllegalStateException("boom1"); + Throwable err2 = new IllegalStateException("boom2"); + Throwable err3 = new IllegalStateException("boom3"); + try { + StepVerifier.create(Flux.from(s -> { + s.onSubscribe(Operators.emptySubscription()); + s.onError(err1); + s.onError(err2); + s.onError(err3); + }).buffer(1)) + .expectError() + .verifyThenAssertThat() + .hasDroppedErrorsSatisfying(c -> assertThat(c).hasSize(3)); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessageContaining("Expected size:<3> but was:<2> in:"); + } + } + + @Test + public void assertDroppedErrorsNotMatching() { + Throwable err1 = new IllegalStateException("boom1"); + Throwable err2 = new IllegalStateException("boom2"); + Throwable err3 = new IllegalStateException("boom3"); + try { + StepVerifier.create(Flux.from(s -> { + s.onSubscribe(Operators.emptySubscription()); + s.onError(err1); + s.onError(err2); + s.onError(err3); + }).buffer(1)) + .expectError() + .verifyThenAssertThat() + .hasDroppedErrorsMatching(c -> c.size() == 3); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected collection of dropped errors matching the " + + "given predicate, did not match: <[java.lang.IllegalStateException: boom2, " + + "java.lang.IllegalStateException: boom3]>."); + } + } + + @Test + public void assertOperatorErrorAllPass() { + Throwable err1 = new IllegalStateException("boom1"); + StepVerifier.create(Flux.error(err1)) + .expectError() + .verifyThenAssertThat() + .hasOperatorErrors() + .hasOperatorErrors(1) + .hasOperatorErrorOfType(IllegalStateException.class) + .hasOperatorErrorWithMessageContaining("boom") + .hasOperatorErrorWithMessage("boom1") + .hasOperatorErrorMatching(t -> t instanceof IllegalStateException && "boom1".equals(t.getMessage())); + } + + @Test + public void assertOperatorErrorFailureNoDrop() { + try { + StepVerifier.create(Mono.empty()) + .expectComplete() + .verifyThenAssertThat() + .hasOperatorErrors(); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected at least 1 operator error, none found."); + } + } + + @Test + public void assertOperatorErrorFailureWrongType() { + try { + Throwable err1 = new IllegalStateException("boom1"); + StepVerifier.create(Flux.error(err1)) + .expectError() + .verifyThenAssertThat() + .hasOperatorErrorOfType(IllegalArgumentException.class); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected operator error to be of type java.lang.IllegalArgumentException, was java.lang.IllegalStateException."); + } + } + + @Test + public void assertOperatorErrorFailureWrongContains() { + try { + Throwable err1 = new IllegalStateException("boom1"); + StepVerifier.create(Flux.error(err1)) + .expectError() + .verifyThenAssertThat() + .hasOperatorErrorWithMessageContaining("foo"); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected operator error with message containing <\"foo\">, was <\"boom1\">."); + } + } + + @Test + public void assertOperatorErrorFailureWrongMessage() { + try { + Throwable err1 = new IllegalStateException("boom1"); + StepVerifier.create(Flux.error(err1)) + .expectError() + .verifyThenAssertThat() + .hasOperatorErrorWithMessage("boom2"); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected operator error with message <\"boom2\">, was <\"boom1\">."); + } + } + + @Test + public void assertOperatorErrorFailureWrongMatch() { + try { + Throwable err1 = new IllegalStateException("boom1"); + StepVerifier.create(Flux.error(err1)) + .expectError() + .verifyThenAssertThat() + .hasOperatorErrorMatching(t -> t instanceof IllegalStateException && "foo".equals(t.getMessage())); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected operator error matching the given predicate," + + " did not match: <[java.lang.IllegalStateException: boom1,]>."); + } + } + + @Test + public void assertOperatorErrorsFailureWrongCount() { + Throwable err1 = new IllegalStateException("boom1"); + Throwable err2 = new IllegalStateException("boom2"); + Throwable err3 = new IllegalStateException("boom3"); + try { + StepVerifier.create(Flux.from(s -> { + s.onSubscribe(Operators.emptySubscription()); + s.onError(err1); + Operators.onOperatorError(err2); + Operators.onOperatorError(err3); + }).buffer(1)) + .expectError() + .verifyThenAssertThat() + .hasOperatorErrors() + .hasOperatorErrors(3); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected exactly 3 operator errors, 2 found."); + } + } + + @Test + public void assertOperatorErrorsNotSatisfying() { + Throwable err1 = new IllegalStateException("boom1"); + Throwable err2 = new IllegalStateException("boom2"); + Throwable err3 = new IllegalStateException("boom3"); + try { + StepVerifier.create(Flux.from(s -> { + s.onSubscribe(Operators.emptySubscription()); + s.onError(err1); + Operators.onOperatorError(err2); + Operators.onOperatorError(err3); + }).buffer(1)) + .expectError() + .verifyThenAssertThat() + .hasOperatorErrorsSatisfying(c -> assertThat(c).hasSize(3)); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessageEndingWith("Expected size:<3> but was:<2> in:\n" + + "<[[java.lang.IllegalStateException: boom2,],\n" + + " [java.lang.IllegalStateException: boom3,]]>"); + } + } + + @Test + public void assertOperatorErrorsNotMatching() { + Throwable err1 = new IllegalStateException("boom1"); + Throwable err2 = new IllegalStateException("boom2"); + Throwable err3 = new IllegalStateException("boom3"); + try { + StepVerifier.create(Flux.from(s -> { + s.onSubscribe(Operators.emptySubscription()); + s.onError(err1); + Operators.onOperatorError(err2); + Operators.onOperatorError(err3); + }).buffer(1)) + .expectError() + .verifyThenAssertThat() + .hasOperatorErrorsMatching(c -> c.size() == 3); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected collection of operator errors matching the" + + " given predicate, did not match: <[[java.lang.IllegalStateException: boom2,]," + + " [java.lang.IllegalStateException: boom3,]]>."); + } + } + + @Test + public void assertDurationLessThanOk() { + StepVerifier.create(Mono.delay(Duration.ofMillis(500)).then()) + .expectComplete() + .verifyThenAssertThat() + .tookLessThan(Duration.ofSeconds(1)); + } + + @Test + public void assertDurationLessThanFailure() { + try { + StepVerifier.create(Mono.delay(Duration.ofMillis(500)).then()) + .expectComplete() + .verifyThenAssertThat() + .tookLessThan(Duration.ofMillis(200)); + + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae) + //the actual duration can vary a bit + .hasMessageStartingWith("Expected scenario to be verified in less than 200ms, took 5") + .hasMessageEndingWith("ms."); + } + } + + @Test + public void assertDurationConsidersEqualsASuccess() { + new DefaultStepVerifierBuilder.DefaultStepVerifierAssertions(null, null, null, Duration.ofSeconds(3)) + .tookLessThan(Duration.ofMillis(3000L)) + .tookMoreThan(Duration.ofSeconds(3)); + } + + @Test + public void assertDurationMoreThanOk() { + StepVerifier.create(Mono.delay(Duration.ofMillis(500)).then()) + .expectComplete() + .verifyThenAssertThat() + .tookMoreThan(Duration.ofMillis(100)); + } + + @Test + public void assertDurationMoreThanFailure() { + try { + StepVerifier.create(Mono.delay(Duration.ofMillis(500)).then()) + .expectComplete() + .verifyThenAssertThat() + .tookMoreThan(Duration.ofMillis(800)); + + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae) + //the actual duration can vary a bit + .hasMessageStartingWith("Expected scenario to be verified in more than 800ms, took 5") + .hasMessageEndingWith("ms."); + } + } + + @Test + public void assertOperationErrorShortcutTestExactCount() { + try { + StepVerifier.create(Flux.empty()) + .expectComplete() + .verifyThenAssertThat() + .hasOperatorErrorWithMessage("boom2"); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected exactly one operator error, 0 found."); + } + } + + @Test + public void assertOperationErrorShortcutTestTupleContainsError() { + try { + StepVerifier.create(Flux.from(f -> { + f.onSubscribe(Operators.emptySubscription()); + Operators.onOperatorError(null, null, "foo"); + f.onComplete(); + })) + .expectComplete() + .verifyThenAssertThat() + .hasOperatorErrorWithMessage("boom2"); + fail("expected an AssertionError"); + } + catch (AssertionError ae) { + assertThat(ae).hasMessage("Expected exactly one operator error with an actual throwable content, no throwable found."); + } + } + +} diff --git a/reactor-test/src/test/java/reactor/test/StepVerifierTests.java b/reactor-test/src/test/java/reactor/test/StepVerifierTests.java new file mode 100644 index 0000000000..809e205df4 --- /dev/null +++ b/reactor-test/src/test/java/reactor/test/StepVerifierTests.java @@ -0,0 +1,1665 @@ +/* + * Copyright (c) 2011-2017 Pivotal Software Inc, All Rights Reserved. + * + * 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 + * + * http://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 reactor.test; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.LongAdder; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import reactor.core.Fuseable; +import reactor.core.publisher.DirectProcessor; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; +import reactor.core.publisher.UnicastProcessor; +import reactor.core.scheduler.Schedulers; +import reactor.test.publisher.TestPublisher; +import reactor.test.scheduler.VirtualTimeScheduler; + +import static org.assertj.core.api.Assertions.*; +import static reactor.test.publisher.TestPublisher.Violation.REQUEST_OVERFLOW; + +/** + * @author Arjen Poutsma + * @author Sebastien Deleuze + * @author Stephane Maldini + * @author Simon Basle + */ +public class StepVerifierTests { + + @Test + public void expectNext() { + Flux flux = Flux.just("foo", "bar"); + + StepVerifier.create(flux) + .expectNext("foo") + .expectNext("bar") + .expectComplete() + .verify(); + } + + @Test + public void expectInvalidNext() { + Flux flux = Flux.just("foo", "bar"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNext("foo") + .expectNext("baz") + .expectComplete() + .verify()) + .withMessageEndingWith("(expected value: baz; actual value: bar)"); + } + + @Test + public void expectNextAsync() { + Flux flux = Flux.just("foo", "bar") + .publishOn(Schedulers.parallel()); + + StepVerifier.create(flux) + .expectNext("foo") + .expectNext("bar") + .expectComplete() + .verify(); + } + + @Test + public void expectNexts() { + Flux flux = Flux.just("foo", "bar"); + + StepVerifier.create(flux) + .expectNext("foo", "bar") + .expectComplete() + .verify(); + } + + @Test + public void expectInvalidNexts() { + Flux flux = Flux.just("foo", "bar"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNext("foo", "baz") + .expectComplete() + .verify()) + .withMessage("expectation \"expectNext(baz)\" failed (expected value: baz; actual value: bar)"); + } + + @Test + public void expectNextMatches() { + Flux flux = Flux.just("foo", "bar"); + + StepVerifier.create(flux) + .expectNextMatches("foo"::equals) + .expectNextMatches("bar"::equals) + .expectComplete() + .verify(); + } + + @Test + public void expectInvalidNextMatches() { + Flux flux = Flux.just("foo", "bar"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNextMatches("foo"::equals) + .expectNextMatches("baz"::equals) + .expectComplete() + .verify()) + .withMessage("expectation \"expectNextMatches\" failed (predicate failed on value: bar)"); + } + + @Test + public void consumeNextWith() throws Exception { + Flux flux = Flux.just("bar"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .consumeNextWith(s -> { + if (!"foo".equals(s)) { + throw new AssertionError("e:" + s); + } + }) + .expectComplete() + .verify()) + .withMessage("e:bar"); + } + + @Test + public void consumeNextWith2() throws Exception { + Flux flux = Flux.just("bar"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .consumeNextWith(s -> { + if (!"foo".equals(s)) { + throw new AssertionError(s); + } + }) + .expectComplete() + .verify()) + .withMessage("bar"); + } + + @Test + public void assertNext() throws Exception { + Flux flux = Flux.just("foo"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .assertNext(s -> assertThat(s).endsWith("ooz")) + .expectComplete() + .verify()) + .withMessage("\nExpecting:\n <\"foo\">\nto end with:\n <\"ooz\">\n"); + } + + @Test + public void missingNext() { + Flux flux = Flux.just("foo", "bar"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNext("foo") + .expectComplete() + .verify()) + .withMessage("expectation \"expectComplete\" failed (expected: onComplete(); actual: onNext(bar))"); + } + + @Test + public void missingNextAsync() { + Flux flux = Flux.just("foo", "bar") + .publishOn(Schedulers.parallel()); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNext("foo") + .expectComplete() + .verify()) + .withMessage("expectation \"expectComplete\" failed (expected: onComplete(); actual: onNext(bar))"); + } + + @Test + public void expectNextCount() { + Flux flux = Flux.just("foo", "bar"); + + StepVerifier.create(flux, 0) + .thenRequest(1) + .expectNextCount(1) + .thenRequest(1) + .expectNextCount(1) + .expectComplete() + .verify(); + } + + @Test + public void expectNextCountLots() { + Flux flux = Flux.range(0, 1_000_000); + + StepVerifier.create(flux, 0) + .thenRequest(100_000) + .expectNextCount(100_000) + .thenRequest(500_000) + .expectNextCount(500_000) + .thenRequest(500_000) + .expectNextCount(400_000) + .expectComplete() + .verify(); + } + + @Test + public void expectNextCountLotsError() { + Flux flux = Flux.range(0, 1_000_000); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux, 0) + .thenRequest(100_000) + .expectNextCount(100_000) + .thenRequest(Integer.MAX_VALUE) + .expectNextCount(900_001) + .expectComplete() + .verify()) + .withMessageStartingWith("expectation \"expectNextCount(900001)\" failed") + .withMessageContaining("expected: count = 900001; actual: counted = 900000; signal: onComplete()"); + } + + @Test + public void expectNextCountLotsUnderRequestErrorReportedAtEnd() { + Flux flux = Flux.range(0, 1_000_000); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux, 0) + .thenRequest(100_000) + .expectNextCount(100_000) + .thenRequest(500_000) + .expectNextCount(499_999) + .thenRequest(500_000) + .expectNextCount(400_000) + .expectComplete() + .verify()) + .withMessage("expectation \"expectComplete\" failed (expected: onComplete(); actual: onNext(999999))"); + } + + @Test + public void expectNextCount2() { + Flux flux = Flux.just("foo", "bar"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNext("foo", "bar") + .expectNextCount(2) + .expectComplete() + .verify()) + .withMessage("expectation \"expectNextCount(2)\" failed (expected: count = 2; actual: counted = 0; signal: onComplete())"); + } + + @Test + public void expectNextCount3() { + Flux flux = Flux.just("foo", "bar"); + + StepVerifier.create(flux) + .expectNext("foo") + .expectNextCount(1) + .expectComplete() + .verify(); + } + + @Test + public void expectNextCountZero() { + Flux flux = Flux.empty(); + + StepVerifier.create(flux) + .expectNextCount(0) + .expectComplete() + .verify(); + } + + @Test + public void expectNextCountError() { + Flux flux = Flux.just("foo", "bar"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNextCount(4) + .thenCancel() + .verify()) + .withMessage("expectation \"expectNextCount(4)\" failed (expected: count = 4; actual: counted = 2; signal: onComplete())"); + } + + @Test + public void error() { + Flux flux = Flux.just("foo") + .concatWith(Mono.error(new IllegalArgumentException())); + + StepVerifier.create(flux) + .expectNext("foo") + .expectError() + .verify(); + } + + @Test + public void errorClass() { + Flux flux = Flux.just("foo") + .concatWith(Mono.error(new IllegalArgumentException())); + + StepVerifier.create(flux) + .expectNext("foo") + .expectError(IllegalArgumentException.class) + .verify(); + } + + @Test + public void errorMessage() { + Flux flux = Flux.just("foo") + .concatWith(Mono.error(new IllegalArgumentException( + "Error message"))); + + StepVerifier.create(flux) + .expectNext("foo") + .expectErrorMessage("Error message") + .verify(); + } + + @Test + public void errorMatches() { + Flux flux = Flux.just("foo") + .concatWith(Mono.error(new IllegalArgumentException())); + + StepVerifier.create(flux) + .expectNext("foo") + .expectErrorMatches(t -> t instanceof IllegalArgumentException) + .verify(); + } + + @Test + public void errorMatchesInvalid() { + Flux flux = Flux.just("foo") + .concatWith(Mono.error(new IllegalArgumentException())); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNext("foo") + .expectErrorMatches(t -> t instanceof IllegalStateException) + .verify()) + .withMessage("expectation \"expectErrorMatches\" failed (predicate failed on exception: java.lang.IllegalArgumentException)"); + } + + @Test + public void consumeErrorWith() { + Flux flux = Flux.just("foo") + .concatWith(Mono.error(new IllegalArgumentException())); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNext("foo") + .consumeErrorWith(throwable -> { + if (!(throwable instanceof IllegalStateException)) { + throw new AssertionError(throwable.getClass() + .getSimpleName()); + } + }) + .verify()) + .withMessage("IllegalArgumentException"); + } + + @Test + public void request() { + Flux flux = Flux.just("foo", "bar"); + + StepVerifier.create(flux, 1) + .thenRequest(1) + .expectNext("foo") + .thenRequest(1) + .expectNext("bar") + .expectComplete() + .verify(); + } + + @Test + public void cancel() { + Flux flux = Flux.just("foo", "bar", "baz"); + + StepVerifier.create(flux) + .expectNext("foo") + .thenCancel() + .verify(); + } + + @Test + public void cancelInvalid() { + Flux flux = Flux.just("bar", "baz"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNext("foo") + .thenCancel() + .verify()) + .withMessage("expectation \"expectNext(foo)\" failed (expected value: foo; actual value: bar)"); + } + + @Test + public void subscribedTwice() { + Flux flux = Flux.just("foo", "bar"); + + StepVerifier s = StepVerifier.create(flux) + .expectNext("foo") + .expectNext("bar") + .expectComplete(); + + s.verify(); + s.verify(); + } + + @Test + public void verifyThenOnCompleteRange() { + DirectProcessor p = DirectProcessor.create(); + + Flux flux = Flux.range(0, 3) + .map(d -> "t" + d) + .takeUntilOther(p); + + StepVerifier.create(flux, 2) + .expectNext("t0", "t1") + .then(p::onComplete) + .expectComplete() + .verify(); + + } + + @Test + public void verifyDuration() { + long interval = 200; + Flux flux = Flux.interval(Duration.ofMillis(interval)) + .map(l -> "foo") + .take(2); + + Duration duration = StepVerifier.create(flux) + .thenAwait(Duration.ofSeconds(100)) + .expectNext("foo") + .expectNext("foo") + .expectComplete() + .verify(Duration.ofMillis(500)); + + assertThat(duration.toMillis()).isGreaterThan(2 * interval); + } + + @Test + public void verifyDurationTimeout() { + Flux flux = Flux.interval(Duration.ofMillis(200)) + .map(l -> "foo") + .take(2); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNext("foo") + .expectNext("foo") + .expectComplete() + .verify(Duration.ofMillis(300))) + .withMessageStartingWith("VerifySubscriber timed out on"); + } + + @Test + public void verifyNever() { + Flux flux = Flux.never(); + + StepVerifier.create(flux) + .expectSubscription() + .thenCancel() + .verify(); + } + + @Test + public void verifySubscription() { + Mono flux = Mono.just("foo"); + + StepVerifier.create(flux) + .expectSubscriptionMatches(s -> s instanceof Fuseable.QueueSubscription) + .expectNext("foo") + .expectComplete() + .verify(); + } + + @Test + public void verifyNextAs() { + Flux flux = Flux.just("foo", "bar", "foobar"); + + StepVerifier.create(flux) + .expectNextSequence(Arrays.asList("foo", "bar", "foobar")) + .expectComplete() + .verify(); + } + + @Test + public void verifyNextAsErrorTooFewInIterable() { + Flux flux = Flux.just("foo", "bar", "foobar"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNextSequence(Arrays.asList("foo", "bar")) + .expectComplete() + .verify()) + .withMessage("expectation \"expectComplete\" failed (expected: onComplete(); actual: onNext(foobar))"); + } + + @Test + public void verifyNextAsErrorTooManyInIterable() { + Flux flux = Flux.just("foo", "bar", "foobar"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNextSequence(Arrays.asList("foo", "bar", "foobar", "bar")) + .expectComplete() + .verify()) + .withMessageStartingWith("expectation \"expectNextSequence\" failed (") + .withMessageEndingWith("expected next value: bar; actual signal: onComplete(); iterable: [foo, bar, foobar, bar])"); + } + + @Test + public void verifyNextAs2() { + final List source = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + + Flux flux = Flux.fromStream(source.stream()); + + StepVerifier.create(flux) + .expectNextSequence(source) + .expectComplete() + .verify(); + } + + @Test + public void verifyNextAsWithEmptyFlux() { + final List source = Arrays.asList(1,2,3); + Flux flux = Flux.empty(); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNextSequence(source) + .expectComplete() + .verify()) + .withMessageStartingWith("expectation \"expectNextSequence\" failed (") + .withMessageEndingWith("expected next value: 1; actual signal: onComplete(); iterable: [1, 2, 3])"); + } + + @Test + public void verifyRecordMatches() { + Flux flux = Flux.just("foo", "bar", "foobar"); + + StepVerifier.create(flux) + .recordWith(ArrayList::new) + .expectNextCount(3) + .expectRecordedMatches(c -> c.contains("foobar")) + .expectComplete() + .verify(); + } + + @Test + public void verifyRecordMatchesError() { + Flux flux = Flux.just("foo", "bar", "foobar"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .recordWith(ArrayList::new) + .expectNextCount(3) + .expectRecordedMatches(c -> c.contains("foofoo")) + .expectComplete() + .verify()) + .withMessage("expectation \"expectRecordedMatches\" failed (expected collection predicate match; actual: [foo, bar, foobar])"); + } + + @Test + public void verifyRecordNullError() { + Flux flux = Flux.just("foo", "bar"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .recordWith(() -> null) + .expectComplete() + .verify()) + .withMessage("expectation \"recordWith\" failed (expected collection; actual supplied is [null])"); + } + + @Test + public void verifyRecordMatchesError2() { + Flux flux = Flux.just("foo", "bar", "foobar"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectNext("foo", "bar", "foobar") + .expectRecordedMatches(c -> c.size() == 3) + .expectComplete() + .verify()) + .withMessage("expectation \"expectRecordedMatches\" failed (expected record collector; actual record is [null])"); + } + + @Test + public void verifyRecordWith2() { + final List source = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + + Flux flux = Flux.fromStream(source.stream()); + + StepVerifier.create(flux) + .recordWith(ArrayList::new) + .expectNextCount(10) + .consumeRecordedWith(c -> assertThat(c).containsExactlyElementsOf(source)) + .expectComplete() + .verify(); + } + + @Test + public void verifySubscriptionError() { + Mono flux = Mono.just("foo"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectSubscriptionMatches(s -> false) + .expectNext("foo") + .expectComplete() + .verify()) + .withMessageStartingWith("expectation \"expectSubscriptionMatches\" failed (predicate failed on subscription: "); + } + + @Test + public void verifyConsumeSubscription() { + Mono flux = Mono.just("foo"); + + StepVerifier.create(flux) + .consumeSubscriptionWith(s -> assertThat(s).isInstanceOf(Fuseable.QueueSubscription.class)) + .expectNext("foo") + .expectComplete() + .verify(); + } + + @Test + public void verifyConsumeSubscriptionAfterFirst() { + Mono flux = Mono.just("foo"); + + StepVerifier.create(flux) + .expectNext("foo") + .consumeSubscriptionWith(s -> assertThat(s).isInstanceOf(Fuseable.QueueSubscription.class)) + .expectComplete() + .verify(); + } + + @Test + public void verifyConsumeSubscriptionError() { + Mono flux = Mono.just("foo"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .consumeSubscriptionWith(s -> Assertions.fail("boom")) + .expectNext("foo") + .expectComplete() + .verify()) + .withMessage("boom"); + } + + @Test + public void verifyFusion() { + Mono flux = Mono.just("foo"); + + StepVerifier.create(flux) + .expectFusion() + .expectNext("foo") + .expectComplete() + .verify(); + } + + @Test + public void verifyFusionError() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> Mono.just("foo") + .hide() + .as(StepVerifier::create) + .expectFusion() + .expectNext("foo") + .expectComplete() + .verify()) + .withMessage("expectation failed (expected fuseable source but actual " + + "Subscription is not: 3)"); + } + + @Test + public void verifyNoFusion() { + Mono flux = Mono.just("foo") + .hide(); + + StepVerifier.create(flux) + .expectNoFusionSupport() + .expectNext("foo") + .expectComplete() + .verify(); + } + + @Test + public void verifyNoFusionError() { + Mono flux = Mono.just("foo"); + + StepVerifier.create(flux.hide()) + .expectNoFusionSupport() + .expectNext("foo") + .expectComplete() + .verify(); + } + + @Test + public void verifyFusionModeRequest() { + Mono flux = Mono.just("foo"); + + StepVerifier.create(flux) + .expectFusion(Fuseable.SYNC) + .expectNext("foo") + .expectComplete() + .verify(); + } + + @Test + public void verifyFusionModeExpected() { + Mono flux = Mono.just("foo"); + + StepVerifier.create(flux) + .expectFusion(Fuseable.SYNC, Fuseable.SYNC) + .expectNext("foo") + .expectComplete() + .verify(); + } + + @Test + public void verifyFusionModeExpectedError() { + Mono flux = Mono.just("foo"); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectFusion(Fuseable.SYNC, Fuseable.ASYNC) + .expectNext("foo") + .expectComplete() + .verify()) + .withMessage("expectation failed (expected fusion mode: (async); actual: (sync))"); + } + + @Test + public void verifyFusionModeExpected2() { + Flux flux = Flux.just("foo", "bar") + .publishOn(Schedulers.immediate()); + + StepVerifier.create(flux) + .expectFusion(Fuseable.SYNC | Fuseable.ASYNC, Fuseable.ASYNC) + .expectNext("foo", "bar") + .expectComplete() + .verify(); + } + + @Test + public void verifyFusionModeExpectedCancel() { + Flux flux = Flux.just("foo", "bar"); + + StepVerifier.create(flux) + .expectFusion(Fuseable.SYNC, Fuseable.SYNC) + .expectNext("foo") + .thenCancel() + .verify(); + } + + @Test + public void verifyFusionModeExpected2Error() { + Flux flux = Flux.just("foo", "bar") + .publishOn(Schedulers.immediate()); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(flux) + .expectFusion(Fuseable.ASYNC, Fuseable.SYNC) + .expectNext("foo", "bar") + .expectComplete() + .verify()) + .withMessage("expectation failed (expected fusion mode: (sync); actual: (async))"); + } + + @Test + public void verifyVirtualTimeOnSubscribe() { + StepVerifier.withVirtualTime(() -> Mono.delay(Duration.ofDays(2)) + .map(l -> "foo")) + .thenAwait(Duration.ofDays(3)) + .expectNext("foo") + .expectComplete() + .verify(); + } + + @Test + public void verifyVirtualTimeOnError() { + StepVerifier.withVirtualTime(() -> Mono.never() + .timeout(Duration.ofDays(2)) + .map(l -> "foo")) + .thenAwait(Duration.ofDays(2)) + .expectError(TimeoutException.class) + .verify(); + } + + @Test + public void verifyVirtualTimeNoEvent() { + StepVerifier.withVirtualTime(() -> Mono.just("foo") + .delaySubscription(Duration.ofDays(2))) + .expectSubscription() + .expectNoEvent(Duration.ofDays(2)) + .expectNext("foo") + .expectComplete() + .verify(); + } + + @Test + public void verifyVirtualTimeNoEventError() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.withVirtualTime(() -> Mono.just("foo") + .delaySubscription(Duration.ofDays(2))) + .expectSubscription() + .expectNoEvent(Duration.ofDays(2)) + .expectNext("foo") + .expectNoEvent(Duration.ofDays(2)) + .expectComplete() + .verify()) + .withMessage("unexpected end during a no-event expectation"); + } + + @Test + public void verifyVirtualTimeNoEventInterval() { + StepVerifier.withVirtualTime(() -> Flux.interval(Duration.ofSeconds(3)) + .take(2)) + .expectSubscription() + .expectNoEvent(Duration.ofSeconds(3)) + .expectNext(0L) + .expectNoEvent(Duration.ofSeconds(3)) + .expectNext(1L) + .expectComplete() + .verify(); + } + + @Test + //TODO shouldn't there be only one error rather than Multiple exceptions? + public void verifyVirtualTimeNoEventIntervalError() { + Throwable thrown = catchThrowable(() -> + StepVerifier.withVirtualTime(() -> Flux.interval(Duration.ofSeconds(3)) + .take(2)) + .expectSubscription() + .expectNoEvent(Duration.ofSeconds(3)) + .expectNext(0L) + .expectNoEvent(Duration.ofSeconds(4)) + .expectNext(1L) + .thenAwait() + .expectComplete() + .verify()); + + assertThat(thrown).isInstanceOf(AssertionError.class) + .hasMessageContaining("Multiple exceptions") + .hasMessageContaining("expectation failed (expected no event: onNext(1)") + .hasMessageContaining("expectation failed (expected no event: onComplete()"); + } + + @Test + public void verifyVirtualTimeNoEventNever() { + StepVerifier.withVirtualTime(() -> Mono.never() + .log()) + .expectSubscription() + .expectNoEvent(Duration.ofDays(10000)) + .thenCancel() + .verify(); + } + + @Test + public void verifyVirtualTimeNoEventNeverError() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.withVirtualTime(() -> Mono.never() + .log()) + .expectNoEvent(Duration.ofDays(10000)) + .thenCancel() + .verify()) + .withMessageStartingWith("expectation failed (expected no event: onSubscribe("); + } + + @Test + public void verifyVirtualTimeOnNext() { + StepVerifier.withVirtualTime(() -> Flux.just("foo", "bar", "foobar") + .delayElements(Duration.ofHours(1)) + .log()) + .thenAwait(Duration.ofHours(1)) + .expectNext("foo") + .thenAwait(Duration.ofHours(1)) + .expectNext("bar") + .thenAwait(Duration.ofHours(1)) + .expectNext("foobar") + .expectComplete() + .verify(); + } + + @Test + public void verifyVirtualTimeOnComplete() { + StepVerifier.withVirtualTime(() -> Flux.empty() + .delaySubscription(Duration.ofHours(1)) + .log()) + .thenAwait(Duration.ofHours(1)) + .expectComplete() + .verify(); + } + + @Test + public void verifyVirtualTimeOnNextInterval() { + Duration r; + + r = StepVerifier.withVirtualTime(() -> Flux.interval(Duration.ofSeconds(3)) + .map(d -> "t" + d)) + .thenAwait(Duration.ofSeconds(3)) + .expectNext("t0") + .thenAwait(Duration.ofSeconds(3)) + .expectNext("t1") + .thenAwait(Duration.ofSeconds(3)) + .expectNext("t2") + .thenCancel() + .verify(); + + assertThat(r.minus(Duration.ofSeconds(9)).toMillis()).isNegative(); + } + + @Test + public void verifyVirtualTimeNoLookupFails() { + assertThatExceptionOfType(NullPointerException.class) + .isThrownBy(() -> StepVerifier.withVirtualTime(Flux::empty, null, 1)) + .withMessage("vtsLookup"); + } + + @Test + public void verifyVirtualTimeNoScenarioFails() { + assertThatExceptionOfType(NullPointerException.class) + .isThrownBy(() -> StepVerifier.withVirtualTime(null, 1)) + .withMessage("scenarioSupplier"); + } + + @Test(timeout = 3000) + public void verifyVirtualTimeOnNextIntervalManual() { + VirtualTimeScheduler vts = VirtualTimeScheduler.create(); + + StepVerifier.withVirtualTime(() -> Flux.interval(Duration.ofMillis(1000), vts) + .map(d -> "t" + d)) + .then(() -> vts.advanceTimeBy(Duration.ofHours(1))) + .expectNextCount(3600) + .thenCancel() + .verify(); + } + + @Test + public void verifyVirtualTimeOnErrorInterval() { + StepVerifier.withVirtualTime(() -> Flux.interval(Duration.ofSeconds(3)) + .map(d -> "t" + d), + 0) + .thenRequest(1) + .thenAwait(Duration.ofSeconds(3)) + .expectNext("t0") + .thenRequest(1) + .thenAwait(Duration.ofSeconds(3)) + .expectNext("t1") + .thenAwait(Duration.ofSeconds(3)) + .expectError(IllegalStateException.class) + .verify(); + + } + + @Test + public void verifyVirtualTimeOnErrorAsync() { + VirtualTimeScheduler vts = VirtualTimeScheduler.create(); + StepVerifier.withVirtualTime(() -> Flux.just(123) + .subscribeOn(vts), + () -> vts, 0) + .thenRequest(1) + .thenAwait() + .expectNext(123) + .expectComplete() + .verify(); + + } + + @Test(timeout = 1000) + public void verifyCreatedForAllSchedulerUsesVirtualTime() { + //a timeout will occur if virtual time isn't used + StepVerifier.withVirtualTime(() -> Flux.interval(Duration.ofSeconds(3)) + .map(d -> "t" + d), + VirtualTimeScheduler::create, + 0) + .thenRequest(1) + .thenAwait(Duration.ofSeconds(1)) + .thenAwait(Duration.ofSeconds(2)) + .expectNext("t0") + .thenCancel() + .verify(); + } + + @Test + public void noSignalRealTime() { + Duration verifyDuration = StepVerifier.create(Mono.never()) + .expectSubscription() + .expectNoEvent(Duration.ofSeconds(1)) + .thenCancel() + .verify(Duration.ofMillis(1100)); + + assertThat(verifyDuration.toMillis()).isGreaterThanOrEqualTo(1000L); + } + + @Test(timeout = 500) + public void noSignalVirtualTime() { + StepVerifier.withVirtualTime(Mono::never, 1) + .expectSubscription() + .expectNoEvent(Duration.ofSeconds(100)) + .thenCancel() + .verify(); + } + + @Test + public void longDelayAndNoTermination() { + StepVerifier.withVirtualTime(() -> Flux.just("foo", "bar") + .delayElements(Duration.ofSeconds(5)) + .concatWith(Mono.never()), + Long.MAX_VALUE) + .expectSubscription() + .expectNoEvent(Duration.ofSeconds(5)) + .expectNext("foo") + .expectNoEvent(Duration.ofSeconds(5)) + .expectNextCount(1) + .expectNoEvent(Duration.ofMillis(10)) + .thenCancel() + .verify(); + } + + @Test + public void thenAwaitThenCancelWaitsForDuration() { + Duration verifyDuration = StepVerifier.create(Flux.just("foo", "bar") + .delayElements(Duration.ofMillis(500))) + .expectSubscription() + .thenAwait(Duration.ofMillis(500)) + .expectNext("foo") + .thenAwait(Duration.ofMillis(200)) + .thenCancel() + .verify(Duration.ofMillis(1000)); + + assertThat(verifyDuration.toMillis()).isGreaterThanOrEqualTo(700L); + } + + @Test + public void testThenConsumeWhile() { + StepVerifier.create(Flux.range(3, 8)) + .expectNextMatches(first -> first == 3) + .thenConsumeWhile(v -> v < 9) + .expectNext(9) + .expectNext(10) + .expectComplete() + .log() + .verify(); + } + + @Test + public void testThenConsumeWhileWithConsumer() { + LongAdder count = new LongAdder(); + + StepVerifier.create(Flux.range(3, 8)) + .expectNextMatches(first -> first == 3) + .thenConsumeWhile(v -> v < 9, v -> count.increment()) + .expectNext(9) + .expectNext(10) + .expectComplete() + .log() + .verify(); + + assertThat(count.intValue()).isEqualTo(5); + } + + @Test + public void testThenConsumeWhileFails() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.range(3, 8)) + .expectNextMatches(first -> first == 3) + .thenConsumeWhile(v -> v <= 9) + .expectNext(9) + .expectNext(10) + .expectComplete() + .log() + .verify()) + .withMessageContaining("expectNext(9)"); + } + + @Test + public void testWithDescription() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.just("foo", "bar", "baz"), 3) + .expectNext("foo") + .as("first") + .expectNext("bar") + .as("second") + .expectNext("bar") + .as("third") + .as("this is ignored") + .expectComplete() + .log() + .verify()) + .withMessageStartingWith("expectation \"third\" failed"); + } + + @Test + public void noCancelOnUnexpectedErrorSignal() { + LongAdder cancelled = new LongAdder(); + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.error(new IllegalArgumentException()) + .doOnCancel(cancelled::increment)) + .expectComplete() + .verify()) + .withMessageContaining("expected: onComplete(); actual: onError"); + assertThat(cancelled.intValue()) + .overridingErrorMessage("the expectComplete assertion caused a cancellation") + .isZero(); + } + + @Test + public void noCancelOnUnexpectedCompleteSignal() { + LongAdder cancelled = new LongAdder(); + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.empty() + .doOnCancel(cancelled::increment)) + .expectError() + .verify()) + .withMessageContaining("expected: onError(); actual: onComplete()"); + assertThat(cancelled.intValue()) + .overridingErrorMessage("the expectError assertion caused a cancellation") + .isZero(); + } + + @Test + public void noCancelOnUnexpectedCompleteSignal2() { + LongAdder cancelled = new LongAdder(); + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.just("foo") + .doOnCancel(cancelled::increment)) + .expectNext("foo", "bar") + .expectComplete() + .verify()) + .withMessageContaining("expected: onNext(bar); actual: onComplete()"); + + assertThat(cancelled.intValue()) + .overridingErrorMessage("the expectNext assertion caused a cancellation") + .isZero(); + } + + @Test + public void noCancelOnCompleteWhenSequenceUnexpected() { + LongAdder cancelled = new LongAdder(); + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.just("foo") + .doOnCancel(cancelled::increment)) + .expectNextSequence(Arrays.asList("foo", "bar")) + .expectComplete() + .verify()) + .withMessageContaining("expectNextSequence"); + assertThat(cancelled.intValue()) + .overridingErrorMessage("the expectNextSequence assertion caused a cancellation") + .isZero(); + } + + @Test + public void noCancelOnCompleteWhenCountUnexpected() { + LongAdder cancelled = new LongAdder(); + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.just("foo") + .doOnCancel(cancelled::increment)) + .expectNextCount(2) + .expectComplete() + .verify()) + .withMessageContaining("expectNextCount"); + + assertThat(cancelled.intValue()) + .overridingErrorMessage("the expectNextCount assertion caused a cancellation") + .isZero(); + } + + @Test + public void noCancelOnErrorWhenCollectUnexpected() { + LongAdder cancelled = new LongAdder(); + LongAdder records = new LongAdder(); + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.error(new IllegalArgumentException()) + .doOnCancel(cancelled::increment)) + .recordWith(() -> { + records.increment(); + return new ArrayList<>(); + }) + .expectRecordedMatches(l -> l.size() == 2) + .expectComplete() + .verify()) + .withMessageContaining("expected collection predicate match"); + + assertThat(cancelled.intValue()) + .overridingErrorMessage("the expectRecordedMatches assertion caused a cancellation") + .isZero(); + assertThat(records.intValue()) + .as("unexpected number of records") + .isEqualTo(1); + } + + //TODO records: find a way to test the case where supplied collection is null, and signal is complete/error + //TODO records: find a way to test the case where there hasn't been a recorder set, and signal is complete/error + + @Test + public void cancelOnUnexpectedNextWithMoreData() { + LongAdder cancelled = new LongAdder(); + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.just("foo", "bar") + .doOnCancel(cancelled::increment)) + .expectNext("baz") + .expectComplete() + .verify()) + .withMessageContaining("expected value: baz;"); + + assertThat(cancelled.intValue()) + .overridingErrorMessage("the expectNext assertion didn't cause a cancellation") + .isEqualTo(1); + } + + @Test + public void boundedInitialOverflowIsDetected() { + TestPublisher publisher = TestPublisher.createNoncompliant( + REQUEST_OVERFLOW); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(publisher, 1) + .then(() -> publisher.emit("foo", "bar")) + .expectNext("foo") + .expectComplete() + .verify()) + .withMessageStartingWith("request overflow (") + .withMessageEndingWith("expected production of at most 1;" + + " produced: 2; request overflown by signal: onNext(bar))"); + } + + @Test + public void boundedRequestOverflowIsDetected() { + TestPublisher publisher = TestPublisher.createNoncompliant( + REQUEST_OVERFLOW); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(publisher, 0) + .thenRequest(2) + .then(() -> publisher.emit("foo", "bar", "baz")) + .expectNext("foo", "bar") + .expectComplete() + .verify()) + .withMessageStartingWith("request overflow (") + .withMessageEndingWith("expected production of at most 2;" + + " produced: 3; request overflown by signal: onNext(baz))"); + } + + @Test + public void initialBoundedThenUnboundedRequestDoesntOverflow() { + TestPublisher publisher = TestPublisher.createNoncompliant( + REQUEST_OVERFLOW); + + StepVerifier.create(publisher, 2) + .thenRequest(Long.MAX_VALUE - 2) + .then(() -> publisher.emit("foo", "bar", "baz")) + .expectNext("foo", "bar", "baz") + .expectComplete() + .verify(); + } + + @Test + public void verifyErrorTriggersVerificationFail() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.empty()) + .verifyError()) + .withMessage("expectation \"expectError()\" failed (expected: onError(); actual: onComplete())"); + } + + @Test + public void verifyErrorTriggersVerificationSuccess() { + StepVerifier.create(Flux.error(new IllegalArgumentException())) + .verifyError(); + } + + @Test + public void verifyErrorClassTriggersVerificationFail() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.empty()) + .verifyError(IllegalArgumentException.class)) + .withMessage("expectation \"expectError(Class)\" failed (expected: onError(IllegalArgumentException); actual: onComplete())"); + } + + @Test + public void verifyErrorClassTriggersVerificationSuccess() { + StepVerifier.create(Flux.error(new IllegalArgumentException())) + .verifyError(IllegalArgumentException.class); + } + + @Test + public void verifyErrorMessageTriggersVerificationFail() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.empty()) + .verifyErrorMessage("boom")) + .withMessage("expectation \"expectErrorMessage\" failed (expected: onError(\"boom\"); actual: onComplete())"); + } + + @Test + public void verifyErrorMessageTriggersVerificationSuccess() { + StepVerifier.create(Flux.error(new IllegalArgumentException("boom"))) + .verifyErrorMessage("boom"); + } + + @Test + public void verifyErrorPredicateTriggersVerificationFail() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.empty()) + .verifyErrorMatches(e -> e instanceof IllegalArgumentException)) + .withMessage("expectation \"expectErrorMatches\" failed (expected: onError(); actual: onComplete())"); + } + + @Test + public void verifyErrorPredicateTriggersVerificationSuccess() { + StepVerifier.create(Flux.error(new IllegalArgumentException("boom"))) + .verifyErrorMatches(e -> e instanceof IllegalArgumentException); + } + + @Test + public void verifyCompleteTriggersVerificationFail() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.error(new IllegalArgumentException())) + .verifyComplete()) + .withMessage("expectation \"expectComplete\" failed (expected: onComplete(); actual: onError(java.lang.IllegalArgumentException))"); + } + + @Test + public void verifyCompleteTriggersVerificationSuccess() { + StepVerifier.create(Flux.just(1, 2)) + .expectNext(1, 2) + .verifyComplete(); + } + + @Test + public void expectNextCountAfterExpectNext() { + StepVerifier.create(Flux.range(1, 5)) + .expectNext(1, 2) + .expectNextCount(3) + .verifyComplete(); + } + + @Test + public void expectNextCountAfterThenConsumeWhile() { + StepVerifier.create(Flux.range(1, 5).log()) + .thenConsumeWhile(i -> i <= 2) + .expectNextCount(3) + .verifyComplete(); + } + + @Test + public void expectNextCountAfterExpectNextCount() { + StepVerifier.create(Flux.range(1, 5)) + .expectNextCount(2) + .expectNextCount(3) + .verifyComplete(); + } + + @Test + public void expectNextCountAfterExpectNextMatches() { + StepVerifier.create(Flux.range(1, 5)) + .expectNextMatches(i -> true) + .expectNextMatches(i -> true) + .expectNextCount(3) + .verifyComplete(); + } + + @Test + public void expectNextCountAfterExpectNextSequence() { + StepVerifier.create(Flux.range(1, 5)) + .expectNextSequence(Arrays.asList(1, 2)) + .expectNextCount(3) + .verifyComplete(); + } + + @Test + public void expectNextCountAfterConsumeNextWith() { + StepVerifier.create(Flux.range(1, 5)) + .consumeNextWith(i -> {}) + .consumeNextWith(i -> {}) + .expectNextCount(3) + .verifyComplete(); + } + + @Test + public void expectNextSequenceWithPartialMatchingSequence() { + StepVerifier.create(Flux.range(1, 5)) + .expectNextSequence(Arrays.asList(1, 2, 3)) + .expectNext(4, 5) + .verifyComplete(); + } + + @Test + public void expectNextSequenceWithPartialMatchingSequenceNoMoreExpectation() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.range(1, 5)) + .expectNextSequence(Arrays.asList(1, 2, 3)) + .verifyComplete()) + .withMessage("expectation \"expectComplete\" failed (expected: onComplete(); actual: onNext(4))"); + } + + @Test + public void expectNextErrorIsSuppressed() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.just("foo") + .flatMap(r -> { throw new ArrayIndexOutOfBoundsException();})) + .expectNext("foo") + .verifyError()) + .satisfies(error -> { + assertThat(error) + .hasMessageStartingWith("expectation \"expectNext(foo)\" failed") + .hasMessageContaining("actual: onError(java.lang.ArrayIndexOutOfBoundsException)"); + assertThat(error.getSuppressed()) + .hasSize(1) + .allMatch(spr -> spr instanceof ArrayIndexOutOfBoundsException); + }); + } + + @Test + public void consumeNextErrorIsSuppressed() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.just("foo") + .flatMap(r -> { throw new ArrayIndexOutOfBoundsException();})) + .consumeNextWith(v -> assertThat(v).isNotNull()) + .verifyError()) + .satisfies(error -> { + assertThat(error) + .hasMessageStartingWith("expectation \"consumeNextWith\" failed") + .hasMessageContaining("actual: onError(java.lang.ArrayIndexOutOfBoundsException)"); + assertThat(error.getSuppressed()) + .hasSize(1) + .allMatch(spr -> spr instanceof ArrayIndexOutOfBoundsException); + }); + } + + @Test + public void expectNextCountErrorIsSuppressed() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.just("foo") + .flatMap(r -> { throw new ArrayIndexOutOfBoundsException();})) + .expectNextCount(1) + .verifyError()) + .satisfies(error -> { + assertThat(error) + .hasMessageStartingWith("expectation \"expectNextCount(1)\" failed") + .hasMessageContaining("signal: onError(java.lang.ArrayIndexOutOfBoundsException)"); + assertThat(error.getSuppressed()) + .hasSize(1) + .allMatch(spr -> spr instanceof ArrayIndexOutOfBoundsException); + }); + } + + @Test + public void expectNextSequenceErrorIsSuppressed() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.just("foo") + .flatMap(r -> { throw new ArrayIndexOutOfBoundsException();})) + .expectNextSequence(Arrays.asList("foo")) + .verifyError()) + .satisfies(error -> { + assertThat(error) + .hasMessageStartingWith("expectation \"expectNextSequence\" failed") + .hasMessageContaining("actual signal: onError(java.lang.ArrayIndexOutOfBoundsException)"); + assertThat(error.getSuppressed()) + .hasSize(1) + .allMatch(spr -> spr instanceof ArrayIndexOutOfBoundsException); + }); + } + + @Test + public void consumeWhileErrorIsSuppressed() { + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.just("foo", "bar", "foobar") + .map(r -> { if (r.length() > 3) throw new ArrayIndexOutOfBoundsException(); return r;})) + .thenConsumeWhile(s -> s.length() <= 3) //doesn't fail by itself... + .verifyComplete()) //...so this will fail + .satisfies(error -> { + assertThat(error) + .hasMessageStartingWith("expectation \"expectComplete\" failed") + .hasMessageContaining("actual: onError(java.lang.ArrayIndexOutOfBoundsException)"); + assertThat(error.getSuppressed()) + .hasSize(1) + .allMatch(spr -> spr instanceof ArrayIndexOutOfBoundsException); + }); + } + + @Test + @SuppressWarnings("unchecked") + public void requestBufferDoesntOverflow() { + LongAdder requestCallCount = new LongAdder(); + LongAdder totalRequest = new LongAdder(); + Flux source = Flux.range(1, 10).hide() + .doOnRequest(r -> requestCallCount.increment()) + .doOnRequest(totalRequest::add); + + StepVerifier.withVirtualTime(//start with a request for 1 buffer + () -> source.bufferUntil(i -> i % 3 == 0), 1) + .expectSubscription() + .expectNext(Arrays.asList(1, 2, 3)) + .expectNoEvent(Duration.ofSeconds(1)) + .thenRequest(2) + .expectNext(Arrays.asList(4, 5, 6), Arrays.asList(7, 8, 9)) + .expectNoEvent(Duration.ofSeconds(1)) + .thenRequest(3) + .expectNext(Collections.singletonList(10)) + .expectComplete() + .verify(); + + assertThat(requestCallCount.intValue()).isEqualTo(11); //10 elements then the completion + assertThat(totalRequest.longValue()).isEqualTo(11L); //ignores the main requests + } + + @Test(timeout = 1000L) + public void expectCancelDoNotHang() { + StepVerifier.create(Flux.just("foo", "bar"), 1) + .expectNext("foo") + .thenCancel() + .verify(); + } + + @Test(timeout = 1000L) + public void consumeNextWithLowRequestShortcircuits() { + StepVerifier.Step validSoFar = StepVerifier.create(Flux.just("foo", "bar"), 1) + .expectNext("foo"); + + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> validSoFar.consumeNextWith(s -> {})) + .withMessageStartingWith("The scenario will hang at consumeNextWith due to too little request being performed for the expectations to finish") + .withMessageEndingWith("request remaining since last step: 0, expected: 1"); + } + + @Test(timeout = 1000L) + public void assertNextLowRequestShortcircuits() { + StepVerifier.Step validSoFar = StepVerifier.create(Flux.just("foo", "bar"), 1) + .expectNext("foo"); + + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> validSoFar.assertNext(s -> {})) + .withMessageStartingWith("The scenario will hang at assertNext due to too little request being performed for the expectations to finish") + .withMessageEndingWith("request remaining since last step: 0, expected: 1"); + } + + @Test(timeout = 1000L) + public void expectNextLowRequestShortcircuits() { + StepVerifier.Step validSoFar = StepVerifier.create(Flux.just("foo", "bar"), 1) + .expectNext("foo"); + + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> validSoFar.expectNext("bar")) + .withMessageStartingWith("The scenario will hang at expectNext(bar) due to too little request being performed for the expectations to finish") + .withMessageEndingWith("request remaining since last step: 0, expected: 1"); + } + + @Test(timeout = 1000L) + public void expectNextCountLowRequestShortcircuits() { + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> StepVerifier.create(Flux.just("foo", "bar"), 1) + .expectNextCount(2) + ) + .withMessageStartingWith("The scenario will hang at expectNextCount(2) due to too little request being performed for the expectations to finish; ") + .withMessageEndingWith("request remaining since last step: 1, expected: 2"); + } + + @Test(timeout = 1000L) + public void expectNextMatchesLowRequestShortcircuits() { + StepVerifier.Step validSoFar = StepVerifier.create(Flux.just("foo", "bar"), 1) + .expectNext("foo"); + + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> validSoFar.expectNextMatches("bar"::equals)) + .withMessageStartingWith("The scenario will hang at expectNextMatches due to too little request being performed for the expectations to finish") + .withMessageEndingWith("request remaining since last step: 0, expected: 1"); + } + + @Test(timeout = 1000L) + public void expectNextSequenceLowRequestShortcircuits() { + StepVerifier.Step validSoFar = StepVerifier.create(Flux.just("foo", "bar"), 1); + List expected = Arrays.asList("foo", "bar"); + + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> validSoFar.expectNextSequence(expected)) + .withMessageStartingWith("The scenario will hang at expectNextSequence due to too little request being performed for the expectations to finish") + .withMessageEndingWith("request remaining since last step: 1, expected: 2"); + } + + @Test(timeout = 1000L) + public void thenConsumeWhileLowRequestShortcircuits() { + StepVerifier.Step validSoFar = StepVerifier.create(Flux.just(1, 2), 1) + .expectNext(1); + + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> validSoFar.thenConsumeWhile(s -> s == 1)) + .withMessageStartingWith("The scenario will hang at thenConsumeWhile due to too little request being performed for the expectations to finish; ") + .withMessageEndingWith("request remaining since last step: 0, expected: at least 1 (best effort estimation)"); + } + + @Test(timeout = 1000L) + public void lowRequestCheckCanBeDisabled() { + StepVerifier.create(Flux.just(1, 2), + StepVerifierOptions.create().initialRequest(1).checkUnderRequesting(false)) + .expectNext(1) + .thenConsumeWhile(s -> s == 1); //don't verify, this alone would throw an exception if check activated + } + + @Test + public void takeAsyncFusedBackpressured() { + UnicastProcessor up = UnicastProcessor.create(); + StepVerifier.create(up.take(3), 0) + .expectFusion() + .then(() -> up.onNext("test")) + .then(() -> up.onNext("test")) + .then(() -> up.onNext("test")) + .thenRequest(2) + .expectNext("test", "test") + .thenRequest(1) + .expectNext("test") + .verifyComplete(); + } + + @Test + public void cancelAsyncFusion() { + UnicastProcessor up = UnicastProcessor.create(); + StepVerifier.create(up.take(3), 0) + .expectFusion() + .then(() -> up.onNext("test")) + .then(() -> up.onNext("test")) + .then(() -> up.onNext("test")) + .thenRequest(2) + .expectNext("test", "test") + .thenCancel() + .verify(); + } + + @Test + public void virtualTimeSchedulerUseExactlySupplied() { + VirtualTimeScheduler vts1 = VirtualTimeScheduler.create(); + VirtualTimeScheduler vts2 = VirtualTimeScheduler.create(); + VirtualTimeScheduler.getOrSet(vts1); + + StepVerifier.withVirtualTime(Mono::empty, () -> vts2, Long.MAX_VALUE) + .then(() -> assertThat(VirtualTimeScheduler.get()).isSameAs(vts2)) + .verifyComplete(); + + assertThat(vts1.isDisposed()).isFalse(); + assertThat(vts2.isDisposed()).isTrue(); + assertThat(VirtualTimeScheduler.isFactoryEnabled()).isFalse(); + } +} \ No newline at end of file diff --git a/reactor-test/src/test/java/reactor/test/publisher/DefaultTestPublisherTests.java b/reactor-test/src/test/java/reactor/test/publisher/DefaultTestPublisherTests.java new file mode 100644 index 0000000000..1dc9c595eb --- /dev/null +++ b/reactor-test/src/test/java/reactor/test/publisher/DefaultTestPublisherTests.java @@ -0,0 +1,282 @@ +/* + * Copyright (c) 2011-2017 Pivotal Software Inc, All Rights Reserved. + * + * 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 + * + * http://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 reactor.test.publisher; + +import java.util.concurrent.atomic.AtomicLong; + +import org.junit.Test; +import org.reactivestreams.Subscriber; +import org.reactivestreams.Subscription; +import reactor.core.Disposable; +import reactor.core.publisher.Flux; +import reactor.test.StepVerifier; +import reactor.test.publisher.TestPublisher.Violation; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + +public class DefaultTestPublisherTests { + + @Test + public void normalDisallowsNull() { + TestPublisher publisher = TestPublisher.create(); + + assertThatExceptionOfType(NullPointerException.class) + .isThrownBy(() -> publisher.next(null)) + .withMessage("emitted values must be non-null"); + } + + @Test + public void misbehavingAllowsNull() { + TestPublisher publisher = TestPublisher.createNoncompliant(Violation.ALLOW_NULL); + + StepVerifier.create(publisher) + .then(() -> publisher.emit("foo", null)) + .expectNext("foo", null) + .expectComplete() + .verify(); + } + + @Test + public void normalDisallowsOverflow() { + TestPublisher publisher = TestPublisher.create(); + + StepVerifier.create(publisher, 1) + .then(() -> publisher.next("foo")).as("should pass") + .then(() -> publisher.emit("bar")).as("should fail") + .expectNext("foo") + .expectErrorMatches(e -> e instanceof IllegalStateException && + "Can't deliver value due to lack of requests".equals(e.getMessage())) + .verify(); + + publisher.assertNoRequestOverflow(); + } + + @Test + public void misbehavingAllowsOverflow() { + TestPublisher publisher = TestPublisher.createNoncompliant(Violation.REQUEST_OVERFLOW); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(publisher, 1) + .then(() -> publisher.emit("foo", "bar")) + .expectNext("foo") + .expectComplete() //n/a + .verify()) + .withMessageContaining("expected production of at most 1;"); + + publisher.assertRequestOverflow(); + } + + @Test + public void normalIgnoresMultipleTerminations() { + TestPublisher publisher = TestPublisher.create(); + AtomicLong count = new AtomicLong(); + + Subscriber subscriber = new Subscriber() { + @Override + public void onSubscribe(Subscription s) { } + + @Override + public void onNext(String s) { } + + @Override + public void onError(Throwable t) { + count.incrementAndGet(); + } + + @Override + public void onComplete() { + count.incrementAndGet(); + } + }; + + publisher.subscribe(subscriber); + publisher.complete() + .emit("A", "B", "C") + .error(new IllegalStateException("boom")); + + assertThat(count.get()).isEqualTo(1); + } + + @Test + public void misbehavingAllowsMultipleTerminations() { + TestPublisher publisher = TestPublisher.createNoncompliant(Violation.CLEANUP_ON_TERMINATE); + AtomicLong count = new AtomicLong(); + + Subscriber subscriber = new Subscriber() { + @Override + public void onSubscribe(Subscription s) { + s.request(Long.MAX_VALUE); + s.cancel(); + } + + @Override + public void onNext(String s) { } + + @Override + public void onError(Throwable t) { + count.incrementAndGet(); + } + + @Override + public void onComplete() { + count.incrementAndGet(); + } + }; + + publisher.subscribe(subscriber); + + publisher.error(new IllegalStateException("boom")) + .complete(); + + publisher.emit("A", "B", "C"); + + assertThat(count.get()).isEqualTo(3); + publisher.assertCancelled(); + } + + @Test + public void expectSubscribers() { + TestPublisher publisher = TestPublisher.create(); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(publisher::assertSubscribers) + .withMessage("Expected subscribers"); + + StepVerifier.create(publisher) + .then(() -> publisher.assertSubscribers() + .complete()) + .expectComplete() + .verify(); + } + + @Test + public void expectSubscribersN() { + TestPublisher publisher = TestPublisher.create(); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> publisher.assertSubscribers(1)) + .withMessage("Expected 1 subscribers, got 0"); + + publisher.assertNoSubscribers(); + Flux.from(publisher).subscribe(); + publisher.assertSubscribers(1); + Flux.from(publisher).subscribe(); + publisher.assertSubscribers(2); + + publisher.complete() + .assertNoSubscribers(); + } + + @Test + public void expectCancelled() { + TestPublisher publisher = TestPublisher.create(); + StepVerifier.create(publisher) + .then(publisher::assertNotCancelled) + .thenCancel() + .verify(); + publisher.assertCancelled(); + + StepVerifier.create(publisher) + .then(() -> publisher.assertCancelled(1)) + .thenCancel() + .verify(); + publisher.assertCancelled(2); + } + + @Test + public void expectMinRequestedNormal() { + TestPublisher publisher = TestPublisher.create(); + + StepVerifier.create(Flux.from(publisher).limitRate(5)) + .then(publisher::assertNotCancelled) + .then(() -> publisher.assertMinRequested(5)) + .thenCancel() + .verify(); + publisher.assertCancelled(); + publisher.assertNoSubscribers(); + publisher.assertMinRequested(0); + } + + @Test + public void expectMinRequestedFailure() { + TestPublisher publisher = TestPublisher.create(); + + assertThatExceptionOfType(AssertionError.class) + .isThrownBy(() -> StepVerifier.create(Flux.from(publisher).limitRate(5)) + .then(() -> publisher.assertMinRequested(6) + .emit("foo")) + .expectNext("foo").expectComplete() // N/A + .verify()) + .withMessageContaining("Expected minimum request of 6; got 5"); + + publisher.assertCancelled(); + publisher.assertNoSubscribers(); + publisher.assertMinRequested(0); + } + + @Test + public void emitCompletes() { + TestPublisher publisher = TestPublisher.create(); + StepVerifier.create(publisher) + .then(() -> publisher.emit("foo", "bar")) + .expectNextCount(2) + .expectComplete() + .verify(); + } + + @Test + public void nextVarargNull() { + TestPublisher publisher = TestPublisher.create(); + + assertThatExceptionOfType(NullPointerException.class) + .isThrownBy(() -> publisher.next(null, null)) //this causes a compiler warning, on purpose + .withMessage("rest array is null, please cast to T if null T required"); + } + + @Test + public void emitVarargNull() { + TestPublisher publisher = TestPublisher.create(); + + assertThatExceptionOfType(NullPointerException.class) + .isThrownBy(() -> publisher.emit(null)) //this causes a compiler warning, on purpose + .withMessage("values array is null, please cast to T if null T required"); + } + + @Test + public void testError() { + TestPublisher publisher = TestPublisher.create(); + StepVerifier.create(publisher) + .then(() -> publisher.next("foo", "bar").error(new IllegalArgumentException("boom"))) + .expectNextCount(2) + .expectErrorMessage("boom") + .verify(); + } + + + + @Test + public void conditionalSupport() { + TestPublisher up = TestPublisher.create(); + StepVerifier.create(up.flux().filter("test"::equals), 2) + .then(() -> up.next("test")) + .then(() -> up.next("test2")) + .then(() -> up.emit("test")) + .expectNext("test", "test") + .verifyComplete(); + } + +} \ No newline at end of file diff --git a/reactor-test/src/test/java/reactor/test/scheduler/VirtualTimeSchedulerTests.java b/reactor-test/src/test/java/reactor/test/scheduler/VirtualTimeSchedulerTests.java new file mode 100644 index 0000000000..8ea8341248 --- /dev/null +++ b/reactor-test/src/test/java/reactor/test/scheduler/VirtualTimeSchedulerTests.java @@ -0,0 +1,123 @@ +/* + * Copyright (c) 2011-2017 Pivotal Software Inc, All Rights Reserved. + * + * 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 + * + * http://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 reactor.test.scheduler; + +import java.util.function.Supplier; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; +import reactor.core.publisher.Mono; +import reactor.core.scheduler.Scheduler; +import reactor.core.scheduler.Schedulers; +import reactor.test.StepVerifier; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * @author Stephane Maldini + */ +public class VirtualTimeSchedulerTests { + + @Test + public void allEnabled() { + Assert.assertFalse(Schedulers.newParallel("") instanceof VirtualTimeScheduler); + Assert.assertFalse(Schedulers.newElastic("") instanceof VirtualTimeScheduler); + Assert.assertFalse(Schedulers.newSingle("") instanceof VirtualTimeScheduler); + + VirtualTimeScheduler.getOrSet(); + + Assert.assertTrue(Schedulers.newParallel("") instanceof VirtualTimeScheduler); + Assert.assertTrue(Schedulers.newElastic("") instanceof VirtualTimeScheduler); + Assert.assertTrue(Schedulers.newSingle("") instanceof VirtualTimeScheduler); + + VirtualTimeScheduler t = VirtualTimeScheduler.get(); + + Assert.assertSame(Schedulers.newParallel(""), t); + Assert.assertSame(Schedulers.newElastic(""), t); + Assert.assertSame(Schedulers.newSingle(""), t); + } + + @Test + public void enableProvidedAllSchedulerIdempotent() { + VirtualTimeScheduler vts = VirtualTimeScheduler.create(); + + VirtualTimeScheduler.getOrSet(vts); + + Assert.assertSame(vts, uncache(Schedulers.single())); + Assert.assertFalse(vts.shutdown); + + + VirtualTimeScheduler.getOrSet(vts); + + Assert.assertSame(vts, uncache(Schedulers.single())); + Assert.assertFalse(vts.shutdown); + } + + @Test + public void enableTwoSimilarSchedulersUsesFirst() { + VirtualTimeScheduler vts1 = VirtualTimeScheduler.create(); + VirtualTimeScheduler vts2 = VirtualTimeScheduler.create(); + + VirtualTimeScheduler firstEnableResult = VirtualTimeScheduler.getOrSet(vts1); + VirtualTimeScheduler secondEnableResult = VirtualTimeScheduler.getOrSet(vts2); + + Assert.assertSame(vts1, firstEnableResult); + Assert.assertSame(vts1, secondEnableResult); + Assert.assertSame(vts1, uncache(Schedulers.single())); + Assert.assertFalse(vts1.shutdown); + } + + @Test + public void disposedSchedulerIsStillCleanedUp() { + VirtualTimeScheduler vts = VirtualTimeScheduler.create(); + vts.dispose(); + assertThat(VirtualTimeScheduler.isFactoryEnabled()).isFalse(); + + StepVerifier.withVirtualTime(() -> Mono.just("foo"), + () -> vts, Long.MAX_VALUE) + .then(() -> assertThat(VirtualTimeScheduler.isFactoryEnabled()).isTrue()) + .then(() -> assertThat(VirtualTimeScheduler.get()).isSameAs(vts)) + .expectNext("foo") + .verifyComplete(); + + assertThat(VirtualTimeScheduler.isFactoryEnabled()).isFalse(); + + StepVerifier.withVirtualTime(() -> Mono.just("foo")) + .then(() -> assertThat(VirtualTimeScheduler.isFactoryEnabled()).isTrue()) + .then(() -> assertThat(VirtualTimeScheduler.get()).isNotSameAs(vts)) + .expectNext("foo") + .verifyComplete(); + + assertThat(VirtualTimeScheduler.isFactoryEnabled()).isFalse(); + } + + + @SuppressWarnings("unchecked") + private static Scheduler uncache(Scheduler potentialCached) { + if (potentialCached instanceof Supplier) { + return ((Supplier) potentialCached).get(); + } + return potentialCached; + } + + @After + public void cleanup() { + VirtualTimeScheduler.reset(); + } + +} \ No newline at end of file diff --git a/settings.gradle b/settings.gradle index 08101b042f..394e746ab7 100644 --- a/settings.gradle +++ b/settings.gradle @@ -16,4 +16,4 @@ rootProject.name = 'reactor' -include 'reactor-core' \ No newline at end of file +include 'reactor-core', 'reactor-test' \ No newline at end of file