diff --git a/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/InternalStages.java b/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/InternalStages.java index 3630219..e3eed93 100644 --- a/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/InternalStages.java +++ b/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/InternalStages.java @@ -23,7 +23,10 @@ /** * Internal stages, used to capture the graph while being built, but never passed to a - * {@link org.eclipse.microprofile.reactive.streams.spi.ReactiveStreamsEngine}. + * {@link org.eclipse.microprofile.reactive.streams.spi.ReactiveStreamsEngine}. These exist for performance reasons, + * allowing the builder to hold the graph as an immutable linked tree where multiple stages can be appended in constant + * time, rather than needing to copy an array each time. However, when it comes to building the graph, it is first + * flattened out to an array, removing any of the internal stages that held nested stages, etc. */ class InternalStages { diff --git a/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/ProcessorBuilder.java b/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/ProcessorBuilder.java index d1c1300..62d08a4 100644 --- a/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/ProcessorBuilder.java +++ b/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/ProcessorBuilder.java @@ -27,10 +27,10 @@ import org.reactivestreams.Subscriber; import java.util.List; +import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletionStage; import java.util.function.BiConsumer; -import java.util.function.BiFunction; import java.util.function.BinaryOperator; import java.util.function.Consumer; import java.util.function.Function; @@ -242,6 +242,7 @@ public ProcessorBuilder dropWhile(Predicate predicate) { * @return A new subscriber builder. */ public SubscriberBuilder forEach(Consumer action) { + Objects.requireNonNull(action, "Action must not be null"); return collect(Collector.of( () -> null, (n, r) -> action.accept(r), @@ -303,24 +304,6 @@ public SubscriberBuilder> reduce(BinaryOperator accumulator) { return addTerminalStage(new Stage.Collect(Reductions.reduce(accumulator))); } - /** - * Perform a reduction on the elements of this stream, using the provided identity value, accumulation function and - * combiner function. - *

- * The result of the reduction is returned in the {@link CompletionSubscriber}. - * - * @param identity The identity value. - * @param accumulator The accumulator function. - * @param combiner The combiner function. - * @return A new subscriber builder. - */ - public SubscriberBuilder reduce(S identity, - BiFunction accumulator, - BinaryOperator combiner) { - - return addTerminalStage(new Stage.Collect(Reductions.reduce(identity, accumulator, combiner))); - } - /** * Collect the elements emitted by this processor builder using the given {@link Collector}. *

@@ -536,6 +519,7 @@ public Processor buildRs() { * @return A {@link Processor} that will run this stream. */ public Processor buildRs(ReactiveStreamsEngine engine) { + Objects.requireNonNull(engine, "Engine must not be null"); return engine.buildProcessor(toGraph()); } diff --git a/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/PublisherBuilder.java b/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/PublisherBuilder.java index 3744778..f9a94f4 100644 --- a/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/PublisherBuilder.java +++ b/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/PublisherBuilder.java @@ -27,10 +27,10 @@ import org.reactivestreams.Subscriber; import java.util.List; +import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletionStage; import java.util.function.BiConsumer; -import java.util.function.BiFunction; import java.util.function.BinaryOperator; import java.util.function.Consumer; import java.util.function.Function; @@ -241,6 +241,7 @@ public PublisherBuilder dropWhile(Predicate predicate) { * @return A new completion builder. */ public CompletionRunner forEach(Consumer action) { + Objects.requireNonNull(action, "Action must not be null"); return collect(Collector.of( () -> null, (n, t) -> action.accept(t), @@ -301,24 +302,6 @@ public CompletionRunner> reduce(BinaryOperator accumulator) { return addTerminalStage(new Stage.Collect(Reductions.reduce(accumulator))); } - /** - * Perform a reduction on the elements of this stream, using the provided identity value, accumulation function and - * combiner function. - *

- * The result of the reduction is returned in the {@link CompletionStage}. - * - * @param identity The identity value. - * @param accumulator The accumulator function. - * @param combiner The combiner function. - * @return A new completion builder. - */ - public CompletionRunner reduce(S identity, - BiFunction accumulator, - BinaryOperator combiner) { - - return addTerminalStage(new Stage.Collect(Reductions.reduce(identity, accumulator, combiner))); - } - /** * Find the first element emitted by the {@link Publisher}, and return it in a * {@link CompletionStage}. @@ -390,6 +373,7 @@ public CompletionRunner to(Subscriber subscriber) { * @return A {@link CompletionRunner} that completes when the stream completes. */ public CompletionRunner to(SubscriberBuilder subscriber) { + Objects.requireNonNull(subscriber, "Subscriber must not be null"); return addTerminalStage(new InternalStages.Nested(subscriber.getGraphBuilder())); } @@ -536,6 +520,7 @@ public Publisher buildRs() { * @return A {@link Publisher} that will run this stream. */ public Publisher buildRs(ReactiveStreamsEngine engine) { + Objects.requireNonNull(engine, "Engine must not be null"); return engine.buildPublisher(toGraph()); } diff --git a/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/ReactiveStreams.java b/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/ReactiveStreams.java index eadf690..c4be734 100644 --- a/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/ReactiveStreams.java +++ b/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/ReactiveStreams.java @@ -26,6 +26,7 @@ import java.util.Arrays; import java.util.Collections; +import java.util.Objects; import java.util.function.Supplier; import java.util.function.UnaryOperator; import java.util.stream.Stream; @@ -163,6 +164,7 @@ public static SubscriberBuilder fromSubscriber(Subscriber PublisherBuilder iterate(T seed, UnaryOperator f) { + Objects.requireNonNull(f, "Operator must not be null"); return fromIterable(() -> Stream.iterate(seed, f).iterator()); } @@ -174,6 +176,7 @@ public static PublisherBuilder iterate(T seed, UnaryOperator f) { * @return A publisher builder. */ public static PublisherBuilder generate(Supplier s) { + Objects.requireNonNull(s, "Supplier must not be null"); return fromIterable(() -> Stream.generate((Supplier) s).iterator()); } diff --git a/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/Reductions.java b/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/Reductions.java index 28e32cf..08b52ad 100644 --- a/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/Reductions.java +++ b/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/Reductions.java @@ -19,8 +19,8 @@ package org.eclipse.microprofile.reactive.streams; +import java.util.Objects; import java.util.Optional; -import java.util.function.BiFunction; import java.util.function.BinaryOperator; import java.util.stream.Collector; @@ -33,7 +33,7 @@ private Reductions() { } static Collector> reduce(BinaryOperator reducer) { - + Objects.requireNonNull(reducer, "Reduction function must not be null"); return Collector.of(Reduction::new, (r, t) -> { if (r.value == null) { @@ -59,7 +59,7 @@ else if (s.value != null) { } static Collector reduce(T identity, BinaryOperator reducer) { - + Objects.requireNonNull(reducer, "Reduction function must not be null"); return Collector.of(() -> new Reduction<>(identity), (r, t) -> r.value = reducer.apply(r.value, t), (r, s) -> r.replace(reducer.apply(r.value, s.value)), @@ -67,17 +67,6 @@ else if (s.value != null) { ); } - static Collector reduce(S identity, - BiFunction accumulator, - BinaryOperator combiner) { - - return Collector.of(() -> new Reduction<>(identity), - (r, t) -> r.value = accumulator.apply(r.value, t), - (r, s) -> r.replace(combiner.apply(r.value, s.value)), - r -> r.value - ); - } - private static class Reduction { private T value; diff --git a/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/SubscriberBuilder.java b/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/SubscriberBuilder.java index 2ea405a..af0e36e 100644 --- a/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/SubscriberBuilder.java +++ b/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/SubscriberBuilder.java @@ -23,6 +23,8 @@ import org.eclipse.microprofile.reactive.streams.spi.ReactiveStreamsEngine; import org.eclipse.microprofile.reactive.streams.spi.Stage; +import java.util.Objects; + /** * A builder for a {@link org.reactivestreams.Subscriber} and its result. *

@@ -63,6 +65,7 @@ public CompletionSubscriber build() { * @return A {@link CompletionSubscriber} that will run this stream. */ public CompletionSubscriber build(ReactiveStreamsEngine engine) { + Objects.requireNonNull(engine, "Engine must not be null"); return engine.buildSubscriber(toGraph()); } diff --git a/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/spi/Stage.java b/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/spi/Stage.java index bb954f2..5f9a2c8 100644 --- a/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/spi/Stage.java +++ b/streams/api/src/main/java/org/eclipse/microprofile/reactive/streams/spi/Stage.java @@ -24,6 +24,7 @@ import org.reactivestreams.Subscriber; import java.util.Collections; +import java.util.Objects; import java.util.concurrent.CompletionStage; import java.util.function.Consumer; import java.util.function.Function; @@ -91,7 +92,7 @@ final class Map implements Inlet, Outlet { private final Function mapper; public Map(Function mapper) { - this.mapper = mapper; + this.mapper = Objects.requireNonNull(mapper, "Mapper function must not be null"); } /** @@ -116,7 +117,7 @@ final class Peek implements Inlet, Outlet { private final Consumer consumer; public Peek(Consumer consumer) { - this.consumer = consumer; + this.consumer = Objects.requireNonNull(consumer, "Consumer must not be null"); } /** @@ -141,7 +142,7 @@ final class Filter implements Inlet, Outlet { private final Predicate predicate; public Filter(Predicate predicate) { - this.predicate = predicate; + this.predicate = Objects.requireNonNull(predicate, "Predicate must not be null"); } /** @@ -166,7 +167,7 @@ final class DropWhile implements Inlet, Outlet { private final Predicate predicate; public DropWhile(Predicate predicate) { - this.predicate = predicate; + this.predicate = Objects.requireNonNull(predicate, "Predicate must not be null"); } /** @@ -249,7 +250,7 @@ final class TakeWhile implements Inlet, Outlet { private final Predicate predicate; public TakeWhile(Predicate predicate) { - this.predicate = predicate; + this.predicate = Objects.requireNonNull(predicate, "Predicate must not be null"); } /** @@ -272,7 +273,7 @@ final class PublisherStage implements Outlet { private final Publisher publisher; public PublisherStage(Publisher publisher) { - this.publisher = publisher; + this.publisher = Objects.requireNonNull(publisher, "Publisher must not be null"); } /** @@ -294,11 +295,15 @@ public Publisher getRsPublisher() { * Any exceptions thrown by the iterator must be propagated downstream. */ final class Of implements Outlet { + /** + * Instance used for an empty stream. + */ public static final Of EMPTY = new Of(Collections.emptyList()); + private final Iterable elements; public Of(Iterable elements) { - this.elements = elements; + this.elements = Objects.requireNonNull(elements, "Iterable must not be null"); } /** @@ -320,7 +325,7 @@ final class ProcessorStage implements Inlet, Outlet { private final Processor processor; public ProcessorStage(Processor processor) { - this.processor = processor; + this.processor = Objects.requireNonNull(processor, "Processor must not be null"); } /** @@ -362,7 +367,7 @@ final class SubscriberStage implements Inlet { private final Subscriber subscriber; public SubscriberStage(Subscriber subscriber) { - this.subscriber = subscriber; + this.subscriber = Objects.requireNonNull(subscriber, "Subscriber must not be null"); } /** @@ -391,7 +396,7 @@ final class Collect implements Inlet { private final Collector collector; public Collect(Collector collector) { - this.collector = collector; + this.collector = Objects.requireNonNull(collector, "Collector must not be null"); } /** @@ -418,7 +423,7 @@ final class FlatMap implements Inlet, Outlet { private final Function mapper; public FlatMap(Function mapper) { - this.mapper = mapper; + this.mapper = Objects.requireNonNull(mapper, "Mapper function must not be null"); } /** @@ -444,7 +449,7 @@ final class FlatMapCompletionStage implements Inlet, Outlet { private final Function> mapper; public FlatMapCompletionStage(Function> mapper) { - this.mapper = mapper; + this.mapper = Objects.requireNonNull(mapper, "Mapper function must not be null"); } /** @@ -467,7 +472,7 @@ final class FlatMapIterable implements Inlet, Outlet { private final Function> mapper; public FlatMapIterable(Function> mapper) { - this.mapper = mapper; + this.mapper = Objects.requireNonNull(mapper, "Mapper function must not be null"); } /** @@ -493,7 +498,7 @@ final class OnError implements Inlet, Outlet { public OnError(Consumer consumer) { - this.consumer = consumer; + this.consumer = Objects.requireNonNull(consumer, "Consumer must not be null"); } /** @@ -520,7 +525,7 @@ final class OnTerminate implements Inlet, Outlet { private final Runnable action; public OnTerminate(Runnable runnable) { - this.action = runnable; + this.action = Objects.requireNonNull(runnable, "Action must not be null"); } /** @@ -546,7 +551,7 @@ final class OnComplete implements Inlet, Outlet { private final Runnable action; public OnComplete(Runnable runnable) { - this.action = runnable; + this.action = Objects.requireNonNull(runnable, "Action must not be null"); } /** @@ -579,7 +584,7 @@ final class OnErrorResume implements Inlet, Outlet { public OnErrorResume(Function function) { - this.function = function; + this.function = Objects.requireNonNull(function, "Resume function must not be null"); } /** @@ -612,8 +617,8 @@ final class OnErrorResumeWith implements Inlet, Outlet { private final Function function; - public OnErrorResumeWith(Function function) { - this.function = function; + public OnErrorResumeWith(Function function) { + this.function = Objects.requireNonNull(function, "Resume with function must be empty"); } /** @@ -635,7 +640,7 @@ final class Failed implements Outlet { private final Throwable error; public Failed(Throwable error) { - this.error = error; + this.error = Objects.requireNonNull(error, "Exception must not be null"); } public Throwable getError() { diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/ConcatStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/ConcatStageVerification.java deleted file mode 100644 index c687c44..0000000 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/ConcatStageVerification.java +++ /dev/null @@ -1,130 +0,0 @@ -/******************************************************************************* - * Copyright (c) 2018 Contributors to the Eclipse Foundation - * - * See the NOTICE file(s) distributed with this work for additional - * information regarding copyright ownership. - * - * 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 org.eclipse.microprofile.reactive.streams.tck; - -import org.eclipse.microprofile.reactive.streams.ReactiveStreams; -import org.reactivestreams.Publisher; -import org.reactivestreams.Subscriber; -import org.reactivestreams.Subscription; -import org.testng.annotations.Test; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionStage; -import java.util.stream.IntStream; -import java.util.stream.LongStream; - -import static org.testng.Assert.assertEquals; - -public class ConcatStageVerification extends AbstractStageVerification { - - ConcatStageVerification(ReactiveStreamsTck.VerificationDeps deps) { - super(deps); - } - - @Test - public void concatStageShouldConcatTwoGraphs() { - assertEquals(await( - ReactiveStreams.concat( - ReactiveStreams.of(1, 2, 3), - ReactiveStreams.of(4, 5, 6) - ) - .toList() - .run(getEngine()) - ), Arrays.asList(1, 2, 3, 4, 5, 6)); - } - - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") - public void concatStageShouldCancelSecondStageIfFirstFails() { - CancelCapturingPublisher cancelCapture = new CancelCapturingPublisher<>(); - - CompletionStage completion = ReactiveStreams.concat( - ReactiveStreams.failed(new RuntimeException("failed")), - ReactiveStreams.fromPublisher(cancelCapture) - ) - .ignore() - .run(getEngine()); - - await(cancelCapture.getCancelled()); - await(completion); - } - - @Test - public void concatStageShouldCancelSecondStageIfFirstCancellationOccursDuringFirst() { - CancelCapturingPublisher cancelCapture = new CancelCapturingPublisher<>(); - - CompletionStage> result = ReactiveStreams.concat( - ReactiveStreams.fromIterable(() -> IntStream.range(1, 1000000).boxed().iterator()), - ReactiveStreams.fromPublisher(cancelCapture) - ) - .limit(5) - .toList() - .run(getEngine()); - - await(cancelCapture.getCancelled()); - assertEquals(await(result), Arrays.asList(1, 2, 3, 4, 5)); - } - - @Override - List reactiveStreamsTckVerifiers() { - return Collections.singletonList(new PublisherVerification()); - } - - private static class CancelCapturingPublisher implements Publisher { - private final CompletableFuture cancelled = new CompletableFuture<>(); - - @Override - public void subscribe(Subscriber subscriber) { - subscriber.onSubscribe(new Subscription() { - @Override - public void request(long n) { - } - - @Override - public void cancel() { - cancelled.complete(null); - } - }); - } - - public CompletableFuture getCancelled() { - return cancelled; - } - } - - class PublisherVerification extends StagePublisherVerification { - @Override - public Publisher createPublisher(long elements) { - long toEmitFromFirst = elements / 2; - - return ReactiveStreams.concat( - ReactiveStreams.fromIterable( - () -> LongStream.rangeClosed(1, toEmitFromFirst).boxed().iterator() - ), - ReactiveStreams.fromIterable( - () -> LongStream.rangeClosed(toEmitFromFirst + 1, elements).boxed().iterator() - ) - ).buildRs(getEngine()); - } - } - -} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FlatMapIterableStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FlatMapIterableStageVerification.java deleted file mode 100644 index e21a064..0000000 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FlatMapIterableStageVerification.java +++ /dev/null @@ -1,83 +0,0 @@ -/******************************************************************************* - * Copyright (c) 2018 Contributors to the Eclipse Foundation - * - * See the NOTICE file(s) distributed with this work for additional - * information regarding copyright ownership. - * - * 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 org.eclipse.microprofile.reactive.streams.tck; - -import org.eclipse.microprofile.reactive.streams.ReactiveStreams; -import org.reactivestreams.Processor; -import org.reactivestreams.Publisher; -import org.testng.annotations.Test; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -import static org.testng.Assert.assertEquals; - -public class FlatMapIterableStageVerification extends AbstractStageVerification { - FlatMapIterableStageVerification(ReactiveStreamsTck.VerificationDeps deps) { - super(deps); - } - - @Test - public void flatMapIterableStageShouldMapElements() { - assertEquals(await(ReactiveStreams.of(1, 2, 3) - .flatMapIterable(n -> Arrays.asList(n, n, n)) - .toList() - .run(getEngine())), Arrays.asList(1, 1, 1, 2, 2, 2, 3, 3, 3)); - } - - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") - public void flatMapIterableStageShouldPropagateRuntimeExceptions() { - await(ReactiveStreams.of("foo") - .flatMapIterable(foo -> { - throw new RuntimeException("failed"); - }) - .toList() - .run(getEngine())); - } - - @Override - List reactiveStreamsTckVerifiers() { - return Collections.singletonList(new ProcessorVerification()); - } - - /** - * Verifies the outer processor. - */ - public class ProcessorVerification extends StageProcessorVerification { - - @Override - public Processor createIdentityProcessor(int bufferSize) { - return ReactiveStreams.builder().flatMapIterable(Arrays::asList).buildRs(getEngine()); - } - - @Override - public Publisher createFailedPublisher() { - return ReactiveStreams.failed(new RuntimeException("failed")) - .flatMapIterable(Arrays::asList).buildRs(getEngine()); - } - - @Override - public Integer createElement(int element) { - return element; - } - } - -} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FlatMapPublisherStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FlatMapPublisherStageVerification.java deleted file mode 100644 index d506e23..0000000 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FlatMapPublisherStageVerification.java +++ /dev/null @@ -1,157 +0,0 @@ -/******************************************************************************* - * Copyright (c) 2018 Contributors to the Eclipse Foundation - * - * See the NOTICE file(s) distributed with this work for additional - * information regarding copyright ownership. - * - * 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 org.eclipse.microprofile.reactive.streams.tck; - -import org.eclipse.microprofile.reactive.streams.PublisherBuilder; -import org.eclipse.microprofile.reactive.streams.ReactiveStreams; -import org.reactivestreams.Processor; -import org.reactivestreams.Publisher; -import org.reactivestreams.Subscriber; -import org.reactivestreams.Subscription; -import org.testng.annotations.Test; - -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionStage; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -import static org.testng.Assert.assertEquals; - -public class FlatMapPublisherStageVerification extends AbstractStageVerification { - FlatMapPublisherStageVerification(ReactiveStreamsTck.VerificationDeps deps) { - super(deps); - } - - @Test - public void flatMapStageShouldMapElements() { - assertEquals(await(ReactiveStreams.of(1, 2, 3) - .flatMapPublisher(n -> ReactiveStreams.of(n, n, n).buildRs()) - .toList() - .run(getEngine())), Arrays.asList(1, 1, 1, 2, 2, 2, 3, 3, 3)); - } - - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") - public void flatMapStageShouldPropagateRuntimeExceptions() { - await(ReactiveStreams.of("foo") - .flatMapPublisher(foo -> { - throw new RuntimeException("failed"); - }) - .toList() - .run(getEngine())); - } - - @Test - public void flatMapStageShouldOnlySubscribeToOnePublisherAtATime() throws Exception { - AtomicInteger activePublishers = new AtomicInteger(); - - - CompletionStage> result = ReactiveStreams.of(1, 2, 3, 4, 5) - .flatMapPublisher(id -> new ScheduledPublisher(id, activePublishers, this::getExecutorService)) - .toList() - .run(getEngine()); - - assertEquals(result.toCompletableFuture().get(2, TimeUnit.SECONDS), - Arrays.asList(1, 2, 3, 4, 5)); - } - - - @Override - List reactiveStreamsTckVerifiers() { - return Arrays.asList(new OuterProcessorVerification(), new InnerSubscriberVerification()); - } - - /** - * Verifies the outer processor. - */ - public class OuterProcessorVerification extends StageProcessorVerification { - - @Override - public Processor createIdentityProcessor(int bufferSize) { - return ReactiveStreams.builder().flatMapPublisher(x -> ReactiveStreams.of(x).buildRs()).buildRs(getEngine()); - } - - @Override - public Publisher createFailedPublisher() { - return ReactiveStreams.failed(new RuntimeException("failed")) - .flatMapPublisher(x -> ReactiveStreams.of(x).buildRs()).buildRs(getEngine()); - } - - @Override - public Integer createElement(int element) { - return element; - } - } - - /** - * Verifies the inner subscriber passed to publishers produced by the mapper function. - */ - public class InnerSubscriberVerification extends StageSubscriberWhiteboxVerification { - - @Override - public Subscriber createSubscriber(WhiteboxSubscriberProbe probe) { - CompletableFuture> subscriber = new CompletableFuture<>(); - ReactiveStreams.of(ReactiveStreams.fromPublisher(subscriber::complete)) - .flatMapPublisher(PublisherBuilder::buildRs) - .to(new Subscriber() { - @Override - public void onSubscribe(Subscription subscription) { - // We need to initially request an element to ensure that we get the publisher. - subscription.request(1); - probe.registerOnSubscribe(new SubscriberPuppet() { - @Override - public void triggerRequest(long elements) { - subscription.request(elements); - } - - @Override - public void signalCancel() { - subscription.cancel(); - } - }); - } - - @Override - public void onNext(Integer item) { - probe.registerOnNext(item); - } - - @Override - public void onError(Throwable throwable) { - probe.registerOnError(throwable); - } - - @Override - public void onComplete() { - probe.registerOnComplete(); - } - }) - .run(getEngine()); - - return (Subscriber) await(subscriber); - } - - @Override - public Integer createElement(int element) { - return element; - } - } -} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/ReactiveStreamsTck.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/ReactiveStreamsTck.java index f10fe4e..c91aaec 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/ReactiveStreamsTck.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/ReactiveStreamsTck.java @@ -20,16 +20,16 @@ package org.eclipse.microprofile.reactive.streams.tck; import org.eclipse.microprofile.reactive.streams.spi.ReactiveStreamsEngine; +import org.eclipse.microprofile.reactive.streams.tck.api.ReactiveStreamsApiVerification; +import org.eclipse.microprofile.reactive.streams.tck.spi.ReactiveStreamsSpiVerification; import org.reactivestreams.tck.TestEnvironment; import org.testng.annotations.AfterSuite; import org.testng.annotations.Factory; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; -import java.util.function.Function; /** * The Reactive Streams TCK. @@ -75,6 +75,7 @@ public void shutdownEngine() { if (engine != null) { shutdownEngine(engine); } + executorService.shutdown(); } @Factory @@ -82,56 +83,15 @@ public Object[] allTests() { engine = createEngine(); executorService = Executors.newScheduledThreadPool(4); - List> stageVerifications = Arrays.asList( - OfStageVerification::new, - MapStageVerification::new, - FlatMapStageVerification::new, - FilterStageVerification::new, - FindFirstStageVerification::new, - CollectStageVerification::new, - TakeWhileStageVerification::new, - FlatMapPublisherStageVerification::new, - FlatMapCompletionStageVerification::new, - FlatMapIterableStageVerification::new, - ConcatStageVerification::new, - EmptyProcessorVerification::new, - CancelStageVerification::new, - SubscriberStageVerification::new, - PeekStageVerification::new, - DistinctStageVerification::new, - OnStagesVerification::new, - LimitStageVerification::new, - SkipStageVerification::new, - DropWhileStageVerification::new, - OnErrorResumeStageVerification::new - ); + ReactiveStreamsApiVerification apiVerification = new ReactiveStreamsApiVerification(); + ReactiveStreamsSpiVerification spiVerification = new ReactiveStreamsSpiVerification(testEnvironment, engine, executorService); + // Add tests that aren't dependent on the dependencies. List allTests = new ArrayList<>(); - VerificationDeps deps = new VerificationDeps(); - for (Function creator : stageVerifications) { - AbstractStageVerification stageVerification = creator.apply(deps); - allTests.add(stageVerification); - allTests.addAll(stageVerification.reactiveStreamsTckVerifiers()); - } - // Add tests that aren't dependent on the dependencies. - allTests.add(new GraphAccessorVerification()); + allTests.addAll(apiVerification.allTests()); + allTests.addAll(spiVerification.allTests()); return allTests.stream().filter(this::isEnabled).toArray(); } - - class VerificationDeps { - ReactiveStreamsEngine engine() { - return engine; - } - - TestEnvironment testEnvironment() { - return testEnvironment; - } - - ScheduledExecutorService executorService() { - return executorService; - } - } - } diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/CompletionRunnerVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/CompletionRunnerVerification.java new file mode 100644 index 0000000..59ab34a --- /dev/null +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/CompletionRunnerVerification.java @@ -0,0 +1,84 @@ +/******************************************************************************* + * Copyright (c) 2018 Contributors to the Eclipse Foundation + * + * See the NOTICE file(s) distributed with this work for additional + * information regarding copyright ownership. + * + * 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 org.eclipse.microprofile.reactive.streams.tck.api; + +import org.eclipse.microprofile.reactive.streams.CompletionSubscriber; +import org.eclipse.microprofile.reactive.streams.ReactiveStreams; +import org.eclipse.microprofile.reactive.streams.spi.Graph; +import org.eclipse.microprofile.reactive.streams.spi.ReactiveStreamsEngine; +import org.eclipse.microprofile.reactive.streams.spi.Stage; +import org.eclipse.microprofile.reactive.streams.spi.UnsupportedStageException; +import org.reactivestreams.Processor; +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import java.util.Iterator; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.atomic.AtomicReference; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertSame; + +/** + * Verification for the {@link org.eclipse.microprofile.reactive.streams.CompletionRunner} class. + */ +public class CompletionRunnerVerification { + + @Test + public void run() { + AtomicReference builtGraph = new AtomicReference<>(); + CompletableFuture expected = new CompletableFuture(); + CompletionStage returned = ReactiveStreams.empty().cancel().run(new ReactiveStreamsEngine() { + @Override + public Publisher buildPublisher(Graph graph) throws UnsupportedStageException { + throw new RuntimeException("Wrong method invoked"); + } + @Override + public CompletionSubscriber buildSubscriber(Graph graph) throws UnsupportedStageException { + throw new RuntimeException("Wrong method invoked"); + } + @Override + public Processor buildProcessor(Graph graph) throws UnsupportedStageException { + throw new RuntimeException("Wrong method invoked"); + } + + @Override + public CompletionStage buildCompletion(Graph graph) throws UnsupportedStageException { + builtGraph.set(graph); + return expected; + } + }); + + assertSame(returned, expected); + assertFalse(builtGraph.get().hasInlet()); + assertFalse(builtGraph.get().hasOutlet()); + assertEquals(builtGraph.get().getStages().size(), 2); + Iterator stages = builtGraph.get().getStages().iterator(); + assertSame(stages.next(), Stage.Of.EMPTY); + assertSame(stages.next(), Stage.Cancel.INSTANCE); + } + + @Test(expectedExceptions = NullPointerException.class) + public void runNull() { + ReactiveStreams.empty().cancel().run(null); + } +} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/CompletionSubscriberVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/CompletionSubscriberVerification.java new file mode 100644 index 0000000..d5b09a5 --- /dev/null +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/CompletionSubscriberVerification.java @@ -0,0 +1,92 @@ +/******************************************************************************* + * Copyright (c) 2018 Contributors to the Eclipse Foundation + * + * See the NOTICE file(s) distributed with this work for additional + * information regarding copyright ownership. + * + * 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 org.eclipse.microprofile.reactive.streams.tck.api; + +import org.eclipse.microprofile.reactive.streams.CompletionSubscriber; +import org.reactivestreams.Subscriber; +import org.reactivestreams.Subscription; +import org.testng.annotations.Test; + +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.concurrent.CompletableFuture; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertSame; +import static org.testng.Assert.assertTrue; + +/** + * Verification for {@link org.eclipse.microprofile.reactive.streams.CompletionSubscriber}. + */ +public class CompletionSubscriberVerification { + + @Test + public void completionSubscriberShouldReturnSameCompletionStage() { + CompletableFuture future = new CompletableFuture(); + assertSame(CompletionSubscriber.of(Mocks.SUBSCRIBER, future).getCompletion(), future); + } + + @Test + public void completionSubscriberShouldDelegateToSubscriber() { + Deque calls = new ArrayDeque<>(); + CompletionSubscriber subscriber = CompletionSubscriber.of(new Subscriber() { + @Override + public void onSubscribe(Subscription s) { + calls.add(s); + } + + @Override + public void onNext(Object o) { + calls.add(o); + } + + @Override + public void onError(Throwable t) { + calls.add(t); + } + + @Override + public void onComplete() { + calls.add("onComplete"); + } + }, new CompletableFuture<>()); + + subscriber.onSubscribe(Mocks.SUBSCRIPTION); + assertSame(calls.removeFirst(), Mocks.SUBSCRIPTION); + subscriber.onNext("element"); + assertEquals(calls.removeFirst(), "element"); + Exception e = new Exception(); + subscriber.onError(e); + assertSame(calls.removeFirst(), e); + subscriber.onComplete(); + assertEquals(calls.removeFirst(),"onComplete"); + assertTrue(calls.isEmpty()); + } + + @Test(expectedExceptions = NullPointerException.class) + public void completionSubscriberShouldNotAcceptNullSubscriber() { + CompletionSubscriber.of(null, new CompletableFuture<>()); + } + + @Test(expectedExceptions = NullPointerException.class) + public void completionSubscriberShouldNotAcceptNullCompletionStage() { + CompletionSubscriber.of(Mocks.SUBSCRIBER, null); + } +} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/GraphAccessorVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/GraphAccessorVerification.java similarity index 96% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/GraphAccessorVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/GraphAccessorVerification.java index 6a174c3..222e5e5 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/GraphAccessorVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/GraphAccessorVerification.java @@ -17,7 +17,7 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.api; import org.eclipse.microprofile.reactive.streams.GraphAccessor; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; @@ -33,8 +33,6 @@ /** * Test for the GraphAccessor class. - *

- * This does not need an implementation of the engine to verify it. */ public class GraphAccessorVerification { diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/Mocks.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/Mocks.java new file mode 100644 index 0000000..3b5eeb5 --- /dev/null +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/Mocks.java @@ -0,0 +1,79 @@ +/******************************************************************************* + * Copyright (c) 2018 Contributors to the Eclipse Foundation + * + * See the NOTICE file(s) distributed with this work for additional + * information regarding copyright ownership. + * + * 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 org.eclipse.microprofile.reactive.streams.tck.api; + +import org.reactivestreams.Processor; +import org.reactivestreams.Publisher; +import org.reactivestreams.Subscriber; +import org.reactivestreams.Subscription; + +/** + * Shared mock objects + */ +class Mocks { + + private Mocks() { + } + + static final Subscriber SUBSCRIBER = new Subscriber() { + @Override + public void onSubscribe(Subscription s) { + } + @Override + public void onNext(Object o) { + } + @Override + public void onError(Throwable t) { + } + @Override + public void onComplete() { + } + }; + + static final Publisher PUBLISHER = s -> {}; + + static final Processor PROCESSOR = new Processor() { + @Override + public void subscribe(Subscriber s) { + } + @Override + public void onSubscribe(Subscription s) { + } + @Override + public void onNext(Object o) { + } + @Override + public void onError(Throwable t) { + } + @Override + public void onComplete() { + } + }; + + static final Subscription SUBSCRIPTION = new Subscription() { + @Override + public void request(long n) { + } + @Override + public void cancel() { + } + }; + +} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/ProcessorBuilderVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/ProcessorBuilderVerification.java new file mode 100644 index 0000000..f34a3e3 --- /dev/null +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/ProcessorBuilderVerification.java @@ -0,0 +1,577 @@ +/******************************************************************************* + * Copyright (c) 2018 Contributors to the Eclipse Foundation + * + * See the NOTICE file(s) distributed with this work for additional + * information regarding copyright ownership. + * + * 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 org.eclipse.microprofile.reactive.streams.tck.api; + +import org.eclipse.microprofile.reactive.streams.CompletionSubscriber; +import org.eclipse.microprofile.reactive.streams.GraphAccessor; +import org.eclipse.microprofile.reactive.streams.ProcessorBuilder; +import org.eclipse.microprofile.reactive.streams.ReactiveStreams; +import org.eclipse.microprofile.reactive.streams.SubscriberBuilder; +import org.eclipse.microprofile.reactive.streams.spi.Graph; +import org.eclipse.microprofile.reactive.streams.spi.ReactiveStreamsEngine; +import org.eclipse.microprofile.reactive.streams.spi.Stage; +import org.eclipse.microprofile.reactive.streams.spi.UnsupportedStageException; +import org.reactivestreams.Processor; +import org.reactivestreams.Publisher; +import org.reactivestreams.Subscriber; +import org.testng.annotations.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.stream.Collector; +import java.util.stream.Collectors; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertSame; +import static org.testng.Assert.assertTrue; + +/** + * Verification for the {@link ProcessorBuilder} class. + */ +public class ProcessorBuilderVerification { + + @Test + public void map() { + Graph graph = GraphAccessor.buildGraphFor(builder().map(i -> i + 1)); + assertTrue(graph.hasOutlet()); + assertEquals(((Function) getAddedStage(Stage.Map.class, graph).getMapper()).apply(1), 2); + } + + @Test(expectedExceptions = NullPointerException.class) + public void mapNullFunction() { + builder().map(null); + } + + @Test + public void peek() { + AtomicInteger peeked = new AtomicInteger(); + Graph graph = GraphAccessor.buildGraphFor(builder().peek(peeked::set)); + assertTrue(graph.hasOutlet()); + ((Consumer) getAddedStage(Stage.Peek.class, graph).getConsumer()).accept(1); + assertEquals(peeked.get(), 1); + } + + @Test(expectedExceptions = NullPointerException.class) + public void peekNullConsumer() { + builder().peek(null); + } + + @Test + public void filter() { + Graph graph = GraphAccessor.buildGraphFor(builder().filter(i -> i < 3)); + assertTrue(graph.hasOutlet()); + assertTrue(((Predicate) getAddedStage(Stage.Filter.class, graph).getPredicate()).test(1)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void filterNullPredicate() { + builder().filter(null); + } + + @Test + public void distinct() { + Graph graph = GraphAccessor.buildGraphFor(builder().distinct()); + assertTrue(graph.hasOutlet()); + assertSame(getAddedStage(Stage.Distinct.class, graph), Stage.Distinct.INSTANCE); + } + + @Test + public void flatMap() { + Graph graph = GraphAccessor.buildGraphFor(builder().flatMap(i -> ReactiveStreams.empty())); + assertTrue(graph.hasOutlet()); + Function flatMap = getAddedStage(Stage.FlatMap.class, graph).getMapper(); + Object result = flatMap.apply(1); + assertTrue(result instanceof Graph); + Graph innerGraph = (Graph) result; + assertFalse(innerGraph.hasInlet()); + assertTrue(innerGraph.hasOutlet()); + assertEquals(innerGraph.getStages(), Collections.singletonList(Stage.Of.EMPTY)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void flatMapNullMapper() { + builder().flatMap(null); + } + + @Test + public void flatMapPublisher() { + Graph graph = GraphAccessor.buildGraphFor(builder().flatMapPublisher(i -> Mocks.PUBLISHER)); + assertTrue(graph.hasOutlet()); + Function flatMap = getAddedStage(Stage.FlatMap.class, graph).getMapper(); + Object result = flatMap.apply(1); + assertTrue(result instanceof Graph); + Graph innerGraph = (Graph) result; + assertFalse(innerGraph.hasInlet()); + assertTrue(innerGraph.hasOutlet()); + assertEquals(innerGraph.getStages().size(), 1); + Stage inner = innerGraph.getStages().iterator().next(); + assertTrue(inner instanceof Stage.PublisherStage); + assertEquals(((Stage.PublisherStage) inner).getRsPublisher(), Mocks.PUBLISHER); + } + + @Test(expectedExceptions = NullPointerException.class) + public void flatMapPublisherNullMapper() { + builder().flatMapPublisher(null); + } + + @Test + public void flatMapCompletionStage() throws Exception { + Graph graph = GraphAccessor.buildGraphFor(builder().flatMapCompletionStage(i -> CompletableFuture.completedFuture(i + 1))); + assertTrue(graph.hasOutlet()); + CompletionStage result = (CompletionStage) ((Function) getAddedStage(Stage.FlatMapCompletionStage.class, graph).getMapper()).apply(1); + assertEquals(result.toCompletableFuture().get(1, TimeUnit.SECONDS), 2); + } + + @Test(expectedExceptions = NullPointerException.class) + public void flatMapCompletionStageNullMapper() { + builder().flatMapCompletionStage(null); + } + + @Test + public void flatMapIterable() { + Graph graph = GraphAccessor.buildGraphFor(builder().flatMapIterable(i -> Arrays.asList(i, i + 1))); + assertTrue(graph.hasOutlet()); + assertEquals(((Function) getAddedStage(Stage.FlatMapIterable.class, graph).getMapper()).apply(1), Arrays.asList(1, 2)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void flatMapIterableNullMapper() { + builder().flatMapIterable(null); + } + + @Test + public void limit() { + Graph graph = GraphAccessor.buildGraphFor(builder().limit(3)); + assertTrue(graph.hasOutlet()); + assertEquals(getAddedStage(Stage.Limit.class, graph).getLimit(), 3); + } + + @Test(expectedExceptions = IllegalArgumentException.class) + public void limitNegative() { + builder().limit(-1); + } + + @Test + public void skip() { + Graph graph = GraphAccessor.buildGraphFor(builder().skip(3)); + assertTrue(graph.hasOutlet()); + assertEquals(getAddedStage(Stage.Skip.class, graph).getSkip(), 3); + } + + @Test(expectedExceptions = IllegalArgumentException.class) + public void skipNegative() { + builder().skip(-1); + } + + @Test + public void takeWhile() { + Graph graph = GraphAccessor.buildGraphFor(builder().takeWhile(i -> i < 3)); + assertTrue(graph.hasOutlet()); + assertTrue(((Predicate) getAddedStage(Stage.TakeWhile.class, graph).getPredicate()).test(1)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void takeWhileNullPredicate() { + builder().takeWhile(null); + } + + @Test + public void dropWhile() { + Graph graph = GraphAccessor.buildGraphFor(builder().dropWhile(i -> i < 3)); + assertTrue(graph.hasOutlet()); + assertTrue(((Predicate) getAddedStage(Stage.DropWhile.class, graph).getPredicate()).test(1)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void dropWhileNullPredicate() { + builder().dropWhile(null); + } + + @Test + public void forEach() { + AtomicInteger received = new AtomicInteger(); + Graph graph = GraphAccessor.buildGraphFor(builder().forEach(received::set)); + assertFalse(graph.hasOutlet()); + Collector collector = getAddedStage(Stage.Collect.class, graph).getCollector(); + Object container = collector.supplier().get(); + collector.accumulator().accept(container, 1); + assertEquals(received.get(), 1); + assertNull(collector.finisher().apply(container)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void forEachNullConsumer() { + builder().forEach(null); + } + + @Test + public void ignore() { + Graph graph = GraphAccessor.buildGraphFor(builder().ignore()); + assertFalse(graph.hasOutlet()); + Collector collector = getAddedStage(Stage.Collect.class, graph).getCollector(); + Object container = collector.supplier().get(); + collector.accumulator().accept(container, 1); + assertNull(collector.finisher().apply(container)); + } + + @Test + public void cancel() { + Graph graph = GraphAccessor.buildGraphFor(builder().cancel()); + assertFalse(graph.hasOutlet()); + assertSame(getAddedStage(Stage.Cancel.class, graph), Stage.Cancel.INSTANCE); + } + + @Test + public void reduceWithIdentity() { + Graph graph = GraphAccessor.buildGraphFor(builder().reduce(1, (a, b) -> a - b)); + assertFalse(graph.hasOutlet()); + Collector collector = getAddedStage(Stage.Collect.class, graph).getCollector(); + Object container1 = collector.supplier().get(); + assertEquals(collector.finisher().apply(container1), 1); + // Create a new container because we don't necessarily want to require that the container be reusable after + // the finishers has been applied to it. + Object container2 = collector.supplier().get(); + collector.accumulator().accept(container2, 3); + assertEquals(collector.finisher().apply(container2), -2); + } + + @Test + public void reduceWithIdentityNullIdentityAllowed() { + builder().reduce(null, (a, b) -> a); + } + + @Test(expectedExceptions = NullPointerException.class) + public void reduceWithIdentityNullAccumulator() { + builder().reduce(1, null); + } + + @Test + public void reduce() { + Graph graph = GraphAccessor.buildGraphFor(builder().reduce((a, b) -> a - b)); + assertFalse(graph.hasOutlet()); + Collector collector = getAddedStage(Stage.Collect.class, graph).getCollector(); + Object container1 = collector.supplier().get(); + assertEquals(collector.finisher().apply(container1), Optional.empty()); + Object container2 = collector.supplier().get(); + collector.accumulator().accept(container2, 2); + assertEquals(collector.finisher().apply(container2), Optional.of(2)); + Object container3 = collector.supplier().get(); + collector.accumulator().accept(container3, 5); + collector.accumulator().accept(container3, 2); + assertEquals(collector.finisher().apply(container3), Optional.of(3)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void reduceNullAccumulator() { + builder().reduce(null); + } + + @Test + public void findFirst() { + Graph graph = GraphAccessor.buildGraphFor(builder().findFirst()); + assertFalse(graph.hasOutlet()); + assertSame(getAddedStage(Stage.FindFirst.class, graph), Stage.FindFirst.INSTANCE); + } + + @Test + public void collect() { + Collector collector = Collectors.toList(); + Graph graph = GraphAccessor.buildGraphFor(builder().collect(collector)); + assertFalse(graph.hasOutlet()); + assertSame(getAddedStage(Stage.Collect.class, graph).getCollector(), collector); + } + + @Test(expectedExceptions = NullPointerException.class) + public void collectNull() { + builder().collect(null); + } + + @Test + public void collectComponents() { + Supplier supplier = () -> null; + BiConsumer accumulator = (a, b) -> {}; + Graph graph = GraphAccessor.buildGraphFor(builder().collect(supplier, accumulator)); + assertFalse(graph.hasOutlet()); + Collector collector = getAddedStage(Stage.Collect.class, graph).getCollector(); + assertSame(collector.supplier(), supplier); + assertSame(collector.accumulator(), accumulator); + // Finisher must be identity function + Object myObject = new Object(); + assertSame(collector.finisher().apply(myObject), myObject); + } + + @Test(expectedExceptions = NullPointerException.class) + public void collectComponentsSupplierNull() { + builder().collect(null, (a, b) -> {}); + } + + @Test(expectedExceptions = NullPointerException.class) + public void collectComponentsAccumulatorNull() { + builder().collect(() -> null, null); + } + + @Test + public void toList() { + Graph graph = GraphAccessor.buildGraphFor(builder().toList()); + assertFalse(graph.hasOutlet()); + Collector collector = getAddedStage(Stage.Collect.class, graph).getCollector(); + Object container = collector.supplier().get(); + collector.accumulator().accept(container, 1); + collector.accumulator().accept(container, 2); + collector.accumulator().accept(container, 3); + assertEquals(collector.finisher().apply(container), Arrays.asList(1, 2, 3)); + } + + @Test + public void toSubscriber() { + Graph graph = GraphAccessor.buildGraphFor(builder().to(Mocks.SUBSCRIBER)); + assertFalse(graph.hasOutlet()); + assertSame(getAddedStage(Stage.SubscriberStage.class, graph).getRsSubscriber(), Mocks.SUBSCRIBER); + } + + @Test(expectedExceptions = NullPointerException.class) + public void toSubscriberNull() { + builder().to((Subscriber) null); + } + + @Test + public void to() { + Graph graph = GraphAccessor.buildGraphFor(builder().to(ReactiveStreams.fromSubscriber(Mocks.SUBSCRIBER))); + assertFalse(graph.hasOutlet()); + assertSame(getAddedStage(Stage.SubscriberStage.class, graph).getRsSubscriber(), Mocks.SUBSCRIBER); + } + + @Test + public void toMultipleStages() { + Graph graph = GraphAccessor.buildGraphFor(builder().to( + ReactiveStreams.builder().map(Function.identity()).cancel())); + assertTrue(graph.hasInlet()); + assertFalse(graph.hasOutlet()); + assertEquals(graph.getStages().size(), 3); + Iterator stages = graph.getStages().iterator(); + assertTrue(stages.next() instanceof Stage.Map); + assertTrue(stages.next() instanceof Stage.Map); + assertSame(stages.next(), Stage.Cancel.INSTANCE); + } + + @Test(expectedExceptions = NullPointerException.class) + public void toNull() { + builder().to((SubscriberBuilder) null); + } + + @Test + public void viaProcessor() { + Graph graph = GraphAccessor.buildGraphFor(builder().via(Mocks.PROCESSOR)); + assertTrue(graph.hasOutlet()); + assertSame(getAddedStage(Stage.ProcessorStage.class, graph).getRsProcessor(), Mocks.PROCESSOR); + } + + @Test(expectedExceptions = NullPointerException.class) + public void viaProcessorNull() { + builder().via((Processor) null); + } + + @Test + public void via() { + Graph graph = GraphAccessor.buildGraphFor(builder().via(ReactiveStreams.fromProcessor(Mocks.PROCESSOR))); + assertTrue(graph.hasOutlet()); + assertSame(getAddedStage(Stage.ProcessorStage.class, graph).getRsProcessor(), Mocks.PROCESSOR); + } + + @Test + public void viaEmpty() { + Graph graph = GraphAccessor.buildGraphFor(builder().via(ReactiveStreams.builder())); + assertTrue(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + assertEquals(graph.getStages().size(), 1); + assertTrue(graph.getStages().iterator().next() instanceof Stage.Map); + } + + @Test + public void viaMultipleStages() { + Graph graph = GraphAccessor.buildGraphFor(builder().via( + ReactiveStreams.builder().map(Function.identity()).filter(t -> true))); + assertTrue(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + assertEquals(graph.getStages().size(), 3); + Iterator stages = graph.getStages().iterator(); + assertTrue(stages.next() instanceof Stage.Map); + assertTrue(stages.next() instanceof Stage.Map); + assertTrue(stages.next() instanceof Stage.Filter); + } + + @Test(expectedExceptions = NullPointerException.class) + public void viaNull() { + builder().via((ProcessorBuilder) null); + } + + @Test + public void onError() { + Consumer consumer = t -> {}; + Graph graph = GraphAccessor.buildGraphFor(builder().onError(consumer)); + assertTrue(graph.hasOutlet()); + assertSame(getAddedStage(Stage.OnError.class, graph).getConsumer(), consumer); + } + + @Test(expectedExceptions = NullPointerException.class) + public void onErrorNullConsumer() { + builder().onError(null); + } + + @Test + public void onErrorResume() { + Graph graph = GraphAccessor.buildGraphFor(builder().onErrorResume(t -> 2)); + assertTrue(graph.hasOutlet()); + assertEquals(getAddedStage(Stage.OnErrorResume.class, graph).getFunction().apply(new RuntimeException()), 2); + } + + @Test(expectedExceptions = NullPointerException.class) + public void onErrorResumeNull() { + builder().onErrorResume(null); + } + + @Test + public void onErrorResumeWith() { + Graph graph = GraphAccessor.buildGraphFor(builder().onErrorResumeWith(t -> ReactiveStreams.empty())); + assertTrue(graph.hasOutlet()); + Graph resumeWith = getAddedStage(Stage.OnErrorResumeWith.class, graph).getFunction().apply(new RuntimeException()); + assertEquals(resumeWith.getStages(), Arrays.asList(Stage.Of.EMPTY)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void onErrorResumeWithNull() { + builder().onErrorResumeWith(null); + } + + @Test + public void onErrorResumeWithPublisher() { + Graph graph = GraphAccessor.buildGraphFor(builder().onErrorResumeWithPublisher(t -> Mocks.PUBLISHER)); + assertTrue(graph.hasOutlet()); + Graph resumeWith = getAddedStage(Stage.OnErrorResumeWith.class, graph).getFunction().apply(new RuntimeException()); + assertEquals(resumeWith.getStages().size(), 1); + assertSame(((Stage.PublisherStage) resumeWith.getStages().iterator().next()).getRsPublisher(), Mocks.PUBLISHER); + } + + @Test(expectedExceptions = NullPointerException.class) + public void onErrorResumeWithPublisherNull() { + builder().onErrorResumeWithPublisher(null); + } + + @Test + public void onTerminate() { + Runnable action = () -> {}; + Graph graph = GraphAccessor.buildGraphFor(builder().onTerminate(action)); + assertTrue(graph.hasOutlet()); + assertSame(getAddedStage(Stage.OnTerminate.class, graph).getAction(), action); + } + + @Test(expectedExceptions = NullPointerException.class) + public void onTerminateNull() { + builder().onTerminate(null); + } + + @Test + public void onComplete() { + Runnable action = () -> {}; + Graph graph = GraphAccessor.buildGraphFor(builder().onComplete(action)); + assertTrue(graph.hasOutlet()); + assertSame(getAddedStage(Stage.OnComplete.class, graph).getAction(), action); + } + + @Test(expectedExceptions = NullPointerException.class) + public void onCompleteNull() { + builder().onComplete(null); + } + + @Test + public void buildRs() { + AtomicReference builtGraph = new AtomicReference<>(); + Processor processor = builder().distinct().buildRs(new ReactiveStreamsEngine() { + @Override + public Publisher buildPublisher(Graph graph) throws UnsupportedStageException { + throw new RuntimeException("Wrong method invoked"); + } + @Override + public CompletionSubscriber buildSubscriber(Graph graph) throws UnsupportedStageException { + throw new RuntimeException("Wrong method invoked"); + } + + @Override + public Processor buildProcessor(Graph graph) throws UnsupportedStageException { + builtGraph.set(graph); + return Mocks.PROCESSOR; + } + + @Override + public CompletionStage buildCompletion(Graph graph) throws UnsupportedStageException { + throw new RuntimeException("Wrong method invoked"); + } + }); + + assertSame(processor, Mocks.PROCESSOR); + assertSame(getAddedStage(Stage.Distinct.class, builtGraph.get()), Stage.Distinct.INSTANCE); + } + + @Test(expectedExceptions = NullPointerException.class) + public void buildRsNull() { + builder().buildRs(null); + } + + @Test + public void builderShouldBeImmutable() { + ProcessorBuilder builder = builder(); + ProcessorBuilder mapped = builder.map(Function.identity()); + ProcessorBuilder distinct = builder.distinct(); + SubscriberBuilder cancelled = builder.cancel(); + getAddedStage(Stage.Map.class, GraphAccessor.buildGraphFor(mapped)); + getAddedStage(Stage.Distinct.class, GraphAccessor.buildGraphFor(distinct)); + getAddedStage(Stage.Cancel.class, GraphAccessor.buildGraphFor(cancelled)); + } + + private ProcessorBuilder builder() { + return ReactiveStreams.builder().map(Function.identity()); + } + + private S getAddedStage(Class clazz, Graph graph) { + assertTrue(graph.hasInlet(), "Graph doesn't have inlet but should because it's meant to be a processor: " + graph); + assertEquals(graph.getStages().size(), 2, "Graph does not have two stages"); + Iterator stages = graph.getStages().iterator(); + Stage first = stages.next(); + assertTrue(first instanceof Stage.Map, "First stage " + first + " is not a " + Stage.Map.class); + Stage second = stages.next(); + assertTrue(clazz.isInstance(second), "Second stage " + second + " is not a " + clazz); + return clazz.cast(second); + } + +} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/PublisherBuilderVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/PublisherBuilderVerification.java new file mode 100644 index 0000000..10a9dc5 --- /dev/null +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/PublisherBuilderVerification.java @@ -0,0 +1,579 @@ +/******************************************************************************* + * Copyright (c) 2018 Contributors to the Eclipse Foundation + * + * See the NOTICE file(s) distributed with this work for additional + * information regarding copyright ownership. + * + * 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 org.eclipse.microprofile.reactive.streams.tck.api; + +import org.eclipse.microprofile.reactive.streams.CompletionRunner; +import org.eclipse.microprofile.reactive.streams.CompletionSubscriber; +import org.eclipse.microprofile.reactive.streams.GraphAccessor; +import org.eclipse.microprofile.reactive.streams.ProcessorBuilder; +import org.eclipse.microprofile.reactive.streams.PublisherBuilder; +import org.eclipse.microprofile.reactive.streams.ReactiveStreams; +import org.eclipse.microprofile.reactive.streams.SubscriberBuilder; +import org.eclipse.microprofile.reactive.streams.spi.Graph; +import org.eclipse.microprofile.reactive.streams.spi.ReactiveStreamsEngine; +import org.eclipse.microprofile.reactive.streams.spi.Stage; +import org.eclipse.microprofile.reactive.streams.spi.UnsupportedStageException; +import org.reactivestreams.Processor; +import org.reactivestreams.Publisher; +import org.reactivestreams.Subscriber; +import org.testng.annotations.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.stream.Collector; +import java.util.stream.Collectors; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertSame; +import static org.testng.Assert.assertTrue; + +/** + * Verification for the {@link org.eclipse.microprofile.reactive.streams.PublisherBuilder} class. + */ +public class PublisherBuilderVerification { + + @Test + public void map() { + Graph graph = GraphAccessor.buildGraphFor(builder().map(i -> i + 1)); + assertTrue(graph.hasOutlet()); + assertEquals(((Function) getAddedStage(Stage.Map.class, graph).getMapper()).apply(1), 2); + } + + @Test(expectedExceptions = NullPointerException.class) + public void mapNullFunction() { + builder().map(null); + } + + @Test + public void peek() { + AtomicInteger peeked = new AtomicInteger(); + Graph graph = GraphAccessor.buildGraphFor(builder().peek(peeked::set)); + assertTrue(graph.hasOutlet()); + ((Consumer) getAddedStage(Stage.Peek.class, graph).getConsumer()).accept(1); + assertEquals(peeked.get(), 1); + } + + @Test(expectedExceptions = NullPointerException.class) + public void peekNullConsumer() { + builder().peek(null); + } + + @Test + public void filter() { + Graph graph = GraphAccessor.buildGraphFor(builder().filter(i -> i < 3)); + assertTrue(graph.hasOutlet()); + assertTrue(((Predicate) getAddedStage(Stage.Filter.class, graph).getPredicate()).test(1)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void filterNullPredicate() { + builder().filter(null); + } + + @Test + public void distinct() { + Graph graph = GraphAccessor.buildGraphFor(builder().distinct()); + assertTrue(graph.hasOutlet()); + assertSame(getAddedStage(Stage.Distinct.class, graph), Stage.Distinct.INSTANCE); + } + + @Test + public void flatMap() { + Graph graph = GraphAccessor.buildGraphFor(builder().flatMap(i -> ReactiveStreams.empty())); + assertTrue(graph.hasOutlet()); + Function flatMap = getAddedStage(Stage.FlatMap.class, graph).getMapper(); + Object result = flatMap.apply(1); + assertTrue(result instanceof Graph); + Graph innerGraph = (Graph) result; + assertFalse(innerGraph.hasInlet()); + assertTrue(innerGraph.hasOutlet()); + assertEquals(innerGraph.getStages(), Collections.singletonList(Stage.Of.EMPTY)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void flatMapNullMapper() { + builder().flatMap(null); + } + + @Test + public void flatMapPublisher() { + Graph graph = GraphAccessor.buildGraphFor(builder().flatMapPublisher(i -> Mocks.PUBLISHER)); + assertTrue(graph.hasOutlet()); + Function flatMap = getAddedStage(Stage.FlatMap.class, graph).getMapper(); + Object result = flatMap.apply(1); + assertTrue(result instanceof Graph); + Graph innerGraph = (Graph) result; + assertFalse(innerGraph.hasInlet()); + assertTrue(innerGraph.hasOutlet()); + assertEquals(innerGraph.getStages().size(), 1); + Stage inner = innerGraph.getStages().iterator().next(); + assertTrue(inner instanceof Stage.PublisherStage); + assertEquals(((Stage.PublisherStage) inner).getRsPublisher(), Mocks.PUBLISHER); + } + + @Test(expectedExceptions = NullPointerException.class) + public void flatMapPublisherNullMapper() { + builder().flatMapPublisher(null); + } + + @Test + public void flatMapCompletionStage() throws Exception { + Graph graph = GraphAccessor.buildGraphFor(builder().flatMapCompletionStage(i -> CompletableFuture.completedFuture(i + 1))); + assertTrue(graph.hasOutlet()); + CompletionStage result = (CompletionStage) ((Function) getAddedStage(Stage.FlatMapCompletionStage.class, graph).getMapper()).apply(1); + assertEquals(result.toCompletableFuture().get(1, TimeUnit.SECONDS), 2); + } + + @Test(expectedExceptions = NullPointerException.class) + public void flatMapCompletionStageNullMapper() { + builder().flatMapCompletionStage(null); + } + + @Test + public void flatMapIterable() { + Graph graph = GraphAccessor.buildGraphFor(builder().flatMapIterable(i -> Arrays.asList(i, i + 1))); + assertTrue(graph.hasOutlet()); + assertEquals(((Function) getAddedStage(Stage.FlatMapIterable.class, graph).getMapper()).apply(1), Arrays.asList(1, 2)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void flatMapIterableNullMapper() { + builder().flatMapIterable(null); + } + + @Test + public void limit() { + Graph graph = GraphAccessor.buildGraphFor(builder().limit(3)); + assertTrue(graph.hasOutlet()); + assertEquals(getAddedStage(Stage.Limit.class, graph).getLimit(), 3); + } + + @Test(expectedExceptions = IllegalArgumentException.class) + public void limitNegative() { + builder().limit(-1); + } + + @Test + public void skip() { + Graph graph = GraphAccessor.buildGraphFor(builder().skip(3)); + assertTrue(graph.hasOutlet()); + assertEquals(getAddedStage(Stage.Skip.class, graph).getSkip(), 3); + } + + @Test(expectedExceptions = IllegalArgumentException.class) + public void skipNegative() { + builder().skip(-1); + } + + @Test + public void takeWhile() { + Graph graph = GraphAccessor.buildGraphFor(builder().takeWhile(i -> i < 3)); + assertTrue(graph.hasOutlet()); + assertTrue(((Predicate) getAddedStage(Stage.TakeWhile.class, graph).getPredicate()).test(1)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void takeWhileNullPredicate() { + builder().takeWhile(null); + } + + @Test + public void dropWhile() { + Graph graph = GraphAccessor.buildGraphFor(builder().dropWhile(i -> i < 3)); + assertTrue(graph.hasOutlet()); + assertTrue(((Predicate) getAddedStage(Stage.DropWhile.class, graph).getPredicate()).test(1)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void dropWhileNullPredicate() { + builder().dropWhile(null); + } + + @Test + public void forEach() { + AtomicInteger received = new AtomicInteger(); + Graph graph = GraphAccessor.buildGraphFor(builder().forEach(received::set)); + assertFalse(graph.hasOutlet()); + Collector collector = getAddedStage(Stage.Collect.class, graph).getCollector(); + Object container = collector.supplier().get(); + collector.accumulator().accept(container, 1); + assertEquals(received.get(), 1); + assertNull(collector.finisher().apply(container)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void forEachNullConsumer() { + builder().forEach(null); + } + + @Test + public void ignore() { + Graph graph = GraphAccessor.buildGraphFor(builder().ignore()); + assertFalse(graph.hasOutlet()); + Collector collector = getAddedStage(Stage.Collect.class, graph).getCollector(); + Object container = collector.supplier().get(); + collector.accumulator().accept(container, 1); + assertNull(collector.finisher().apply(container)); + } + + @Test + public void cancel() { + Graph graph = GraphAccessor.buildGraphFor(builder().cancel()); + assertFalse(graph.hasOutlet()); + assertSame(getAddedStage(Stage.Cancel.class, graph), Stage.Cancel.INSTANCE); + } + + @Test + public void reduceWithIdentity() { + Graph graph = GraphAccessor.buildGraphFor(builder().reduce(1, (a, b) -> a - b)); + assertFalse(graph.hasOutlet()); + Collector collector = getAddedStage(Stage.Collect.class, graph).getCollector(); + Object container1 = collector.supplier().get(); + assertEquals(collector.finisher().apply(container1), 1); + // Create a new container because we don't necessarily want to require that the container be reusable after + // the finishers has been applied to it. + Object container2 = collector.supplier().get(); + collector.accumulator().accept(container2, 3); + assertEquals(collector.finisher().apply(container2), -2); + } + + @Test + public void reduceWithIdentityNullIdentityAllowed() { + builder().reduce(null, (a, b) -> a); + } + + @Test(expectedExceptions = NullPointerException.class) + public void reduceWithIdentityNullAccumulator() { + builder().reduce(1, null); + } + + @Test + public void reduce() { + Graph graph = GraphAccessor.buildGraphFor(builder().reduce((a, b) -> a - b)); + assertFalse(graph.hasOutlet()); + Collector collector = getAddedStage(Stage.Collect.class, graph).getCollector(); + Object container1 = collector.supplier().get(); + assertEquals(collector.finisher().apply(container1), Optional.empty()); + Object container2 = collector.supplier().get(); + collector.accumulator().accept(container2, 2); + assertEquals(collector.finisher().apply(container2), Optional.of(2)); + Object container3 = collector.supplier().get(); + collector.accumulator().accept(container3, 5); + collector.accumulator().accept(container3, 2); + assertEquals(collector.finisher().apply(container3), Optional.of(3)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void reduceNullAccumulator() { + builder().reduce(null); + } + + @Test + public void findFirst() { + Graph graph = GraphAccessor.buildGraphFor(builder().findFirst()); + assertFalse(graph.hasOutlet()); + assertSame(getAddedStage(Stage.FindFirst.class, graph), Stage.FindFirst.INSTANCE); + } + + @Test + public void collect() { + Collector collector = Collectors.toList(); + Graph graph = GraphAccessor.buildGraphFor(builder().collect(collector)); + assertFalse(graph.hasOutlet()); + assertSame(getAddedStage(Stage.Collect.class, graph).getCollector(), collector); + } + + @Test(expectedExceptions = NullPointerException.class) + public void collectNull() { + builder().collect(null); + } + + @Test + public void collectComponents() { + Supplier supplier = () -> null; + BiConsumer accumulator = (a, b) -> {}; + Graph graph = GraphAccessor.buildGraphFor(builder().collect(supplier, accumulator)); + assertFalse(graph.hasOutlet()); + Collector collector = getAddedStage(Stage.Collect.class, graph).getCollector(); + assertSame(collector.supplier(), supplier); + assertSame(collector.accumulator(), accumulator); + // Finisher must be identity function + Object myObject = new Object(); + assertSame(collector.finisher().apply(myObject), myObject); + } + + @Test(expectedExceptions = NullPointerException.class) + public void collectComponentsSupplierNull() { + builder().collect(null, (a, b) -> {}); + } + + @Test(expectedExceptions = NullPointerException.class) + public void collectComponentsAccumulatorNull() { + builder().collect(() -> null, null); + } + + @Test + public void toList() { + Graph graph = GraphAccessor.buildGraphFor(builder().toList()); + assertFalse(graph.hasOutlet()); + Collector collector = getAddedStage(Stage.Collect.class, graph).getCollector(); + Object container = collector.supplier().get(); + collector.accumulator().accept(container, 1); + collector.accumulator().accept(container, 2); + collector.accumulator().accept(container, 3); + assertEquals(collector.finisher().apply(container), Arrays.asList(1, 2, 3)); + } + + @Test + public void toSubscriber() { + Graph graph = GraphAccessor.buildGraphFor(builder().to(Mocks.SUBSCRIBER)); + assertFalse(graph.hasOutlet()); + assertSame(getAddedStage(Stage.SubscriberStage.class, graph).getRsSubscriber(), Mocks.SUBSCRIBER); + } + + @Test(expectedExceptions = NullPointerException.class) + public void toSubscriberNull() { + builder().to((Subscriber) null); + } + + @Test + public void to() { + Graph graph = GraphAccessor.buildGraphFor(builder().to(ReactiveStreams.fromSubscriber(Mocks.SUBSCRIBER))); + assertFalse(graph.hasOutlet()); + assertSame(getAddedStage(Stage.SubscriberStage.class, graph).getRsSubscriber(), Mocks.SUBSCRIBER); + } + + @Test + public void toMultipleStages() { + Graph graph = GraphAccessor.buildGraphFor(builder().to( + ReactiveStreams.builder().map(Function.identity()).cancel())); + assertFalse(graph.hasInlet()); + assertFalse(graph.hasOutlet()); + assertEquals(graph.getStages().size(), 3); + Iterator stages = graph.getStages().iterator(); + assertTrue(stages.next() instanceof Stage.Of); + assertTrue(stages.next() instanceof Stage.Map); + assertSame(stages.next(), Stage.Cancel.INSTANCE); + } + + @Test(expectedExceptions = NullPointerException.class) + public void toNull() { + builder().to((SubscriberBuilder) null); + } + + @Test + public void viaProcessor() { + Graph graph = GraphAccessor.buildGraphFor(builder().via(Mocks.PROCESSOR)); + assertTrue(graph.hasOutlet()); + assertSame(getAddedStage(Stage.ProcessorStage.class, graph).getRsProcessor(), Mocks.PROCESSOR); + } + + @Test(expectedExceptions = NullPointerException.class) + public void viaProcessorNull() { + builder().via((Processor) null); + } + + @Test + public void via() { + Graph graph = GraphAccessor.buildGraphFor(builder().via(ReactiveStreams.fromProcessor(Mocks.PROCESSOR))); + assertTrue(graph.hasOutlet()); + assertSame(getAddedStage(Stage.ProcessorStage.class, graph).getRsProcessor(), Mocks.PROCESSOR); + } + + @Test + public void viaEmpty() { + Graph graph = GraphAccessor.buildGraphFor(builder().via(ReactiveStreams.builder())); + assertFalse(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + assertEquals(graph.getStages().size(), 1); + assertTrue(graph.getStages().iterator().next() instanceof Stage.Of); + } + + @Test + public void viaMultipleStages() { + Graph graph = GraphAccessor.buildGraphFor(builder().via( + ReactiveStreams.builder().map(Function.identity()).filter(t -> true))); + assertFalse(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + assertEquals(graph.getStages().size(), 3); + Iterator stages = graph.getStages().iterator(); + assertTrue(stages.next() instanceof Stage.Of); + assertTrue(stages.next() instanceof Stage.Map); + assertTrue(stages.next() instanceof Stage.Filter); + } + + @Test(expectedExceptions = NullPointerException.class) + public void viaNull() { + builder().via((ProcessorBuilder) null); + } + + @Test + public void onError() { + Consumer consumer = t -> {}; + Graph graph = GraphAccessor.buildGraphFor(builder().onError(consumer)); + assertTrue(graph.hasOutlet()); + assertSame(getAddedStage(Stage.OnError.class, graph).getConsumer(), consumer); + } + + @Test(expectedExceptions = NullPointerException.class) + public void onErrorNullConsumer() { + builder().onError(null); + } + + @Test + public void onErrorResume() { + Graph graph = GraphAccessor.buildGraphFor(builder().onErrorResume(t -> 2)); + assertTrue(graph.hasOutlet()); + assertEquals(getAddedStage(Stage.OnErrorResume.class, graph).getFunction().apply(new RuntimeException()), 2); + } + + @Test(expectedExceptions = NullPointerException.class) + public void onErrorResumeNull() { + builder().onErrorResume(null); + } + + @Test + public void onErrorResumeWith() { + Graph graph = GraphAccessor.buildGraphFor(builder().onErrorResumeWith(t -> ReactiveStreams.empty())); + assertTrue(graph.hasOutlet()); + Graph resumeWith = getAddedStage(Stage.OnErrorResumeWith.class, graph).getFunction().apply(new RuntimeException()); + assertEquals(resumeWith.getStages(), Arrays.asList(Stage.Of.EMPTY)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void onErrorResumeWithNull() { + builder().onErrorResumeWith(null); + } + + @Test + public void onErrorResumeWithPublisher() { + Graph graph = GraphAccessor.buildGraphFor(builder().onErrorResumeWithPublisher(t -> Mocks.PUBLISHER)); + assertTrue(graph.hasOutlet()); + Graph resumeWith = getAddedStage(Stage.OnErrorResumeWith.class, graph).getFunction().apply(new RuntimeException()); + assertEquals(resumeWith.getStages().size(), 1); + assertSame(((Stage.PublisherStage) resumeWith.getStages().iterator().next()).getRsPublisher(), Mocks.PUBLISHER); + } + + @Test(expectedExceptions = NullPointerException.class) + public void onErrorResumeWithPublisherNull() { + builder().onErrorResumeWithPublisher(null); + } + + @Test + public void onTerminate() { + Runnable action = () -> {}; + Graph graph = GraphAccessor.buildGraphFor(builder().onTerminate(action)); + assertTrue(graph.hasOutlet()); + assertSame(getAddedStage(Stage.OnTerminate.class, graph).getAction(), action); + } + + @Test(expectedExceptions = NullPointerException.class) + public void onTerminateNull() { + builder().onTerminate(null); + } + + @Test + public void onComplete() { + Runnable action = () -> {}; + Graph graph = GraphAccessor.buildGraphFor(builder().onComplete(action)); + assertTrue(graph.hasOutlet()); + assertSame(getAddedStage(Stage.OnComplete.class, graph).getAction(), action); + } + + @Test(expectedExceptions = NullPointerException.class) + public void onCompleteNull() { + builder().onComplete(null); + } + + @Test + public void buildRs() { + AtomicReference builtGraph = new AtomicReference<>(); + Publisher publisher = builder().distinct().buildRs(new ReactiveStreamsEngine() { + @Override + public Publisher buildPublisher(Graph graph) throws UnsupportedStageException { + builtGraph.set(graph); + return Mocks.PUBLISHER; + } + @Override + public CompletionSubscriber buildSubscriber(Graph graph) throws UnsupportedStageException { + throw new RuntimeException("Wrong method invoked"); + } + + @Override + public Processor buildProcessor(Graph graph) throws UnsupportedStageException { + throw new RuntimeException("Wrong method invoked"); + } + + @Override + public CompletionStage buildCompletion(Graph graph) throws UnsupportedStageException { + throw new RuntimeException("Wrong method invoked"); + } + }); + + assertSame(publisher, Mocks.PUBLISHER); + assertSame(getAddedStage(Stage.Distinct.class, builtGraph.get()), Stage.Distinct.INSTANCE); + } + + @Test(expectedExceptions = NullPointerException.class) + public void buildRsNull() { + builder().buildRs(null); + } + + @Test + public void builderShouldBeImmutable() { + PublisherBuilder builder = builder(); + PublisherBuilder mapped = builder.map(Function.identity()); + PublisherBuilder distinct = builder.distinct(); + CompletionRunner cancelled = builder.cancel(); + getAddedStage(Stage.Map.class, GraphAccessor.buildGraphFor(mapped)); + getAddedStage(Stage.Distinct.class, GraphAccessor.buildGraphFor(distinct)); + getAddedStage(Stage.Cancel.class, GraphAccessor.buildGraphFor(cancelled)); + } + + private PublisherBuilder builder() { + return ReactiveStreams.of(1); + } + + private S getAddedStage(Class clazz, Graph graph) { + assertFalse(graph.hasInlet(), "Graph has inlet but shouldn't because it's meant to be a publisher: " + graph); + assertEquals(graph.getStages().size(), 2, "Graph does not have two stages"); + Iterator stages = graph.getStages().iterator(); + Stage first = stages.next(); + assertTrue(first instanceof Stage.Of, "First stage " + first + " is not a " + Stage.Of.class); + Stage second = stages.next(); + assertTrue(clazz.isInstance(second), "Second stage " + second + " is not a " + clazz); + return clazz.cast(second); + } + +} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/ReactiveStreamsApiVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/ReactiveStreamsApiVerification.java new file mode 100644 index 0000000..2fa27c0 --- /dev/null +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/ReactiveStreamsApiVerification.java @@ -0,0 +1,44 @@ +/******************************************************************************* + * Copyright (c) 2018 Contributors to the Eclipse Foundation + * + * See the NOTICE file(s) distributed with this work for additional + * information regarding copyright ownership. + * + * 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 org.eclipse.microprofile.reactive.streams.tck.api; + +import java.util.Arrays; +import java.util.List; + +/** + * Factory for tests that verify the API itself without dependence on an SPI implementation. + *

+ * These tests serve not only to test the API provided by the MicroProfile API project, but also to test clean room + * implementations of the API, that is, implementations that don't depend on the MicroProfile API artifact. + */ +public class ReactiveStreamsApiVerification { + + public List allTests() { + return Arrays.asList( + new ReactiveStreamsVerification(), + new GraphAccessorVerification(), + new PublisherBuilderVerification(), + new ProcessorBuilderVerification(), + new SubscriberBuilderVerification(), + new CompletionRunnerVerification(), + new CompletionSubscriberVerification() + ); + } +} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/ReactiveStreamsVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/ReactiveStreamsVerification.java new file mode 100644 index 0000000..e83d1bc --- /dev/null +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/ReactiveStreamsVerification.java @@ -0,0 +1,219 @@ +/******************************************************************************* + * Copyright (c) 2018 Contributors to the Eclipse Foundation + * + * See the NOTICE file(s) distributed with this work for additional + * information regarding copyright ownership. + * + * 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 org.eclipse.microprofile.reactive.streams.tck.api; + +import org.eclipse.microprofile.reactive.streams.GraphAccessor; +import org.eclipse.microprofile.reactive.streams.ReactiveStreams; +import org.eclipse.microprofile.reactive.streams.spi.Graph; +import org.eclipse.microprofile.reactive.streams.spi.Stage; +import org.testng.annotations.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertSame; +import static org.testng.Assert.assertTrue; + +/** + * Verification for the {@link ReactiveStreams} class. + */ +public class ReactiveStreamsVerification { + + @Test + public void fromPublisher() { + Graph graph = GraphAccessor.buildGraphFor(ReactiveStreams.fromPublisher(Mocks.PUBLISHER)); + assertFalse(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + assertSame(getStage(Stage.PublisherStage.class, graph).getRsPublisher(), Mocks.PUBLISHER); + } + + @Test(expectedExceptions = NullPointerException.class) + public void fromPublisherNull() { + ReactiveStreams.fromPublisher(null); + } + + @Test + public void ofSingle() { + Graph graph = GraphAccessor.buildGraphFor(ReactiveStreams.of("foo")); + assertFalse(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + assertEquals(getStage(Stage.Of.class, graph).getElements(), Collections.singletonList("foo")); + } + + @Test + public void ofVarArgs() { + Graph graph = GraphAccessor.buildGraphFor(ReactiveStreams.of("a", "b", "c")); + assertFalse(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + assertEquals(getStage(Stage.Of.class, graph).getElements(), Arrays.asList("a", "b", "c")); + } + + @Test + public void empty() { + Graph graph = GraphAccessor.buildGraphFor(ReactiveStreams.empty()); + assertFalse(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + assertSame(getStage(Stage.Of.class, graph), Stage.Of.EMPTY, "Empty stage is not Stage.Of.EMPTY"); + } + + @Test + public void ofNullableNull() { + Graph graph = GraphAccessor.buildGraphFor(ReactiveStreams.ofNullable(null)); + assertFalse(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + assertSame(getStage(Stage.Of.class, graph), Stage.Of.EMPTY, "ofNullable(null) stage is not Stage.Of.EMPTY"); + } + + @Test + public void ofNullableNonNull() { + Graph graph = GraphAccessor.buildGraphFor(ReactiveStreams.ofNullable("foo")); + assertFalse(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + assertEquals(getStage(Stage.Of.class, graph).getElements(), Collections.singletonList("foo")); + } + + @Test + public void fromIterable() { + Graph graph = GraphAccessor.buildGraphFor(ReactiveStreams.fromIterable(Arrays.asList("a", "b", "c"))); + assertFalse(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + assertEquals(getStage(Stage.Of.class, graph).getElements(), Arrays.asList("a", "b", "c")); + } + + @Test(expectedExceptions = NullPointerException.class) + public void fromIterableNull() { + ReactiveStreams.fromIterable(null); + } + + @Test + public void failed() { + Graph graph = GraphAccessor.buildGraphFor(ReactiveStreams.failed(new Exception("failed"))); + assertFalse(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + assertEquals(getStage(Stage.Failed.class, graph).getError().getMessage(), "failed"); + } + + @Test(expectedExceptions = NullPointerException.class) + public void failedNull() { + ReactiveStreams.failed(null); + } + + @Test + public void builder() { + Graph graph = GraphAccessor.buildGraphFor(ReactiveStreams.builder()); + assertTrue(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + assertEquals(graph.getStages(), Collections.emptyList(), "Identity builder should have an empty list of stages"); + } + + @Test + public void fromProcessor() { + Graph graph = GraphAccessor.buildGraphFor(ReactiveStreams.fromProcessor(Mocks.PROCESSOR)); + assertTrue(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + assertSame(getStage(Stage.ProcessorStage.class, graph).getRsProcessor(), Mocks.PROCESSOR); + } + + @Test(expectedExceptions = NullPointerException.class) + public void fromProcessorNull() { + ReactiveStreams.fromProcessor(null); + } + + @Test + public void fromSubscriber() { + Graph graph = GraphAccessor.buildGraphFor(ReactiveStreams.fromSubscriber(Mocks.SUBSCRIBER)); + assertTrue(graph.hasInlet()); + assertFalse(graph.hasOutlet()); + assertSame(getStage(Stage.SubscriberStage.class, graph).getRsSubscriber(), Mocks.SUBSCRIBER); + } + + @Test(expectedExceptions = NullPointerException.class) + public void fromSubscriberNull() { + ReactiveStreams.fromSubscriber(null); + } + + @Test + public void iterate() { + Graph graph = GraphAccessor.buildGraphFor(ReactiveStreams.iterate(1, i -> i + 1)); + assertFalse(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + Iterator iter = getStage(Stage.Of.class, graph).getElements().iterator(); + assertTrue(iter.hasNext()); + assertEquals(iter.next(), 1); + assertTrue(iter.hasNext()); + assertEquals(iter.next(), 2); + assertTrue(iter.hasNext()); + assertEquals(iter.next(), 3); + } + + @Test(expectedExceptions = NullPointerException.class) + public void iterateNullOperator() { + ReactiveStreams.iterate(1, null); + } + + @Test + public void generate() { + Graph graph = GraphAccessor.buildGraphFor(ReactiveStreams.generate(() -> 1)); + assertFalse(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + Iterator iter = getStage(Stage.Of.class, graph).getElements().iterator(); + assertTrue(iter.hasNext()); + assertEquals(iter.next(), 1); + assertTrue(iter.hasNext()); + assertEquals(iter.next(), 1); + assertTrue(iter.hasNext()); + assertEquals(iter.next(), 1); + } + + @Test(expectedExceptions = NullPointerException.class) + public void generateNullSupplier() { + ReactiveStreams.generate(null); + } + + @Test + public void concat() { + Graph graph = GraphAccessor.buildGraphFor(ReactiveStreams.concat(ReactiveStreams.empty(), ReactiveStreams.of(1))); + assertFalse(graph.hasInlet()); + assertTrue(graph.hasOutlet()); + Stage.Concat concat = getStage(Stage.Concat.class, graph); + assertSame(getStage(Stage.Of.class, concat.getFirst()), Stage.Of.EMPTY); + assertEquals(getStage(Stage.Of.class, concat.getSecond()).getElements(), Collections.singletonList(1)); + } + + @Test(expectedExceptions = NullPointerException.class) + public void concatFirstNull() { + ReactiveStreams.concat(null, ReactiveStreams.empty()); + } + + @Test(expectedExceptions = NullPointerException.class) + public void concatSecondNull() { + ReactiveStreams.concat(ReactiveStreams.empty(), null); + } + + private S getStage(Class clazz, Graph graph) { + assertEquals(graph.getStages().size(), 1, "Graph does not have a single stage"); + Stage s = graph.getStages().iterator().next(); + assertTrue(clazz.isInstance(s), "Stage " + s + " is not a " + clazz); + return clazz.cast(s); + } +} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/SubscriberBuilderVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/SubscriberBuilderVerification.java new file mode 100644 index 0000000..9c29f9e --- /dev/null +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/api/SubscriberBuilderVerification.java @@ -0,0 +1,82 @@ +/******************************************************************************* + * Copyright (c) 2018 Contributors to the Eclipse Foundation + * + * See the NOTICE file(s) distributed with this work for additional + * information regarding copyright ownership. + * + * 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 org.eclipse.microprofile.reactive.streams.tck.api; + +import org.eclipse.microprofile.reactive.streams.CompletionSubscriber; +import org.eclipse.microprofile.reactive.streams.ReactiveStreams; +import org.eclipse.microprofile.reactive.streams.spi.Graph; +import org.eclipse.microprofile.reactive.streams.spi.ReactiveStreamsEngine; +import org.eclipse.microprofile.reactive.streams.spi.Stage; +import org.eclipse.microprofile.reactive.streams.spi.UnsupportedStageException; +import org.reactivestreams.Processor; +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.atomic.AtomicReference; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertSame; +import static org.testng.Assert.assertTrue; + +/** + * Verification for the {@link org.eclipse.microprofile.reactive.streams.SubscriberBuilder} class. + */ +public class SubscriberBuilderVerification { + + @Test + public void build() { + AtomicReference builtGraph = new AtomicReference<>(); + CompletionSubscriber expected = CompletionSubscriber.of(Mocks.SUBSCRIBER, new CompletableFuture()); + CompletionSubscriber returned = ReactiveStreams.builder().cancel().build(new ReactiveStreamsEngine() { + @Override + public Publisher buildPublisher(Graph graph) throws UnsupportedStageException { + throw new RuntimeException("Wrong method invoked"); + } + @Override + public CompletionSubscriber buildSubscriber(Graph graph) throws UnsupportedStageException { + builtGraph.set(graph); + return expected; + } + @Override + public Processor buildProcessor(Graph graph) throws UnsupportedStageException { + throw new RuntimeException("Wrong method invoked"); + } + + @Override + public CompletionStage buildCompletion(Graph graph) throws UnsupportedStageException { + throw new RuntimeException("Wrong method invoked"); + } + }); + + assertSame(returned, expected); + assertTrue(builtGraph.get().hasInlet()); + assertFalse(builtGraph.get().hasOutlet()); + assertEquals(builtGraph.get().getStages().size(), 1); + assertSame(builtGraph.get().getStages().iterator().next(), Stage.Cancel.INSTANCE); + } + + @Test(expectedExceptions = NullPointerException.class) + public void buildNull() { + ReactiveStreams.builder().cancel().build(null); + } +} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/AbstractStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/AbstractStageVerification.java similarity index 81% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/AbstractStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/AbstractStageVerification.java index 7e2a6da..f698a65 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/AbstractStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/AbstractStageVerification.java @@ -17,8 +17,9 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; +import org.eclipse.microprofile.reactive.streams.PublisherBuilder; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.eclipse.microprofile.reactive.streams.spi.ReactiveStreamsEngine; import org.reactivestreams.Publisher; @@ -35,14 +36,20 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.IntStream; +/** + * All stage verifications should inherit from this. This provides access to dependencies of the TCK, as well as some + * convenient helper functions. + */ abstract class AbstractStageVerification { private final ReactiveStreamsEngine engine; private final TestEnvironment environment; private final ScheduledExecutorService executorService; - AbstractStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + AbstractStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { this.engine = deps.engine(); this.environment = deps.testEnvironment(); this.executorService = deps.executorService(); @@ -56,8 +63,14 @@ ScheduledExecutorService getExecutorService() { return executorService; } + /** + * A stage verification may return multiple tests here for doing Reactive Streams TCK verifications. + */ abstract List reactiveStreamsTckVerifiers(); + /** + * Wait for the given future to complete and return its value, using the configured timeout. + */ T await(CompletionStage future) { try { return future.toCompletableFuture().get(environment.defaultTimeoutMillis(), TimeUnit.MILLISECONDS); @@ -78,6 +91,15 @@ T await(CompletionStage future) { } } + /** + * An infinite stream of integers starting from one. + */ + PublisherBuilder infiniteStream() { + return ReactiveStreams.fromIterable(() -> { + AtomicInteger value = new AtomicInteger(); + return IntStream.generate(value::incrementAndGet).boxed().iterator(); + }); + } abstract class StagePublisherVerification extends PublisherVerification { diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/CancelStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/CancelStageVerification.java similarity index 80% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/CancelStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/CancelStageVerification.java index b9060f9..3ab896d 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/CancelStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/CancelStageVerification.java @@ -17,9 +17,10 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; +import org.eclipse.microprofile.reactive.streams.SubscriberBuilder; import org.reactivestreams.Subscriber; import org.reactivestreams.Subscription; import org.testng.SkipException; @@ -30,8 +31,12 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; +/** + * Verification for the cancel stage. + */ public class CancelStageVerification extends AbstractStageVerification { - CancelStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + + CancelStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @@ -52,6 +57,19 @@ public void cancel() { await(result); } + @Test + public void cancelStageShouldIgnoreAnyUpstreamFailures() { + await(ReactiveStreams.failed(new QuietRuntimeException()) + .cancel().run(getEngine())); + } + + @Test + public void cancelSubscriberBuilderShouldBeReusable() { + SubscriberBuilder cancel = ReactiveStreams.builder().cancel(); + await(ReactiveStreams.of("a").to(cancel).run(getEngine())); + await(ReactiveStreams.of("b").to(cancel).run(getEngine())); + } + @Override List reactiveStreamsTckVerifiers() { return Collections.singletonList(new SubscriberVerification()); diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/CollectStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/CollectStageVerification.java similarity index 52% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/CollectStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/CollectStageVerification.java index ba1084b..96145ab 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/CollectStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/CollectStageVerification.java @@ -17,23 +17,32 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; +import org.eclipse.microprofile.reactive.streams.SubscriberBuilder; import org.reactivestreams.Subscriber; import org.testng.annotations.Test; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collector; import java.util.stream.Collectors; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.fail; +/** + * Verification for the collect stage. + */ public class CollectStageVerification extends AbstractStageVerification { - CollectStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + CollectStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @@ -67,9 +76,9 @@ public void collectShouldSupportEmptyStreams() { ).run(getEngine())).get(), 42); } - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") - public void collectShouldPropagateErrors() { - await(ReactiveStreams.failed(new RuntimeException("failed")) + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void collectShouldPropagateUpstreamErrors() { + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) .collect( () -> new AtomicInteger(0), AtomicInteger::addAndGet @@ -83,12 +92,66 @@ public void finisherFunctionShouldBeInvoked() { .collect(Collectors.joining(", ")).run(getEngine())), "1, 2, 3"); } - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") - public void toListStageShouldPropagateErrors() { - await(ReactiveStreams.failed(new RuntimeException("failed")) + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void toListStageShouldPropagateUpstreamErrors() { + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) .toList().run(getEngine())); } + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void collectStageShouldPropagateErrorsFromSupplierThroughCompletionStage() { + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage result = null; + try { + result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) + .collect(Collector.of(() -> { + throw new QuietRuntimeException("failed"); + }, (a, b) -> { + }, (a, b) -> a + b, Function.identity())) + .run(getEngine()); + } + catch (Exception e) { + fail("Exception thrown directly from stream, it should have been captured by the returned CompletionStage", e); + } + await(cancelled); + await(result); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void collectStageShouldPropagateErrorsFromAccumulator() { + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) + .collect(Collector.of(() -> "", (a, b) -> { + throw new QuietRuntimeException("failed"); + }, (a, b) -> a + b, Function.identity())) + .run(getEngine()); + await(cancelled); + await(result); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void collectStageShouldPropagateErrorsFromFinisher() { + CompletionStage result = ReactiveStreams.of(1, 2, 3) + .collect(Collector.of(() -> 0, (a, b) -> { + }, + (a, b) -> a + b, + r -> { + throw new QuietRuntimeException("failed"); + })) + .run(getEngine()); + await(result); + } + + @Test + public void collectStageBuilderShouldBeReusable() { + SubscriberBuilder> toList = ReactiveStreams.builder().toList(); + assertEquals(await(ReactiveStreams.of(1, 2, 3).to(toList).run(getEngine())), Arrays.asList(1, 2, 3)); + assertEquals(await(ReactiveStreams.of(4, 5, 6).to(toList).run(getEngine())), Arrays.asList(4, 5, 6)); + } + + @Override List reactiveStreamsTckVerifiers() { return Arrays.asList(new ToListSubscriberVerification(), new CollectSubscriberVerification()); diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/ConcatStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/ConcatStageVerification.java new file mode 100644 index 0000000..55f3909 --- /dev/null +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/ConcatStageVerification.java @@ -0,0 +1,199 @@ +/******************************************************************************* + * Copyright (c) 2018 Contributors to the Eclipse Foundation + * + * See the NOTICE file(s) distributed with this work for additional + * information regarding copyright ownership. + * + * 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 org.eclipse.microprofile.reactive.streams.tck.spi; + +import org.eclipse.microprofile.reactive.streams.PublisherBuilder; +import org.eclipse.microprofile.reactive.streams.ReactiveStreams; +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.stream.LongStream; + +import static org.testng.Assert.assertEquals; + +/** + * Verification for the concat stage. + */ +public class ConcatStageVerification extends AbstractStageVerification { + + ConcatStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { + super(deps); + } + + @Test + public void concatStageShouldConcatTwoGraphs() { + assertEquals(await( + ReactiveStreams.concat( + ReactiveStreams.of(1, 2, 3), + ReactiveStreams.of(4, 5, 6) + ) + .toList() + .run(getEngine()) + ), Arrays.asList(1, 2, 3, 4, 5, 6)); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void concatStageShouldCancelSecondStageIfFirstFails() { + CompletableFuture cancelled = new CompletableFuture<>(); + + CompletionStage completion = ReactiveStreams.concat( + ReactiveStreams.failed(new QuietRuntimeException("failed")), + infiniteStream().onTerminate(() -> cancelled.complete(null)) + ) + .ignore() + .run(getEngine()); + + await(cancelled); + await(completion); + } + + @Test + public void concatStageShouldCancelSecondStageIfFirstCancellationOccursDuringFirst() { + CompletableFuture cancelled = new CompletableFuture<>(); + + CompletionStage> result = ReactiveStreams.concat( + infiniteStream(), + infiniteStream().onTerminate(() -> cancelled.complete(null)) + ) + .limit(5) + .toList() + .run(getEngine()); + + await(cancelled); + assertEquals(await(result), Arrays.asList(1, 2, 3, 4, 5)); + } + + @Test + public void concatStageShouldCancelSecondStageIfCancellationOccursDuringSecond() { + CompletableFuture cancelled = new CompletableFuture<>(); + + CompletionStage> result = ReactiveStreams.concat( + ReactiveStreams.of(1, 2, 3), + infiniteStream().onTerminate(() -> cancelled.complete(null)) + ) + .limit(5) + .toList() + .run(getEngine()); + + await(cancelled); + assertEquals(await(result), Arrays.asList(1, 2, 3, 1, 2)); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void concatStageShouldPropagateExceptionsFromSecondStage() { + await( + ReactiveStreams.concat( + ReactiveStreams.of(1, 2, 3), + ReactiveStreams.failed(new QuietRuntimeException("failed")) + ).toList().run(getEngine()) + ); + } + + @Test + public void concatStageShouldWorkWithEmptyFirstGraph() { + assertEquals(await( + ReactiveStreams.concat( + ReactiveStreams.empty(), + ReactiveStreams.of(1, 2, 3) + ) + .toList() + .run(getEngine()) + ), Arrays.asList(1, 2, 3)); + } + + @Test + public void concatStageShouldWorkWithEmptySecondGraph() { + assertEquals(await( + ReactiveStreams.concat( + ReactiveStreams.of(1, 2, 3), + ReactiveStreams.empty() + ) + .toList() + .run(getEngine()) + ), Arrays.asList(1, 2, 3)); + } + + @Test + public void concatStageShouldWorkWithBothGraphsEmpty() { + assertEquals(await( + ReactiveStreams.concat( + ReactiveStreams.empty(), + ReactiveStreams.empty() + ) + .toList() + .run(getEngine()) + ), Collections.emptyList()); + } + + @Test + public void concatStageShouldSupportNestedConcats() { + assertEquals(await( + ReactiveStreams.concat( + ReactiveStreams.concat( + ReactiveStreams.of(1, 2, 3), + ReactiveStreams.of(4, 5, 6) + ), + ReactiveStreams.concat( + ReactiveStreams.of(7, 8, 9), + ReactiveStreams.of(10, 11, 12) + ) + ) + .toList() + .run(getEngine()) + ), Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)); + } + + @Test + public void concatStageBuilderShouldBeReusable() { + PublisherBuilder concated = ReactiveStreams.concat( + ReactiveStreams.of(1, 2, 3), + ReactiveStreams.of(4, 5, 6) + ); + assertEquals(await(concated.toList().run(getEngine())), Arrays.asList(1, 2, 3, 4, 5, 6)); + assertEquals(await(concated.toList().run(getEngine())), Arrays.asList(1, 2, 3, 4, 5, 6)); + } + + @Override + List reactiveStreamsTckVerifiers() { + return Collections.singletonList(new PublisherVerification()); + } + + class PublisherVerification extends StagePublisherVerification { + @Override + public Publisher createPublisher(long elements) { + long toEmitFromFirst = elements / 2; + + return ReactiveStreams.concat( + ReactiveStreams.fromIterable( + () -> LongStream.rangeClosed(1, toEmitFromFirst).boxed().iterator() + ), + ReactiveStreams.fromIterable( + () -> LongStream.rangeClosed(toEmitFromFirst + 1, elements).boxed().iterator() + ) + ).buildRs(getEngine()); + } + } + +} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/DistinctStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/DistinctStageVerification.java similarity index 55% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/DistinctStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/DistinctStageVerification.java index 7a98e1b..c6869b8 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/DistinctStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/DistinctStageVerification.java @@ -17,8 +17,9 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; +import org.eclipse.microprofile.reactive.streams.ProcessorBuilder; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Processor; import org.reactivestreams.Publisher; @@ -27,12 +28,17 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; import static org.testng.Assert.assertEquals; +/** + * Verification for the distinct stage. + */ public class DistinctStageVerification extends AbstractStageVerification { - DistinctStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + DistinctStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @@ -52,14 +58,51 @@ public void distinctStageShouldReturnAnEmptyStreamWhenCalledOnEmptyStreams() { .run(getEngine())), Collections.emptyList()); } - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") - public void distinctStageShouldPropagateExceptions() { - await(ReactiveStreams.failed(new RuntimeException("failed")) + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void distinctStageShouldPropagateUpstreamExceptions() { + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) .distinct() .toList() .run(getEngine())); } + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void distinctStageShouldPropagateExceptionsThrownByEquals() { + CompletableFuture cancelled = new CompletableFuture<>(); + class ObjectThatThrowsFromEquals { + @Override + public int hashCode() { + return 1; + } + + @Override + public boolean equals(Object obj) { + throw new QuietRuntimeException("failed"); + } + } + CompletionStage> result = ReactiveStreams.of( + new ObjectThatThrowsFromEquals(), new ObjectThatThrowsFromEquals() + ) + .onTerminate(() -> cancelled.complete(null)) + .distinct().toList().run(getEngine()); + await(cancelled); + await(result); + } + + @Test + public void distinctStageShouldPropagateCancel() { + CompletableFuture cancelled = new CompletableFuture<>(); + await(infiniteStream().onTerminate(() -> cancelled.complete(null)).distinct().cancel().run(getEngine())); + await(cancelled); + } + + @Test + public void distinctStageBuilderShouldBeReusable() { + ProcessorBuilder distinct = ReactiveStreams.builder().distinct(); + assertEquals(await(ReactiveStreams.of(1, 2, 2, 3).via(distinct).toList().run(getEngine())), Arrays.asList(1, 2, 3)); + assertEquals(await(ReactiveStreams.of(3, 3, 4, 5).via(distinct).toList().run(getEngine())), Arrays.asList(3, 4, 5)); + } + @Override List reactiveStreamsTckVerifiers() { return Collections.singletonList( diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/DropWhileStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/DropWhileStageVerification.java similarity index 53% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/DropWhileStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/DropWhileStageVerification.java index d8bbdd7..0e894a1 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/DropWhileStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/DropWhileStageVerification.java @@ -17,8 +17,9 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; +import org.eclipse.microprofile.reactive.streams.ProcessorBuilder; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Processor; import org.reactivestreams.Publisher; @@ -27,29 +28,59 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; import static org.testng.Assert.assertEquals; +/** + * Verification for the drop while stage. + */ public class DropWhileStageVerification extends AbstractStageVerification { - DropWhileStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + DropWhileStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @Test public void dropWhileStageShouldSupportDroppingElements() { - assertEquals(await(ReactiveStreams.of(1, 2, 3, 4) + assertEquals(await(ReactiveStreams.of(1, 2, 3, 4, 0) .dropWhile(i -> i < 3) .toList() - .run(getEngine())), Arrays.asList(3, 4)); + .run(getEngine())), Arrays.asList(3, 4, 0)); } - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") public void dropWhileStageShouldHandleErrors() { - await(ReactiveStreams.of(1, 2, 3, 4) + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage> result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) .dropWhile(i -> { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); + }) + .toList() + .run(getEngine()); + await(cancelled); + await(result); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void dropWhileStageShouldPropagateUpstreamErrorsWhileDropping() { + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) + .dropWhile(i -> i < 3) + .toList() + .run(getEngine())); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void dropWhileStageShouldPropagateUpstreamErrorsAfterFinishedDropping() { + await(infiniteStream() + .peek(i -> { + if (i == 4) { + throw new QuietRuntimeException("failed"); + } }) + .dropWhile(i -> i < 3) .toList() .run(getEngine())); } @@ -72,6 +103,38 @@ else if (i == 4) { .run(getEngine())), Arrays.asList(3, 4)); } + @Test + public void dropWhileStageShouldAllowCompletionWhileDropping() { + assertEquals(await(ReactiveStreams.of(1, 1, 1, 1) + .dropWhile(i -> i < 3) + .toList() + .run(getEngine())), Collections.emptyList()); + } + + @Test + public void dropWhileStageShouldPropagateCancel() { + CompletableFuture cancelled = new CompletableFuture<>(); + await(infiniteStream().onTerminate(() -> cancelled.complete(null)).dropWhile(i -> i < 3).cancel().run(getEngine())); + await(cancelled); + } + + @Test + public void dropWhileStageBuilderShouldBeReusable() { + ProcessorBuilder dropWhile = ReactiveStreams.builder() + .dropWhile(i -> i < 3); + + assertEquals(await(ReactiveStreams.of(1, 2, 3, 4) + .via(dropWhile) + .toList() + .run(getEngine())), Arrays.asList(3, 4)); + + assertEquals(await(ReactiveStreams.of(0, 1, 6, 7) + .via(dropWhile) + .toList() + .run(getEngine())), Arrays.asList(6, 7)); + + } + @Override List reactiveStreamsTckVerifiers() { return Collections.singletonList( diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/EmptyProcessorVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/EmptyProcessorVerification.java similarity index 86% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/EmptyProcessorVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/EmptyProcessorVerification.java index fe17379..f3f0671 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/EmptyProcessorVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/EmptyProcessorVerification.java @@ -17,7 +17,7 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Processor; @@ -25,9 +25,12 @@ import java.util.Collections; import java.util.List; +/** + * Verification for an empty processor, ie, the processor returned from ReactiveStreams.builder().buildRs(). + */ public class EmptyProcessorVerification extends AbstractStageVerification { - public EmptyProcessorVerification(ReactiveStreamsTck.VerificationDeps deps) { + public EmptyProcessorVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FilterStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FilterStageVerification.java similarity index 57% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FilterStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FilterStageVerification.java index b4ca1b2..24c4050 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FilterStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FilterStageVerification.java @@ -17,8 +17,9 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; +import org.eclipse.microprofile.reactive.streams.ProcessorBuilder; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Processor; import org.reactivestreams.Publisher; @@ -27,12 +28,17 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; import static org.testng.Assert.assertEquals; +/** + * Verification for the filter stage. + */ public class FilterStageVerification extends AbstractStageVerification { - FilterStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + FilterStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @@ -44,16 +50,42 @@ public void filterStageShouldFilterElements() { .run(getEngine())), Arrays.asList(1, 3, 5)); } - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") - public void filterStageShouldPropagateRuntimeExceptions() { - await(ReactiveStreams.of("foo") + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void filterStageShouldPropagateExceptions() { + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage> result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) .filter(foo -> { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); }) .toList() + .run(getEngine()); + await(cancelled); + await(result); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void filterStageShouldPropagateUpstreamExceptions() { + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) + .filter(foo -> true) + .toList() .run(getEngine())); } + @Test + public void filterStageShouldPropagateCancel() { + CompletableFuture cancelled = new CompletableFuture<>(); + await(infiniteStream().onTerminate(() -> cancelled.complete(null)).filter(i -> i < 3).cancel().run(getEngine())); + await(cancelled); + } + + @Test + public void filterStageBuilderShouldBeReusable() { + ProcessorBuilder filter = ReactiveStreams.builder().filter(i -> i < 3); + assertEquals(await(ReactiveStreams.of(1, 2, 3).via(filter).toList().run(getEngine())), Arrays.asList(1, 2)); + assertEquals(await(ReactiveStreams.of(1, 2, 3).via(filter).toList().run(getEngine())), Arrays.asList(1, 2)); + } + @Override List reactiveStreamsTckVerifiers() { return Collections.singletonList( diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FindFirstStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FindFirstStageVerification.java similarity index 65% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FindFirstStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FindFirstStageVerification.java index 67a999a..533b052 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FindFirstStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FindFirstStageVerification.java @@ -17,7 +17,7 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Subscriber; @@ -26,12 +26,16 @@ import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import static org.testng.Assert.assertEquals; +/** + * Verification for find first stage. + */ public class FindFirstStageVerification extends AbstractStageVerification { - FindFirstStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + FindFirstStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @@ -42,17 +46,37 @@ public void findFirstStageShouldFindTheFirstElement() { } @Test - public void findFirstStageShouldReturnEmpty() { + public void findFirstStageShouldFindTheFirstElementInSingleElementStream() { + assertEquals(await(ReactiveStreams.of(1) + .findFirst().run(getEngine())), Optional.of(1)); + } + + @Test + public void findFirstStageShouldReturnEmptyForEmptyStream() { assertEquals(await(ReactiveStreams.of() .findFirst().run(getEngine())), Optional.empty()); } - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") + @Test + public void findFirstStageShouldCancelUpstream() { + CompletableFuture cancelled = new CompletableFuture<>(); + assertEquals(await(infiniteStream().onTerminate(() -> cancelled.complete(null)) + .findFirst().run(getEngine())), Optional.of(1)); + await(cancelled); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") public void findFirstStageShouldPropagateErrors() { - await(ReactiveStreams.failed(new RuntimeException("failed")) + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) .findFirst().run(getEngine())); } + @Test + public void findFirstStageShouldBeReusable() { + assertEquals(await(ReactiveStreams.of(1, 2, 3) + .findFirst().run(getEngine())), Optional.of(1)); + } + @Override List reactiveStreamsTckVerifiers() { return Collections.singletonList(new SubscriberVerification()); diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FlatMapCompletionStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FlatMapCompletionStageVerification.java similarity index 61% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FlatMapCompletionStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FlatMapCompletionStageVerification.java index 1d66133..d30e977 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FlatMapCompletionStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FlatMapCompletionStageVerification.java @@ -17,8 +17,9 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; +import org.eclipse.microprofile.reactive.streams.ProcessorBuilder; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Processor; import org.testng.annotations.Test; @@ -33,8 +34,11 @@ import static org.testng.Assert.assertEquals; +/** + * Verification for flat map completion stage. + */ public class FlatMapCompletionStageVerification extends AbstractStageVerification { - FlatMapCompletionStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + FlatMapCompletionStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @@ -107,6 +111,60 @@ public void flatMapCsStageShouldOnlyMapOneElementAtATime() throws Exception { assertEquals(await(result), Arrays.asList(1, 2, 3)); } + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void flatMapCsStageShouldPropagateUpstreamErrors() { + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) + .flatMapCompletionStage(CompletableFuture::completedFuture) + .toList() + .run(getEngine())); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void flatMapCsStageShouldHandleErrorsThrownByCallback() { + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage> result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) + .flatMapCompletionStage(i -> { + throw new QuietRuntimeException("failed"); + }) + .toList() + .run(getEngine()); + await(cancelled); + await(result); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void flatMapCsStageShouldHandleFailedCompletionStages() { + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage> result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) + .flatMapCompletionStage(i -> { + CompletableFuture failed = new CompletableFuture<>(); + failed.completeExceptionally(new QuietRuntimeException("failed")); + return failed; + }) + .toList() + .run(getEngine()); + await(cancelled); + await(result); + } + + @Test + public void flatMapCsStageShouldPropagateCancel() { + CompletableFuture cancelled = new CompletableFuture<>(); + await(infiniteStream().onTerminate(() -> cancelled.complete(null)) + .flatMapCompletionStage(CompletableFuture::completedFuture).cancel().run(getEngine())); + await(cancelled); + } + + @Test + public void flatMapCsStageBuilderShouldBeResuable() { + ProcessorBuilder mapper = ReactiveStreams.builder() + .flatMapCompletionStage(i -> CompletableFuture.completedFuture(i + 1)); + assertEquals(await(ReactiveStreams.of(1, 2, 3).via(mapper).toList().run(getEngine())), Arrays.asList(2, 3, 4)); + assertEquals(await(ReactiveStreams.of(4, 5, 6).via(mapper).toList().run(getEngine())), Arrays.asList(5, 6, 7)); + } + @Override List reactiveStreamsTckVerifiers() { return Collections.singletonList(new ProcessorVerification()); diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FlatMapIterableStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FlatMapIterableStageVerification.java new file mode 100644 index 0000000..cdd5bad --- /dev/null +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FlatMapIterableStageVerification.java @@ -0,0 +1,173 @@ +/******************************************************************************* + * Copyright (c) 2018 Contributors to the Eclipse Foundation + * + * See the NOTICE file(s) distributed with this work for additional + * information regarding copyright ownership. + * + * 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 org.eclipse.microprofile.reactive.streams.tck.spi; + +import org.eclipse.microprofile.reactive.streams.ProcessorBuilder; +import org.eclipse.microprofile.reactive.streams.ReactiveStreams; +import org.reactivestreams.Processor; +import org.reactivestreams.Publisher; +import org.testng.annotations.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; + +import static org.testng.Assert.assertEquals; + +/** + * Verification for flat map iterable stage. + */ +public class FlatMapIterableStageVerification extends AbstractStageVerification { + FlatMapIterableStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { + super(deps); + } + + @Test + public void flatMapIterableStageShouldMapElements() { + assertEquals(await(ReactiveStreams.of(1, 2, 3) + .flatMapIterable(n -> Arrays.asList(n, n, n)) + .toList() + .run(getEngine())), Arrays.asList(1, 1, 1, 2, 2, 2, 3, 3, 3)); + } + + @Test + public void flatMapIterableStageShouldSupportEmptyMappingOfElements() { + assertEquals(await(ReactiveStreams.of(1, 2, 3) + .flatMapIterable(n -> Collections.emptyList()) + .toList() + .run(getEngine())), Collections.emptyList()); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void flatMapIterableStageShouldHandleExceptionsInCallback() { + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage> result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) + .flatMapIterable(foo -> { + throw new QuietRuntimeException("failed"); + }) + .toList() + .run(getEngine()); + await(cancelled); + await(result); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void flatMapIterableStageShouldHandleExceptionsInIterableIterateMethod() { + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage> result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) + .flatMapIterable(foo -> () -> { + throw new QuietRuntimeException("failed"); + }) + .toList() + .run(getEngine()); + await(cancelled); + await(result); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void flatMapIterableStageShouldHandleExceptionsInIteratorHasNextMethod() { + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage> result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) + .flatMapIterable(foo -> () -> new Iterator() { + @Override + public boolean hasNext() { + throw new QuietRuntimeException("failed"); + } + @Override + public Object next() { + return null; + } + }) + .toList() + .run(getEngine()); + await(cancelled); + await(result); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void flatMapIterableStageShouldHandleExceptionsInIteratorNextMethod() { + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage> result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) + .flatMapIterable(foo -> () -> new Iterator() { + @Override + public boolean hasNext() { + return true; + } + @Override + public Object next() { + throw new QuietRuntimeException("failed"); + } + }) + .toList() + .run(getEngine()); + await(cancelled); + await(result); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void flatMapIterableStageShouldPropagateUpstreamExceptions() { + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) + .flatMapIterable(Collections::singletonList) + .toList() + .run(getEngine())); + } + + @Test + public void flatMapIterableBuilderShouldBeReusable() { + ProcessorBuilder mapper = ReactiveStreams.builder().flatMapIterable(i -> Arrays.asList(i, i)); + assertEquals(await(ReactiveStreams.of(1, 2).via(mapper).toList().run(getEngine())), Arrays.asList(1, 1, 2, 2)); + assertEquals(await(ReactiveStreams.of(1, 2).via(mapper).toList().run(getEngine())), Arrays.asList(1, 1, 2, 2)); + } + + @Override + List reactiveStreamsTckVerifiers() { + return Collections.singletonList(new ProcessorVerification()); + } + + /** + * Verifies the outer processor. + */ + public class ProcessorVerification extends StageProcessorVerification { + + @Override + public Processor createIdentityProcessor(int bufferSize) { + return ReactiveStreams.builder().flatMapIterable(Arrays::asList).buildRs(getEngine()); + } + + @Override + public Publisher createFailedPublisher() { + return ReactiveStreams.failed(new RuntimeException("failed")) + .flatMapIterable(Arrays::asList).buildRs(getEngine()); + } + + @Override + public Integer createElement(int element) { + return element; + } + } + +} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FlatMapStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FlatMapStageVerification.java similarity index 64% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FlatMapStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FlatMapStageVerification.java index 7e24828..647bd25 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/FlatMapStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/FlatMapStageVerification.java @@ -17,8 +17,10 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; +import org.eclipse.microprofile.reactive.streams.ProcessorBuilder; +import org.eclipse.microprofile.reactive.streams.PublisherBuilder; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Processor; import org.reactivestreams.Publisher; @@ -37,7 +39,7 @@ import static org.testng.Assert.assertEquals; public class FlatMapStageVerification extends AbstractStageVerification { - FlatMapStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + FlatMapStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @@ -49,16 +51,48 @@ public void flatMapStageShouldMapElements() { .run(getEngine())), Arrays.asList(1, 1, 1, 2, 2, 2, 3, 3, 3)); } - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") - public void flatMapStageShouldPropagateRuntimeExceptions() { - await(ReactiveStreams.of("foo") + @Test + public void flatMapStageShouldAllowEmptySubStreams() { + assertEquals(await(ReactiveStreams.of(ReactiveStreams.empty(), ReactiveStreams.of(1, 2)) + .flatMap(Function.identity()) + .toList() + .run(getEngine())), Arrays.asList(1, 2)); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void flatMapStageShouldHandleExceptions() { + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage> result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) .flatMap(foo -> { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); }) .toList() + .run(getEngine()); + await(cancelled); + await(result); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void flatMapStageShouldPropagateUpstreamExceptions() { + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) + .flatMap(ReactiveStreams::of) + .toList() .run(getEngine())); } + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void flatMapStageShouldPropagateSubstreamExceptions() { + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage> result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) + .flatMap(f -> ReactiveStreams.failed(new QuietRuntimeException("failed"))) + .toList() + .run(getEngine()); + await(cancelled); + await(result); + } + @Test public void flatMapStageShouldOnlySubscribeToOnePublisherAtATime() throws Exception { AtomicInteger activePublishers = new AtomicInteger(); @@ -72,6 +106,29 @@ public void flatMapStageShouldOnlySubscribeToOnePublisherAtATime() throws Except Arrays.asList(1, 2, 3, 4, 5)); } + @Test + public void flatMapStageShouldPropgateCancelToSubstreams() { + CompletableFuture outerCancelled = new CompletableFuture<>(); + CompletableFuture innerCancelled = new CompletableFuture<>(); + await(infiniteStream() + .onTerminate(() -> outerCancelled.complete(null)) + .flatMap(i -> infiniteStream().onTerminate(() -> innerCancelled.complete(null))) + .limit(5) + .toList() + .run(getEngine())); + + await(outerCancelled); + await(innerCancelled); + } + + @Test + public void flatMapStageBuilderShouldBeReusable() { + ProcessorBuilder, Integer> flatMap = + ReactiveStreams.>builder().flatMap(Function.identity()); + + assertEquals(await(ReactiveStreams.of(ReactiveStreams.of(1, 2)).via(flatMap).toList().run(getEngine())), Arrays.asList(1, 2)); + assertEquals(await(ReactiveStreams.of(ReactiveStreams.of(3, 4)).via(flatMap).toList().run(getEngine())), Arrays.asList(3, 4)); + } @Override List reactiveStreamsTckVerifiers() { diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/LimitStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/LimitStageVerification.java similarity index 70% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/LimitStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/LimitStageVerification.java index eb3134a..62250f9 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/LimitStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/LimitStageVerification.java @@ -17,8 +17,9 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; +import org.eclipse.microprofile.reactive.streams.ProcessorBuilder; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Processor; import org.reactivestreams.Publisher; @@ -34,13 +35,13 @@ public class LimitStageVerification extends AbstractStageVerification { - LimitStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + LimitStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @Test public void limitStageShouldLimitTheOutputElements() { - assertEquals(await(ReactiveStreams.of(1, 2, 3, 4, 5) + assertEquals(await(infiniteStream() .limit(3) .toList() .run(getEngine())), Arrays.asList(1, 2, 3)); @@ -48,7 +49,7 @@ public void limitStageShouldLimitTheOutputElements() { @Test public void limitStageShouldAllowLimitingToZero() { - assertEquals(await(ReactiveStreams.of(1, 2, 3, 4, 5, 6) + assertEquals(await(infiniteStream() .limit(0) .toList() .run(getEngine())), Collections.emptyList()); @@ -72,30 +73,20 @@ public void cancel() { } @Test - public void limitShouldCancelUpStreamWhenDone() { + public void limitStageShouldCancelUpStreamWhenDone() { CompletableFuture cancelled = new CompletableFuture<>(); - ReactiveStreams.fromPublisher(subscriber -> - subscriber.onSubscribe(new Subscription() { - @Override - public void request(long n) { - subscriber.onNext(1); - } - - @Override - public void cancel() { - cancelled.complete(null); - } - }) - ).limit(1) + infiniteStream() + .onTerminate(() -> cancelled.complete(null)) + .limit(1) .toList() .run(getEngine()); await(cancelled); } @Test - public void limitShouldIgnoreSubsequentErrorsWhenDone() { + public void limitStageShouldIgnoreSubsequentErrorsWhenDone() { assertEquals(await( - ReactiveStreams.of(1, 2, 3, 4) + infiniteStream() .flatMap(i -> { if (i == 4) { return ReactiveStreams.failed(new RuntimeException("failed")); @@ -110,6 +101,32 @@ public void limitShouldIgnoreSubsequentErrorsWhenDone() { ), Arrays.asList(1, 2, 3)); } + @Test + public void limitStageShouldPropagateCancellation() { + CompletableFuture cancelled = new CompletableFuture<>(); + await( + infiniteStream() + .onTerminate(() -> cancelled.complete(null)) + .peek(i -> { + if (i == 100) { + cancelled.completeExceptionally(new RuntimeException("Was not cancelled")); + } + }) + .limit(100) + .limit(3) + .toList() + .run(getEngine()) + ); + await(cancelled); + } + + @Test + public void limitStageBuilderShouldBeReusable() { + ProcessorBuilder limit = ReactiveStreams.builder().limit(3); + assertEquals(await(infiniteStream().via(limit).toList().run(getEngine())), Arrays.asList(1, 2, 3)); + assertEquals(await(infiniteStream().map(i -> i + 1).via(limit).toList().run(getEngine())), Arrays.asList(2, 3, 4)); + } + @Override List reactiveStreamsTckVerifiers() { return Collections.singletonList(new ProcessorVerification()); diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/MapStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/MapStageVerification.java similarity index 62% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/MapStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/MapStageVerification.java index 911ca41..1137581 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/MapStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/MapStageVerification.java @@ -17,8 +17,9 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; +import org.eclipse.microprofile.reactive.streams.ProcessorBuilder; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Processor; import org.reactivestreams.Publisher; @@ -27,13 +28,15 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; import java.util.function.Function; import static org.testng.Assert.assertEquals; public class MapStageVerification extends AbstractStageVerification { - MapStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + MapStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @@ -45,16 +48,35 @@ public void mapStageShouldMapElements() { .run(getEngine())), Arrays.asList("1", "2", "3")); } - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") - public void mapStageShouldPropagateRuntimeExceptions() { - await(ReactiveStreams.of("foo") + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void mapStageShouldHandleExceptions() { + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage> result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) .map(foo -> { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); }) .toList() + .run(getEngine()); + await(cancelled); + await(result); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void mapStageShouldPropagateUpstreamExceptions() { + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) + .map(Function.identity()) + .toList() .run(getEngine())); } + @Test + public void mapStageBuilderShouldBeReusable() { + ProcessorBuilder map = ReactiveStreams.builder().map(i -> i + 1); + assertEquals(await(ReactiveStreams.of(1, 2, 3).via(map).toList().run(getEngine())), Arrays.asList(2, 3, 4)); + assertEquals(await(ReactiveStreams.of(4, 5, 6).via(map).toList().run(getEngine())), Arrays.asList(5, 6, 7)); + } + @Override List reactiveStreamsTckVerifiers() { return Collections.singletonList( diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/OfStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/OfStageVerification.java similarity index 51% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/OfStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/OfStageVerification.java index 2ad51d6..1701366 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/OfStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/OfStageVerification.java @@ -17,22 +17,25 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; +import org.eclipse.microprofile.reactive.streams.PublisherBuilder; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Publisher; import org.testng.annotations.Test; import java.util.Arrays; import java.util.Collections; +import java.util.Iterator; import java.util.List; +import java.util.concurrent.CompletionStage; import java.util.stream.LongStream; import static org.testng.Assert.assertEquals; public class OfStageVerification extends AbstractStageVerification { - OfStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + OfStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @@ -63,6 +66,64 @@ public void singleIterableStageShouldEmitOneElement() { ), Collections.singletonList("a")); } + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void ofStageShouldHandleExceptionsInIterableIterateMethod() { + CompletionStage> result; + try { + result = ReactiveStreams.fromIterable(() -> { + throw new QuietRuntimeException("failed"); + }) + .toList() + .run(getEngine()); + } + catch (QuietRuntimeException e) { + throw new AssertionError("Exception was thrown directly, should have been part of the redeemed completion stage", e); + } + await(result); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void ofStageShouldHandleExceptionsInIteratorHasNextMethod() { + await(ReactiveStreams.fromIterable(() -> new Iterator() { + @Override + public boolean hasNext() { + throw new QuietRuntimeException("failed"); + } + + @Override + public Object next() { + return null; + } + }) + .toList() + .run(getEngine())); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void ofStageShouldHandleExceptionsInIteratorNextMethod() { + await(ReactiveStreams.fromIterable(() -> new Iterator() { + @Override + public boolean hasNext() { + return true; + } + + @Override + public Object next() { + throw new QuietRuntimeException("failed"); + } + }) + .toList() + .run(getEngine())); + } + + @Test + public void ofStageShouldBeReusable() { + PublisherBuilder publisher = ReactiveStreams.of(1, 2, 3); + + assertEquals(await(publisher.toList().run(getEngine())), Arrays.asList(1, 2, 3)); + assertEquals(await(publisher.toList().run(getEngine())), Arrays.asList(1, 2, 3)); + } + @Override List reactiveStreamsTckVerifiers() { return Collections.singletonList(new PublisherVerification()); diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/OnErrorResumeStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/OnErrorResumeStageVerification.java similarity index 88% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/OnErrorResumeStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/OnErrorResumeStageVerification.java index e0bae97..a3e93f6 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/OnErrorResumeStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/OnErrorResumeStageVerification.java @@ -17,7 +17,7 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Processor; @@ -40,14 +40,14 @@ */ public class OnErrorResumeStageVerification extends AbstractStageVerification { - OnErrorResumeStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + OnErrorResumeStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @Test public void onErrorResumeShouldCatchErrorFromSource() { AtomicReference exception = new AtomicReference<>(); - assertEquals(await(ReactiveStreams.failed(new Exception("failed")) + assertEquals(await(ReactiveStreams.failed(new QuietRuntimeException("failed")) .onErrorResume(err -> { exception.set(err); return "foo"; @@ -60,7 +60,7 @@ public void onErrorResumeShouldCatchErrorFromSource() { @Test public void onErrorResumeWithShouldCatchErrorFromSource() { AtomicReference exception = new AtomicReference<>(); - assertEquals(await(ReactiveStreams.failed(new Exception("failed")) + assertEquals(await(ReactiveStreams.failed(new QuietRuntimeException("failed")) .onErrorResumeWith(err -> { exception.set(err); return ReactiveStreams.of("foo", "bar"); @@ -73,7 +73,7 @@ public void onErrorResumeWithShouldCatchErrorFromSource() { @Test public void onErrorResumeWithPublisherShouldCatchErrorFromSource() { AtomicReference exception = new AtomicReference<>(); - assertEquals(await(ReactiveStreams.failed(new Exception("failed")) + assertEquals(await(ReactiveStreams.failed(new QuietRuntimeException("failed")) .onErrorResumeWithPublisher(err -> { exception.set(err); return ReactiveStreams.of("foo", "bar").buildRs(); @@ -89,7 +89,7 @@ public void onErrorResumeShouldCatchErrorFromStage() { assertEquals(await(ReactiveStreams.of("a", "b", "c") .map(word -> { if (word.equals("b")) { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); } return word.toUpperCase(); }) @@ -108,7 +108,7 @@ public void onErrorResumeWithShouldCatchErrorFromStage() { assertEquals(await(ReactiveStreams.of("a", "b", "c") .map(word -> { if (word.equals("b")) { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); } return word.toUpperCase(); }) @@ -127,7 +127,7 @@ public void onErrorResumeWithPublisherShouldCatchErrorFromStage() { assertEquals(await(ReactiveStreams.of("a", "b", "c") .map(word -> { if (word.equals("b")) { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); } return word.toUpperCase(); }) @@ -145,7 +145,7 @@ public void onErrorResumeWithPublisherShouldCatchErrorFromStage() { public void onErrorResumeStageShouldPropagateRuntimeExceptions() { await(ReactiveStreams.failed(new Exception("source-failure")) .onErrorResume(t -> { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); }) .toList() .run(getEngine())); @@ -155,7 +155,7 @@ public void onErrorResumeStageShouldPropagateRuntimeExceptions() { public void onErrorResumeWithStageShouldPropagateRuntimeExceptions() { await(ReactiveStreams.failed(new Exception("source-failure")) .onErrorResumeWith(t -> { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); }) .toList() .run(getEngine())); @@ -163,26 +163,26 @@ public void onErrorResumeWithStageShouldPropagateRuntimeExceptions() { @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = ".*failed.*") public void onErrorResumeWithPublisherStageShouldPropagateRuntimeExceptions() { - await(ReactiveStreams.failed(new Exception("source-failure")) + await(ReactiveStreams.failed(new QuietRuntimeException("source-failure")) .onErrorResumeWithPublisher(t -> { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); }) .toList() .run(getEngine())); } - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = ".*boom.*") + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = ".*boom.*") public void onErrorResumeWithShouldBeAbleToInjectAFailure() { - await(ReactiveStreams.failed(new Exception("failed")) - .onErrorResumeWith(err -> ReactiveStreams.failed(new Exception("boom"))) + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) + .onErrorResumeWith(err -> ReactiveStreams.failed(new QuietRuntimeException("boom"))) .toList() .run(getEngine())); } @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = ".*boom.*") public void onErrorResumeWithPublisherShouldBeAbleToInjectAFailure() { - await(ReactiveStreams.failed(new Exception("failed")) - .onErrorResumeWithPublisher(err -> ReactiveStreams.failed(new Exception("boom")).buildRs()) + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) + .onErrorResumeWithPublisher(err -> ReactiveStreams.failed(new QuietRuntimeException("boom")).buildRs()) .toList() .run(getEngine())); } diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/OnStagesVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/OnStagesVerification.java similarity index 88% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/OnStagesVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/OnStagesVerification.java index 74f2369..43c260f 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/OnStagesVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/OnStagesVerification.java @@ -17,7 +17,7 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Processor; @@ -44,7 +44,7 @@ public class OnStagesVerification extends AbstractStageVerification { private Runnable noop = () -> { }; - OnStagesVerification(ReactiveStreamsTck.VerificationDeps deps) { + OnStagesVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @@ -78,11 +78,11 @@ public void onErrorStageShouldNotBeCalledWhenTheStreamComplete() { assertFalse(called.get()); } - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") public void onCompleteStageShouldPropagateRuntimeExceptions() { await(ReactiveStreams.of("foo") .onComplete(() -> { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); }) .toList() .run(getEngine())); @@ -91,7 +91,7 @@ public void onCompleteStageShouldPropagateRuntimeExceptions() { @Test public void onCompleteStageShouldNotBeCalledWhenTheStreamFailed() { AtomicBoolean called = new AtomicBoolean(); - await(ReactiveStreams.failed(new Exception("failed")) + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) .onComplete(() -> called.set(true)) .toList() .run(getEngine()) @@ -103,7 +103,7 @@ public void onCompleteStageShouldNotBeCalledWhenTheStreamFailed() { @Test public void onErrorStageShouldBeCalledWhenTheStreamFailed() { AtomicReference called = new AtomicReference<>(); - await(ReactiveStreams.failed(new Exception("failed")) + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) .onError(called::set) .toList() .run(getEngine()) @@ -118,7 +118,7 @@ public void onErrorStageShouldBeCalledWhenTheStreamFailedBecauseOfAnIntermediate AtomicReference called = new AtomicReference<>(); await(ReactiveStreams.of(1, 2, 3) .map(x -> { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); }) .onError(called::set) .toList() @@ -132,7 +132,7 @@ public void onErrorStageShouldBeCalledWhenTheStreamFailedBecauseOfAnIntermediate @Test public void onTerminateStageShouldBeCalledWhenTheStreamFailed() { AtomicBoolean called = new AtomicBoolean(); - await(ReactiveStreams.failed(new Exception("failed")) + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) .onTerminate(() -> called.set(true)) .toList() .run(getEngine()) @@ -146,7 +146,7 @@ public void onTerminateStageShouldBeCalledWhenTheStreamFailedBecauseOfAnIntermed AtomicBoolean called = new AtomicBoolean(); await(ReactiveStreams.of(1, 2, 3) .map(x -> { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); }) .onTerminate(() -> called.set(true)) .toList() @@ -156,11 +156,11 @@ public void onTerminateStageShouldBeCalledWhenTheStreamFailedBecauseOfAnIntermed assertTrue(called.get()); } - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") public void onTerminateStageShouldPropagateRuntimeExceptions() { await(ReactiveStreams.of("foo") .onTerminate(() -> { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); }) .toList() .run(getEngine())); @@ -186,7 +186,7 @@ public void onErrorAndOnTerminateStagesShouldBeCalledWhenTheStreamFails() { AtomicBoolean onTerminateCalled = new AtomicBoolean(); AtomicReference onErrorCalled = new AtomicReference<>(); - await(ReactiveStreams.failed(new Exception("failed")) + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) .onError(onErrorCalled::set) .onTerminate(() -> onTerminateCalled.set(true)) .toList() @@ -226,7 +226,7 @@ public Processor createIdentityProcessor(int bufferSize) { @Override public Publisher createFailedPublisher() { - return ReactiveStreams.failed(new RuntimeException("failed")) + return ReactiveStreams.failed(new QuietRuntimeException("failed")) .onComplete(noop).buildRs(getEngine()); } @@ -245,7 +245,7 @@ public Processor createIdentityProcessor(int bufferSize) { @Override public Publisher createFailedPublisher() { - return ReactiveStreams.failed(new RuntimeException("failed")) + return ReactiveStreams.failed(new QuietRuntimeException("failed")) .onTerminate(noop).buildRs(getEngine()); } @@ -265,7 +265,7 @@ public Processor createIdentityProcessor(int bufferSize) { @Override public Publisher createFailedPublisher() { - return ReactiveStreams.failed(new RuntimeException("failed")) + return ReactiveStreams.failed(new QuietRuntimeException("failed")) .onError(x -> { }).buildRs(getEngine()); } diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/PeekStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/PeekStageVerification.java similarity index 58% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/PeekStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/PeekStageVerification.java index 7e23553..cc448d6 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/PeekStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/PeekStageVerification.java @@ -17,8 +17,9 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; +import org.eclipse.microprofile.reactive.streams.ProcessorBuilder; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Processor; import org.reactivestreams.Publisher; @@ -27,14 +28,18 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; public class PeekStageVerification extends AbstractStageVerification { - PeekStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + PeekStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @@ -48,16 +53,45 @@ public void peekStageShouldNotModifyElements() { assertEquals(count.get(), 3); } - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") - public void peekStageShouldPropagateRuntimeExceptions() { - await(ReactiveStreams.of("foo") + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void peekStageShouldHandleExceptions() { + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage> result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) .peek(x -> { - throw new RuntimeException("failed"); + throw new QuietRuntimeException("failed"); }) .toList() + .run(getEngine()); + await(cancelled); + await(result); + } + + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") + public void peekStageShouldPropagateUpstreamExceptions() { + await(ReactiveStreams.failed(new QuietRuntimeException("failed")) + .peek(x -> {}) + .toList() .run(getEngine())); } + public void peekStageShouldNotBeExecutedForEmptyStreams() { + AtomicBoolean invoked = new AtomicBoolean(); + await(ReactiveStreams.empty() + .peek(x -> invoked.set(true)) + .toList() + .run(getEngine())); + assertFalse(invoked.get()); + } + + @Test + public void peekStageShouldBeReusable() { + ProcessorBuilder peek = ReactiveStreams.builder().peek(t -> {}); + + assertEquals(await(ReactiveStreams.of(1, 2, 3).via(peek).toList().run(getEngine())), Arrays.asList(1, 2, 3)); + assertEquals(await(ReactiveStreams.of(4, 5, 6).via(peek).toList().run(getEngine())), Arrays.asList(4, 5, 6)); + } + @Override List reactiveStreamsTckVerifiers() { return Collections.singletonList( diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/QuietRuntimeException.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/QuietRuntimeException.java new file mode 100644 index 0000000..fdcf6be --- /dev/null +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/QuietRuntimeException.java @@ -0,0 +1,41 @@ +/******************************************************************************* + * Copyright (c) 2018 Contributors to the Eclipse Foundation + * + * See the NOTICE file(s) distributed with this work for additional + * information regarding copyright ownership. + * + * 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 org.eclipse.microprofile.reactive.streams.tck.spi; + +/** + * RuntimeException with no stack trace for expected failures, to make logging not so noisy. + */ +public class QuietRuntimeException extends RuntimeException { + public QuietRuntimeException() { + this(null, null); + } + + public QuietRuntimeException(String message) { + this(message, null); + } + + public QuietRuntimeException(String message, Throwable cause) { + super(message, cause, true, false); + } + + public QuietRuntimeException(Throwable cause) { + this(null, cause); + } +} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/ReactiveStreamsSpiVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/ReactiveStreamsSpiVerification.java new file mode 100644 index 0000000..fa0c4cf --- /dev/null +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/ReactiveStreamsSpiVerification.java @@ -0,0 +1,94 @@ +/******************************************************************************* + * Copyright (c) 2018 Contributors to the Eclipse Foundation + * + * See the NOTICE file(s) distributed with this work for additional + * information regarding copyright ownership. + * + * 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 org.eclipse.microprofile.reactive.streams.tck.spi; + +import org.eclipse.microprofile.reactive.streams.spi.ReactiveStreamsEngine; +import org.reactivestreams.tck.TestEnvironment; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ScheduledExecutorService; +import java.util.function.Function; + +/** + * This test is a factory for all the tests for verifying implementations of the SPI. + */ +public class ReactiveStreamsSpiVerification { + + private final TestEnvironment testEnvironment; + private final ReactiveStreamsEngine engine; + private final ScheduledExecutorService executorService; + + public ReactiveStreamsSpiVerification(TestEnvironment testEnvironment, ReactiveStreamsEngine engine, ScheduledExecutorService executorService) { + this.testEnvironment = testEnvironment; + this.engine = engine; + this.executorService = executorService; + } + + class VerificationDeps { + ReactiveStreamsEngine engine() { + return engine; + } + + TestEnvironment testEnvironment() { + return testEnvironment; + } + + ScheduledExecutorService executorService() { + return executorService; + } + } + + public List allTests() { + List> stageVerifications = Arrays.asList( + OfStageVerification::new, + MapStageVerification::new, + FlatMapStageVerification::new, + FilterStageVerification::new, + FindFirstStageVerification::new, + CollectStageVerification::new, + TakeWhileStageVerification::new, + FlatMapCompletionStageVerification::new, + FlatMapIterableStageVerification::new, + ConcatStageVerification::new, + EmptyProcessorVerification::new, + CancelStageVerification::new, + SubscriberStageVerification::new, + PeekStageVerification::new, + DistinctStageVerification::new, + OnStagesVerification::new, + LimitStageVerification::new, + SkipStageVerification::new, + DropWhileStageVerification::new, + OnErrorResumeStageVerification::new + ); + + List allTests = new ArrayList<>(); + VerificationDeps deps = new VerificationDeps(); + for (Function creator : stageVerifications) { + AbstractStageVerification stageVerification = creator.apply(deps); + allTests.add(stageVerification); + allTests.addAll(stageVerification.reactiveStreamsTckVerifiers()); + } + + return allTests; + } +} diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/ScheduledPublisher.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/ScheduledPublisher.java similarity index 98% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/ScheduledPublisher.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/ScheduledPublisher.java index b48f06b..5ab51ac 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/ScheduledPublisher.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/ScheduledPublisher.java @@ -17,7 +17,7 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; import org.reactivestreams.Publisher; diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/SkipStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/SkipStageVerification.java similarity index 73% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/SkipStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/SkipStageVerification.java index 2005318..6250187 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/SkipStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/SkipStageVerification.java @@ -17,8 +17,9 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; +import org.eclipse.microprofile.reactive.streams.ProcessorBuilder; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Processor; import org.reactivestreams.Publisher; @@ -32,7 +33,7 @@ public class SkipStageVerification extends AbstractStageVerification { - SkipStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + SkipStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @@ -44,16 +45,20 @@ public void skipStageShouldSkipElements() { .run(getEngine())), Arrays.asList(3, 4)); } - @Test(expectedExceptions = IllegalArgumentException.class) - public void skipOnPublisherShouldRefuseToSkipNegativeElements() { - ReactiveStreams.of(1) - .skip(-1); + @Test + public void skipStageShouldSupportSkippingNoElements() { + assertEquals(await(ReactiveStreams.of(1, 2, 3, 4) + .skip(0) + .toList() + .run(getEngine())), Arrays.asList(1, 2, 3, 4)); } - @Test(expectedExceptions = IllegalArgumentException.class) - public void skipOnProcessorShouldRefuseToSkipNegativeElements() { - ReactiveStreams.builder() - .skip(-1); + @Test + public void skipStageShouldBeReusable() { + ProcessorBuilder skip = ReactiveStreams.builder().skip(2); + + assertEquals(await(ReactiveStreams.of(1, 2, 3, 4).via(skip).toList().run(getEngine())), Arrays.asList(3, 4)); + assertEquals(await(ReactiveStreams.of(5, 6, 7, 8).via(skip).toList().run(getEngine())), Arrays.asList(7, 8)); } @Override diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/SubscriberStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/SubscriberStageVerification.java similarity index 94% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/SubscriberStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/SubscriberStageVerification.java index c4800db..313f14f 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/SubscriberStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/SubscriberStageVerification.java @@ -17,7 +17,7 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Subscription; @@ -29,7 +29,7 @@ import java.util.concurrent.CompletionStage; public class SubscriberStageVerification extends AbstractStageVerification { - SubscriberStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + SubscriberStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } diff --git a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/TakeWhileStageVerification.java b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/TakeWhileStageVerification.java similarity index 72% rename from streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/TakeWhileStageVerification.java rename to streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/TakeWhileStageVerification.java index 8c6f9cb..a45a730 100644 --- a/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/TakeWhileStageVerification.java +++ b/streams/tck/src/main/java/org/eclipse/microprofile/reactive/streams/tck/spi/TakeWhileStageVerification.java @@ -17,24 +17,25 @@ * limitations under the License. ******************************************************************************/ -package org.eclipse.microprofile.reactive.streams.tck; +package org.eclipse.microprofile.reactive.streams.tck.spi; +import org.eclipse.microprofile.reactive.streams.ProcessorBuilder; import org.eclipse.microprofile.reactive.streams.ReactiveStreams; import org.reactivestreams.Processor; import org.reactivestreams.Publisher; -import org.reactivestreams.Subscription; import org.testng.annotations.Test; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; import static org.testng.Assert.assertEquals; public class TakeWhileStageVerification extends AbstractStageVerification { - TakeWhileStageVerification(ReactiveStreamsTck.VerificationDeps deps) { + TakeWhileStageVerification(ReactiveStreamsSpiVerification.VerificationDeps deps) { super(deps); } @@ -57,19 +58,9 @@ public void takeWhileStageShouldEmitEmpty() { @Test public void takeWhileShouldCancelUpStreamWhenDone() { CompletableFuture cancelled = new CompletableFuture<>(); - ReactiveStreams.fromPublisher(subscriber -> - subscriber.onSubscribe(new Subscription() { - @Override - public void request(long n) { - subscriber.onNext(1); - } - - @Override - public void cancel() { - cancelled.complete(null); - } - }) - ).takeWhile(t -> false) + infiniteStream() + .onTerminate(() -> cancelled.complete(null)) + .takeWhile(t -> false) .toList() .run(getEngine()); await(cancelled); @@ -81,7 +72,7 @@ public void takeWhileShouldIgnoreSubsequentErrorsWhenDone() { ReactiveStreams.of(1, 2, 3, 4) .flatMap(i -> { if (i == 4) { - return ReactiveStreams.failed(new RuntimeException("failed")); + return ReactiveStreams.failed(new QuietRuntimeException("failed")); } else { return ReactiveStreams.of(i); @@ -93,14 +84,25 @@ public void takeWhileShouldIgnoreSubsequentErrorsWhenDone() { ), Arrays.asList(1, 2)); } - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = "failed") + @Test(expectedExceptions = QuietRuntimeException.class, expectedExceptionsMessageRegExp = "failed") public void takeWhileStageShouldHandleErrors() { - await(ReactiveStreams.of(1, 2, 3, 4) - .dropWhile(i -> { - throw new RuntimeException("failed"); + CompletableFuture cancelled = new CompletableFuture<>(); + CompletionStage> result = infiniteStream() + .onTerminate(() -> cancelled.complete(null)) + .takeWhile(i -> { + throw new QuietRuntimeException("failed"); }) .toList() - .run(getEngine())); + .run(getEngine()); + await(cancelled); + await(result); + } + + @Test + public void takeWhileStageShouldBeReusable() { + ProcessorBuilder takeWhile = ReactiveStreams.builder().takeWhile(i -> i < 3); + assertEquals(await(ReactiveStreams.of(1, 2, 3, 4).via(takeWhile).toList().run(getEngine())), Arrays.asList(1, 2)); + assertEquals(await(ReactiveStreams.of(2, 1, 5, 6).via(takeWhile).toList().run(getEngine())), Arrays.asList(2, 1)); } @Override diff --git a/streams/tck/src/test/java/org/eclipse/microprofile/reactive/streams/tck/TckTest.java b/streams/tck/src/test/java/org/eclipse/microprofile/reactive/streams/tck/TckTest.java index f5c2fb5..34f300a 100644 --- a/streams/tck/src/test/java/org/eclipse/microprofile/reactive/streams/tck/TckTest.java +++ b/streams/tck/src/test/java/org/eclipse/microprofile/reactive/streams/tck/TckTest.java @@ -19,17 +19,16 @@ package org.eclipse.microprofile.reactive.streams.tck; +import org.eclipse.microprofile.reactive.streams.tck.api.ReactiveStreamsApiVerification; import org.testng.annotations.Factory; /** - * This runs any tests that don't require an implementation to run. + * This runs any tests that don't require an implementation to run, that is, the API verification. */ public class TckTest { @Factory - public Object[] independentTests() { - return new Object[]{ - new GraphAccessorVerification() - }; + public Object[] allTests() { + return new ReactiveStreamsApiVerification().allTests().toArray(); } }