From d6fbadc61e32c31dbf5ce5435431f23558002d73 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Tue, 22 Jul 2014 12:21:53 +0200 Subject: [PATCH] !str: #15474: Migrate to reactive-streams 0.4.0.M1 --- .../java/akka/http/model/japi/HttpEntity.java | 14 +- .../http/model/japi/HttpEntityChunked.java | 4 +- .../model/japi/HttpEntityCloseDelimited.java | 4 +- .../http/model/japi/HttpEntityDefault.java | 4 +- .../http/model/japi/IncomingConnection.java | 10 +- .../akka/http/model/japi/ServerBinding.java | 6 +- .../main/java/akka/http/model/japi/Util.java | 16 +- .../src/main/scala/akka/http/Http.scala | 14 +- .../main/scala/akka/http/HttpManager.scala | 2 +- .../akka/http/client/HttpClientPipeline.scala | 14 +- .../http/client/HttpClientProcessor.scala | 12 +- .../scala/akka/http/model/HttpEntity.scala | 42 +-- .../akka/http/model/MultipartContent.scala | 18 +- .../akka/http/parsing/HttpMessageParser.scala | 10 +- .../akka/http/parsing/HttpRequestParser.scala | 4 +- .../http/parsing/HttpResponseParser.scala | 6 +- .../akka/http/parsing/ParserOutput.scala | 6 +- .../HttpRequestRendererFactory.scala | 18 +- .../HttpResponseRendererFactory.scala | 20 +- .../akka/http/rendering/RenderSupport.scala | 12 +- .../akka/http/server/HttpServerPipeline.scala | 18 +- .../main/scala/akka/http/util/Rendering.scala | 5 +- .../main/scala/akka/http/util/package.scala | 8 +- .../akka/http/model/japi/JavaTestServer.java | 4 +- .../scala/akka/http/ClientServerSpec.scala | 53 ++-- .../src/test/scala/akka/http/TestServer.scala | 4 +- .../akka/http/model/HttpEntitySpec.scala | 24 +- .../akka/http/parsing/RequestParserSpec.scala | 24 +- .../http/parsing/ResponseParserSpec.scala | 22 +- .../http/rendering/RequestRendererSpec.scala | 12 +- .../http/rendering/ResponseRendererSpec.scala | 30 +-- .../stream/PersistentPublisher.scala | 13 +- .../scala/akka/stream/FlattenStrategy.scala | 6 +- .../scala/akka/stream/FlowMaterializer.scala | 11 +- .../scala/akka/stream/OverflowStrategy.scala | 2 +- .../src/main/scala/akka/stream/Support.scala | 2 +- .../main/scala/akka/stream/Transformer.scala | 2 +- ...torProducer.scala => ActorPublisher.scala} | 89 +++---- ...orConsumer.scala => ActorSubscriber.scala} | 83 +++--- .../impl/ActorBasedFlowMaterializer.scala | 93 +++---- .../akka/stream/impl/ActorProcessor.scala | 28 +- ...torProducer.scala => ActorPublisher.scala} | 94 +++---- ...nsumer.scala => BlackholeSubscriber.scala} | 10 +- .../stream/impl/CompletedPublishers.scala | 20 ++ .../akka/stream/impl/ConcatAllImpl.scala | 6 +- .../akka/stream/impl/EmptyProducer.scala | 37 --- .../scala/akka/stream/impl/FlowImpl.scala | 40 ++- ...reProducer.scala => FuturePublisher.scala} | 33 +-- ...Producer.scala => IterablePublisher.scala} | 49 ++-- ...Producer.scala => IteratorPublisher.scala} | 4 +- .../scala/akka/stream/impl/Messages.scala | 2 +- .../akka/stream/impl/PrefixAndTailImpl.scala | 2 +- .../stream/impl/SplitWhenProcessorImpl.scala | 5 +- .../scala/akka/stream/impl/StaticFanins.scala | 8 +- .../akka/stream/impl/StaticFanouts.scala | 9 +- .../impl/StreamOfStreamProcessors.scala | 17 +- .../stream/impl/SubscriberManagement.scala | 19 +- ...=> SynchronousPublisherFromIterable.scala} | 39 ++- ...TickProducer.scala => TickPublisher.scala} | 41 ++- .../scala/akka/stream/impl/Transfer.scala | 2 - .../main/scala/akka/stream/io/StreamIO.scala | 20 +- .../akka/stream/io/TcpConnectionStream.scala | 2 +- .../akka/stream/io/TcpListenStreamActor.scala | 24 +- .../main/scala/akka/stream/javadsl/Duct.scala | 125 +++++---- .../main/scala/akka/stream/javadsl/Flow.scala | 113 ++++---- .../scala/akka/stream/scaladsl/Duct.scala | 101 ++++--- .../scala/akka/stream/scaladsl/Flow.scala | 107 ++++---- .../java/akka/stream/javadsl/DuctTest.java | 40 +-- .../java/akka/stream/javadsl/FlowTest.java | 28 +- .../stream/PersistentPublisherSpec.scala | 30 +-- .../scala/akka/stream/ActorProducerTest.scala | 47 ---- .../akka/stream/ActorPublisherTest.scala | 14 + .../src/test/scala/akka/stream/DuctSpec.scala | 101 ++++--- .../scala/akka/stream/FlowBufferSpec.scala | 124 ++++----- .../scala/akka/stream/FlowConcatAllSpec.scala | 78 +++--- .../scala/akka/stream/FlowConcatSpec.scala | 124 ++++----- .../scala/akka/stream/FlowConflateSpec.scala | 72 ++--- .../akka/stream/FlowDropWithinSpec.scala | 12 +- .../scala/akka/stream/FlowExpandSpec.scala | 80 +++--- .../scala/akka/stream/FlowFilterSpec.scala | 6 +- .../akka/stream/FlowFromFutureSpec.scala | 88 +++---- .../scala/akka/stream/FlowGroupBySpec.scala | 176 ++++++------- .../akka/stream/FlowGroupedWithinSpec.scala | 49 ++-- .../scala/akka/stream/FlowIterableSpec.scala | 106 ++++---- .../scala/akka/stream/FlowIteratorSpec.scala | 84 +++--- .../scala/akka/stream/FlowMapFutureSpec.scala | 26 +- .../test/scala/akka/stream/FlowMapSpec.scala | 6 +- .../scala/akka/stream/FlowMergeSpec.scala | 91 +++---- .../akka/stream/FlowOnCompleteSpec.scala | 16 +- .../akka/stream/FlowPrefixAndTailSpec.scala | 97 +++---- ...cala => FlowProduceToSubscriberSpec.scala} | 10 +- .../src/test/scala/akka/stream/FlowSpec.scala | 124 +++++---- .../scala/akka/stream/FlowSplitWhenSpec.scala | 56 ++-- .../test/scala/akka/stream/FlowTakeSpec.scala | 2 +- .../akka/stream/FlowTakeWithinSpec.scala | 17 +- .../test/scala/akka/stream/FlowTeeSpec.scala | 38 +-- .../stream/FlowTimerTransformerSpec.scala | 36 +-- .../scala/akka/stream/FlowToFutureSpec.scala | 12 +- .../stream/FlowTransformRecoverSpec.scala | 248 +++++++++--------- .../scala/akka/stream/FlowTransformSpec.scala | 233 ++++++++-------- .../test/scala/akka/stream/FlowZipSpec.scala | 65 +++-- .../akka/stream/IdentityProcessorTest.scala | 56 +--- .../akka/stream/IterableProducerTest.scala | 37 +-- .../akka/stream/IteratorProducerTest.scala | 37 +-- .../akka/stream/ProcessorHierarchySpec.scala | 2 +- ...ucerSpec.scala => TickPublisherSpec.scala} | 43 ++- .../scala/akka/stream/TwoStreamsSetup.scala | 76 +++--- .../scala/akka/stream/WithActorSystem.scala | 8 +- ...cerSpec.scala => ActorPublisherSpec.scala} | 126 ++++----- ...erSpec.scala => ActorSubscriberSpec.scala} | 58 ++-- .../akka/stream/extra/FlowTimedSpec.scala | 26 +- ...ynchronousPublisherFromIterableSpec.scala} | 116 ++++---- .../scala/akka/stream/io/TcpFlowSpec.scala | 40 +-- .../stream/testkit/AkkaConsumerProbe.scala | 10 - .../stream/testkit/AkkaProducerProbe.scala | 10 - .../akka/stream/testkit/ChainSetup.scala | 8 +- .../akka/stream/testkit/ConsumerProbe.scala | 31 --- .../akka/stream/testkit/ProducerProbe.scala | 35 --- .../akka/stream/testkit/ScriptedTest.scala | 20 +- .../akka/stream/testkit/StreamTestKit.scala | 231 ++++++++-------- 120 files changed, 2330 insertions(+), 2674 deletions(-) rename akka-stream/src/main/scala/akka/stream/actor/{ActorProducer.scala => ActorPublisher.scala} (73%) rename akka-stream/src/main/scala/akka/stream/actor/{ActorConsumer.scala => ActorSubscriber.scala} (72%) rename akka-stream/src/main/scala/akka/stream/impl/{ActorProducer.scala => ActorPublisher.scala} (78%) rename akka-stream/src/main/scala/akka/stream/impl/{BlackholeConsumer.scala => BlackholeSubscriber.scala} (68%) create mode 100644 akka-stream/src/main/scala/akka/stream/impl/CompletedPublishers.scala delete mode 100644 akka-stream/src/main/scala/akka/stream/impl/EmptyProducer.scala rename akka-stream/src/main/scala/akka/stream/impl/{FutureProducer.scala => FuturePublisher.scala} (83%) rename akka-stream/src/main/scala/akka/stream/impl/{IterableProducer.scala => IterablePublisher.scala} (76%) rename akka-stream/src/main/scala/akka/stream/impl/{IteratorProducer.scala => IteratorPublisher.scala} (81%) rename akka-stream/src/main/scala/akka/stream/impl/{SynchronousProducerFromIterable.scala => SynchronousPublisherFromIterable.scala} (62%) rename akka-stream/src/main/scala/akka/stream/impl/{TickProducer.scala => TickPublisher.scala} (76%) delete mode 100644 akka-stream/src/test/scala/akka/stream/ActorProducerTest.scala create mode 100644 akka-stream/src/test/scala/akka/stream/ActorPublisherTest.scala rename akka-stream/src/test/scala/akka/stream/{FlowProduceToConsumerSpec.scala => FlowProduceToSubscriberSpec.scala} (71%) rename akka-stream/src/test/scala/akka/stream/{TickProducerSpec.scala => TickPublisherSpec.scala} (75%) rename akka-stream/src/test/scala/akka/stream/actor/{ActorProducerSpec.scala => ActorPublisherSpec.scala} (62%) rename akka-stream/src/test/scala/akka/stream/actor/{ActorConsumerSpec.scala => ActorSubscriberSpec.scala} (79%) rename akka-stream/src/test/scala/akka/stream/impl/{SynchronousProducerFromIterableSpec.scala => SynchronousPublisherFromIterableSpec.scala} (59%) delete mode 100644 akka-stream/src/test/scala/akka/stream/testkit/AkkaConsumerProbe.scala delete mode 100644 akka-stream/src/test/scala/akka/stream/testkit/AkkaProducerProbe.scala delete mode 100644 akka-stream/src/test/scala/akka/stream/testkit/ConsumerProbe.scala delete mode 100644 akka-stream/src/test/scala/akka/stream/testkit/ProducerProbe.scala diff --git a/akka-http-core/src/main/java/akka/http/model/japi/HttpEntity.java b/akka-http-core/src/main/java/akka/http/model/japi/HttpEntity.java index 9cb5c28ca0..c65f070073 100644 --- a/akka-http-core/src/main/java/akka/http/model/japi/HttpEntity.java +++ b/akka-http-core/src/main/java/akka/http/model/japi/HttpEntity.java @@ -7,7 +7,7 @@ package akka.http.model.japi; import akka.http.model.HttpEntity$; import akka.stream.FlowMaterializer; import akka.util.ByteString; -import org.reactivestreams.api.Producer; +import org.reactivestreams.Publisher; import java.io.File; @@ -68,7 +68,7 @@ public abstract class HttpEntity { /** * Returns a stream of data bytes this entity consists of. */ - public abstract Producer getDataBytes(FlowMaterializer materializer); + public abstract Publisher getDataBytes(FlowMaterializer materializer); public static HttpEntityStrict create(String string) { return HttpEntity$.MODULE$.apply(string); @@ -91,18 +91,18 @@ public abstract class HttpEntity { public static HttpEntityRegular create(ContentType contentType, File file) { return (HttpEntityRegular) HttpEntity$.MODULE$.apply((akka.http.model.ContentType) contentType, file); } - public static HttpEntityDefault create(ContentType contentType, long contentLength, Producer data) { + public static HttpEntityDefault create(ContentType contentType, long contentLength, Publisher data) { return new akka.http.model.HttpEntity.Default((akka.http.model.ContentType) contentType, contentLength, data); } - public static HttpEntityCloseDelimited createCloseDelimited(ContentType contentType, Producer data) { + public static HttpEntityCloseDelimited createCloseDelimited(ContentType contentType, Publisher data) { return new akka.http.model.HttpEntity.CloseDelimited((akka.http.model.ContentType) contentType, data); } - public static HttpEntityChunked createChunked(ContentType contentType, Producer chunks) { + public static HttpEntityChunked createChunked(ContentType contentType, Publisher chunks) { return new akka.http.model.HttpEntity.Chunked( (akka.http.model.ContentType) contentType, - Util.upcastProducer(chunks)); + Util.upcastPublisher(chunks)); } - public static HttpEntityChunked createChunked(ContentType contentType, Producer data, FlowMaterializer materializer) { + public static HttpEntityChunked createChunked(ContentType contentType, Publisher data, FlowMaterializer materializer) { return akka.http.model.HttpEntity.Chunked$.MODULE$.apply( (akka.http.model.ContentType) contentType, data, materializer); diff --git a/akka-http-core/src/main/java/akka/http/model/japi/HttpEntityChunked.java b/akka-http-core/src/main/java/akka/http/model/japi/HttpEntityChunked.java index a1d9acfae2..2c162034b2 100644 --- a/akka-http-core/src/main/java/akka/http/model/japi/HttpEntityChunked.java +++ b/akka-http-core/src/main/java/akka/http/model/japi/HttpEntityChunked.java @@ -4,12 +4,12 @@ package akka.http.model.japi; -import org.reactivestreams.api.Producer; +import org.reactivestreams.Publisher; /** * Represents an entity transferred using `Transfer-Encoding: chunked`. It consists of a * stream of {@link ChunkStreamPart}. */ public abstract class HttpEntityChunked extends HttpEntityRegular { - public abstract Producer getChunks(); + public abstract Publisher getChunks(); } diff --git a/akka-http-core/src/main/java/akka/http/model/japi/HttpEntityCloseDelimited.java b/akka-http-core/src/main/java/akka/http/model/japi/HttpEntityCloseDelimited.java index a5ab33b8ab..26f3b45a43 100644 --- a/akka-http-core/src/main/java/akka/http/model/japi/HttpEntityCloseDelimited.java +++ b/akka-http-core/src/main/java/akka/http/model/japi/HttpEntityCloseDelimited.java @@ -5,7 +5,7 @@ package akka.http.model.japi; import akka.util.ByteString; -import org.reactivestreams.api.Producer; +import org.reactivestreams.Publisher; /** * Represents an entity without a predetermined content-length. Its length is implicitly @@ -13,5 +13,5 @@ import org.reactivestreams.api.Producer; * available for Http responses. */ public abstract class HttpEntityCloseDelimited extends HttpEntity { - public abstract Producer data(); + public abstract Publisher data(); } diff --git a/akka-http-core/src/main/java/akka/http/model/japi/HttpEntityDefault.java b/akka-http-core/src/main/java/akka/http/model/japi/HttpEntityDefault.java index 9f35dab0aa..7b4a5e8fa8 100644 --- a/akka-http-core/src/main/java/akka/http/model/japi/HttpEntityDefault.java +++ b/akka-http-core/src/main/java/akka/http/model/japi/HttpEntityDefault.java @@ -5,12 +5,12 @@ package akka.http.model.japi; import akka.util.ByteString; -import org.reactivestreams.api.Producer; +import org.reactivestreams.Publisher; /** * The default entity type which has a predetermined length and a stream of data bytes. */ public abstract class HttpEntityDefault extends HttpEntityRegular { public abstract long contentLength(); - public abstract Producer data(); + public abstract Publisher data(); } diff --git a/akka-http-core/src/main/java/akka/http/model/japi/IncomingConnection.java b/akka-http-core/src/main/java/akka/http/model/japi/IncomingConnection.java index 6db37fb9d3..e23a30cc2f 100644 --- a/akka-http-core/src/main/java/akka/http/model/japi/IncomingConnection.java +++ b/akka-http-core/src/main/java/akka/http/model/japi/IncomingConnection.java @@ -4,8 +4,8 @@ package akka.http.model.japi; -import org.reactivestreams.api.Consumer; -import org.reactivestreams.api.Producer; +import org.reactivestreams.Subscriber; +import org.reactivestreams.Publisher; import java.net.InetSocketAddress; @@ -21,10 +21,10 @@ public interface IncomingConnection { /** * A stream of requests coming in from the peer. */ - Producer getRequestProducer(); + Publisher getRequestPublisher(); /** - * A consumer of HttpResponses to be sent to the peer. + * A subscriber of HttpResponses to be sent to the peer. */ - Consumer getResponseConsumer(); + Subscriber getResponseSubscriber(); } diff --git a/akka-http-core/src/main/java/akka/http/model/japi/ServerBinding.java b/akka-http-core/src/main/java/akka/http/model/japi/ServerBinding.java index 4a870ad14e..2d87765558 100644 --- a/akka-http-core/src/main/java/akka/http/model/japi/ServerBinding.java +++ b/akka-http-core/src/main/java/akka/http/model/japi/ServerBinding.java @@ -4,7 +4,7 @@ package akka.http.model.japi; -import org.reactivestreams.api.Producer; +import org.reactivestreams.Publisher; import java.net.InetSocketAddress; @@ -20,8 +20,8 @@ public interface ServerBinding { /** * The stream of incoming connections. The binding is solved and the listening - * socket closed as soon as all consumer of this streams have cancelled their + * socket closed as soon as all subscriber of this streams have cancelled their * subscription. */ - Producer getConnectionStream(); + Publisher getConnectionStream(); } diff --git a/akka-http-core/src/main/java/akka/http/model/japi/Util.java b/akka-http-core/src/main/java/akka/http/model/japi/Util.java index f26c41e7fc..09cabe14fe 100644 --- a/akka-http-core/src/main/java/akka/http/model/japi/Util.java +++ b/akka-http-core/src/main/java/akka/http/model/japi/Util.java @@ -7,7 +7,7 @@ package akka.http.model.japi; import akka.http.model.*; import akka.http.util.ObjectRegistry; import akka.japi.Option; -import org.reactivestreams.api.Producer; +import org.reactivestreams.Publisher; import scala.None; import scala.None$; import scala.NotImplementedError; @@ -22,16 +22,20 @@ import java.util.Map; */ public abstract class Util { @SuppressWarnings("unchecked") // no support for covariance of option in Java + // needed to provide covariant conversions that the Java interfaces don't provide automatically. + // The alternative would be having to cast around everywhere instead of doing it here in a central place. public static Option convertOption(scala.Option o) { return (Option)(Option) akka.japi.Option.fromScalaOption(o); } - @SuppressWarnings("unchecked") // no support for covariance of Producer in Java - public static Producer convertProducer(Producer p) { - return (Producer)(Producer) p; + @SuppressWarnings("unchecked") // no support for covariance of Publisher in Java + // needed to provide covariant conversions that the Java interfaces don't provide automatically. + // The alternative would be having to cast around everywhere instead of doing it here in a central place. + public static Publisher convertPublisher(Publisher p) { + return (Publisher)(Publisher) p; } @SuppressWarnings("unchecked") - public static Producer upcastProducer(Producer p) { - return (Producer)(Producer) p; + public static Publisher upcastPublisher(Publisher p) { + return (Publisher)(Publisher) p; } @SuppressWarnings("unchecked") public static scala.collection.immutable.Map convertMapToScala(Map map) { diff --git a/akka-http-core/src/main/scala/akka/http/Http.scala b/akka-http-core/src/main/scala/akka/http/Http.scala index 22274d7c6a..491f101046 100644 --- a/akka-http-core/src/main/scala/akka/http/Http.scala +++ b/akka-http-core/src/main/scala/akka/http/Http.scala @@ -6,7 +6,7 @@ package akka.http import java.net.InetSocketAddress import com.typesafe.config.Config -import org.reactivestreams.api.{ Producer, Consumer } +import org.reactivestreams.{ Publisher, Subscriber } import scala.collection.immutable import akka.io.Inet import akka.stream.MaterializerSettings @@ -110,18 +110,18 @@ object Http extends ExtensionKey[HttpExt] { } final case class ServerBinding(localAddress: InetSocketAddress, - connectionStream: Producer[IncomingConnection]) extends model.japi.ServerBinding { + connectionStream: Publisher[IncomingConnection]) extends model.japi.ServerBinding { /** Java API */ - def getConnectionStream: Producer[japi.IncomingConnection] = connectionStream.asInstanceOf[Producer[japi.IncomingConnection]] + def getConnectionStream: Publisher[japi.IncomingConnection] = connectionStream.asInstanceOf[Publisher[japi.IncomingConnection]] } final case class IncomingConnection(remoteAddress: InetSocketAddress, - requestProducer: Producer[HttpRequest], - responseConsumer: Consumer[HttpResponse]) extends model.japi.IncomingConnection { + requestPublisher: Publisher[HttpRequest], + responseSubscriber: Subscriber[HttpResponse]) extends model.japi.IncomingConnection { /** Java API */ - def getRequestProducer: Producer[japi.HttpRequest] = requestProducer.asInstanceOf[Producer[japi.HttpRequest]] + def getRequestPublisher: Publisher[japi.HttpRequest] = requestPublisher.asInstanceOf[Publisher[japi.HttpRequest]] /** Java API */ - def getResponseConsumer: Consumer[japi.HttpResponse] = responseConsumer.asInstanceOf[Consumer[japi.HttpResponse]] + def getResponseSubscriber: Subscriber[japi.HttpResponse] = responseSubscriber.asInstanceOf[Subscriber[japi.HttpResponse]] } case object BindFailedException extends SingletonException diff --git a/akka-http-core/src/main/scala/akka/http/HttpManager.scala b/akka-http-core/src/main/scala/akka/http/HttpManager.scala index 6784a9171f..455e3f3504 100644 --- a/akka-http-core/src/main/scala/akka/http/HttpManager.scala +++ b/akka-http-core/src/main/scala/akka/http/HttpManager.scala @@ -65,7 +65,7 @@ private[http] class HttpManager(httpSettings: HttpExt#Settings) extends Actor wi val httpServerPipeline = new HttpServerPipeline(effectiveSettings, materializer, log) val httpConnectionStream = Flow(connectionStream) .map(httpServerPipeline) - .toProducer(materializer) + .toPublisher(materializer) commander ! Http.ServerBinding(localAddress, httpConnectionStream) case Failure(error) ⇒ diff --git a/akka-http-core/src/main/scala/akka/http/client/HttpClientPipeline.scala b/akka-http-core/src/main/scala/akka/http/client/HttpClientPipeline.scala index 070f19b55a..725d211e6c 100644 --- a/akka-http-core/src/main/scala/akka/http/client/HttpClientPipeline.scala +++ b/akka-http-core/src/main/scala/akka/http/client/HttpClientPipeline.scala @@ -37,19 +37,19 @@ private[http] class HttpClientPipeline(effectiveSettings: ClientConnectionSettin def apply(tcpConn: StreamTcp.OutgoingTcpConnection): Http.OutgoingConnection = { val requestMethodByPass = new RequestMethodByPass(tcpConn.remoteAddress) - val (contextBypassConsumer, contextBypassProducer) = + val (contextBypassSubscriber, contextBypassPublisher) = Duct[(HttpRequest, Any)].map(_._2).build(materializer) - val requestConsumer = + val requestSubscriber = Duct[(HttpRequest, Any)] - .tee(contextBypassConsumer) + .tee(contextBypassSubscriber) .map(requestMethodByPass) .transform(responseRendererFactory.newRenderer) .flatten(FlattenStrategy.concat) .transform(errorLogger(log, "Outgoing request stream error")) .produceTo(materializer, tcpConn.outputStream) - val responseProducer = + val responsePublisher = Flow(tcpConn.inputStream) .transform(rootParser.copyWith(warnOnIllegalHeader, requestMethodByPass)) .splitWhen(_.isInstanceOf[MessageStart]) @@ -58,10 +58,10 @@ private[http] class HttpClientPipeline(effectiveSettings: ClientConnectionSettin case (ResponseStart(statusCode, protocol, headers, createEntity, _), entityParts) ⇒ HttpResponse(statusCode, headers, createEntity(entityParts), protocol) } - .zip(contextBypassProducer) - .toProducer(materializer) + .zip(contextBypassPublisher) + .toPublisher(materializer) - val processor = HttpClientProcessor(requestConsumer.getSubscriber, responseProducer.getPublisher) + val processor = HttpClientProcessor(requestSubscriber, responsePublisher) Http.OutgoingConnection(tcpConn.remoteAddress, tcpConn.localAddress, processor) } diff --git a/akka-http-core/src/main/scala/akka/http/client/HttpClientProcessor.scala b/akka-http-core/src/main/scala/akka/http/client/HttpClientProcessor.scala index 055821dbe2..750a480781 100644 --- a/akka-http-core/src/main/scala/akka/http/client/HttpClientProcessor.scala +++ b/akka-http-core/src/main/scala/akka/http/client/HttpClientProcessor.scala @@ -5,8 +5,7 @@ package akka.http.client import akka.http.model.{ HttpResponse, HttpRequest } -import org.reactivestreams.spi.{ Publisher, Subscriber } -import org.reactivestreams.api.{ Consumer, Processor } +import org.reactivestreams.{ Subscription, Publisher, Subscriber, Processor } /** * A `HttpClientProcessor` models an HTTP client as a stream processor that provides @@ -19,8 +18,11 @@ object HttpClientProcessor { def apply[T](requestSubscriber: Subscriber[(HttpRequest, T)], responsePublisher: Publisher[(HttpResponse, T)]): HttpClientProcessor[T] = new HttpClientProcessor[T] { - def getSubscriber = requestSubscriber - def getPublisher = responsePublisher - def produceTo(consumer: Consumer[(HttpResponse, T)]): Unit = responsePublisher.subscribe(consumer.getSubscriber) + override def subscribe(s: Subscriber[(HttpResponse, T)]): Unit = responsePublisher.subscribe(s) + + override def onError(t: Throwable): Unit = requestSubscriber.onError(t) + override def onSubscribe(s: Subscription): Unit = requestSubscriber.onSubscribe(s) + override def onComplete(): Unit = requestSubscriber.onComplete() + override def onNext(t: (HttpRequest, T)): Unit = requestSubscriber.onNext(t) } } \ No newline at end of file diff --git a/akka-http-core/src/main/scala/akka/http/model/HttpEntity.scala b/akka-http-core/src/main/scala/akka/http/model/HttpEntity.scala index 486370414a..185f895c80 100644 --- a/akka-http-core/src/main/scala/akka/http/model/HttpEntity.scala +++ b/akka-http-core/src/main/scala/akka/http/model/HttpEntity.scala @@ -6,13 +6,13 @@ package akka.http.model import language.implicitConversions import java.io.File -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import scala.collection.immutable import akka.util.ByteString import akka.stream.{ TimerTransformer, FlowMaterializer } import akka.stream.scaladsl.Flow -import akka.stream.impl.{ EmptyProducer, SynchronousProducerFromIterable } +import akka.stream.impl.{ EmptyPublisher, SynchronousPublisherFromIterable } import java.lang.Iterable import japi.JavaMapping.Implicits._ import scala.concurrent.{ ExecutionContext, Future } @@ -35,7 +35,7 @@ sealed trait HttpEntity extends japi.HttpEntity { /** * A stream of the data of this entity. */ - def dataBytes(materializer: FlowMaterializer): Producer[ByteString] + def dataBytes(materializer: FlowMaterializer): Publisher[ByteString] /** * Collects all possible parts and returns a future Strict entity for easier processing. The future is failed with an @@ -62,7 +62,7 @@ sealed trait HttpEntity extends japi.HttpEntity { .toFuture(materializer) /** Java API */ - def getDataBytes(materializer: FlowMaterializer): Producer[ByteString] = dataBytes(materializer) + def getDataBytes(materializer: FlowMaterializer): Publisher[ByteString] = dataBytes(materializer) // default implementations, should be overridden def isCloseDelimited: Boolean = false @@ -81,12 +81,12 @@ object HttpEntity { if (bytes.length == 0) empty(contentType) else apply(contentType, ByteString(bytes)) def apply(contentType: ContentType, data: ByteString): Strict = if (data.isEmpty) empty(contentType) else Strict(contentType, data) - def apply(contentType: ContentType, contentLength: Long, data: Producer[ByteString]): Regular = + def apply(contentType: ContentType, contentLength: Long, data: Publisher[ByteString]): Regular = if (contentLength == 0) empty(contentType) else Default(contentType, contentLength, data) def apply(contentType: ContentType, file: File): Regular = { val fileLength = file.length - if (fileLength > 0) Default(contentType, fileLength, ???) // FIXME: attach from-file-Producer + if (fileLength > 0) Default(contentType, fileLength, ???) // FIXME: attach from-file-Publisher else empty(contentType) } @@ -116,7 +116,7 @@ object HttpEntity { final case class Strict(contentType: ContentType, data: ByteString) extends japi.HttpEntityStrict with Regular { def isKnownEmpty: Boolean = data.isEmpty - def dataBytes(materializer: FlowMaterializer): Producer[ByteString] = SynchronousProducerFromIterable(data :: Nil) + def dataBytes(materializer: FlowMaterializer): Publisher[ByteString] = SynchronousPublisherFromIterable(data :: Nil) override def toStrict(timeout: FiniteDuration, materializer: FlowMaterializer)(implicit ec: ExecutionContext): Future[Strict] = Future.successful(this) @@ -127,12 +127,12 @@ object HttpEntity { */ final case class Default(contentType: ContentType, contentLength: Long, - data: Producer[ByteString]) extends japi.HttpEntityDefault with Regular { + data: Publisher[ByteString]) extends japi.HttpEntityDefault with Regular { require(contentLength > 0, "contentLength must be positive (use `HttpEntity.empty(contentType)` for empty entities)") def isKnownEmpty = false override def isDefault: Boolean = true - def dataBytes(materializer: FlowMaterializer): Producer[ByteString] = data + def dataBytes(materializer: FlowMaterializer): Publisher[ByteString] = data } /** @@ -140,33 +140,35 @@ object HttpEntity { * The content-length of such responses is unknown at the time the response headers have been received. * Note that this type of HttpEntity cannot be used for HttpRequests! */ - final case class CloseDelimited(contentType: ContentType, data: Producer[ByteString]) extends japi.HttpEntityCloseDelimited with HttpEntity { - def isKnownEmpty = data eq EmptyProducer + final case class CloseDelimited(contentType: ContentType, data: Publisher[ByteString]) extends japi.HttpEntityCloseDelimited with HttpEntity { + def isKnownEmpty = data eq EmptyPublisher override def isCloseDelimited: Boolean = true - def dataBytes(materializer: FlowMaterializer): Producer[ByteString] = data + def dataBytes(materializer: FlowMaterializer): Publisher[ByteString] = data } /** * The model for the entity of a chunked HTTP message (with `Transfer-Encoding: chunked`). */ - final case class Chunked(contentType: ContentType, chunks: Producer[ChunkStreamPart]) extends japi.HttpEntityChunked with Regular { - def isKnownEmpty = chunks eq EmptyProducer + final case class Chunked(contentType: ContentType, chunks: Publisher[ChunkStreamPart]) extends japi.HttpEntityChunked with Regular { + def isKnownEmpty = chunks eq EmptyPublisher override def isChunked: Boolean = true - def dataBytes(materializer: FlowMaterializer): Producer[ByteString] = - Flow(chunks).map(_.data).filter(_.nonEmpty).toProducer(materializer) + def dataBytes(materializer: FlowMaterializer): Publisher[ByteString] = + Flow(chunks).map(_.data).filter(_.nonEmpty).toPublisher(materializer) /** Java API */ - def getChunks: Producer[japi.ChunkStreamPart] = chunks.asInstanceOf[Producer[japi.ChunkStreamPart]] + def getChunks: Publisher[japi.ChunkStreamPart] = chunks.asInstanceOf[Publisher[japi.ChunkStreamPart]] } object Chunked { /** * Returns a ``Chunked`` entity where one Chunk is produced for every non-empty ByteString of the given - * ``Producer[ByteString]``. + * ``Publisher[ByteString]``. */ - def apply(contentType: ContentType, chunks: Producer[ByteString], materializer: FlowMaterializer): Chunked = - Chunked(contentType, Flow(chunks).filter(_.nonEmpty).map[ChunkStreamPart](Chunk(_)).toProducer(materializer)) + def apply(contentType: ContentType, chunks: Publisher[ByteString], materializer: FlowMaterializer): Chunked = + Chunked(contentType, Flow(chunks).collect[ChunkStreamPart] { + case b: ByteString if b.nonEmpty => Chunk(b) + }.toPublisher(materializer)) } /** diff --git a/akka-http-core/src/main/scala/akka/http/model/MultipartContent.scala b/akka-http-core/src/main/scala/akka/http/model/MultipartContent.scala index e95f1ed186..5060c0e1ef 100644 --- a/akka-http-core/src/main/scala/akka/http/model/MultipartContent.scala +++ b/akka-http-core/src/main/scala/akka/http/model/MultipartContent.scala @@ -5,25 +5,25 @@ package akka.http.model import java.io.File -import org.reactivestreams.api.Producer -import akka.stream.impl.SynchronousProducerFromIterable +import org.reactivestreams.Publisher +import akka.stream.impl.SynchronousPublisherFromIterable import scala.collection.immutable import headers._ trait MultipartParts { - def parts: Producer[BodyPart] + def parts: Publisher[BodyPart] } /** * Basic model for multipart content as defined in RFC 2046. * If you are looking for a model for `multipart/form-data` you should be using [[MultipartFormData]]. */ -final case class MultipartContent(parts: Producer[BodyPart]) extends MultipartParts +final case class MultipartContent(parts: Publisher[BodyPart]) extends MultipartParts object MultipartContent { - val Empty = MultipartContent(SynchronousProducerFromIterable[BodyPart](Nil)) + val Empty = MultipartContent(SynchronousPublisherFromIterable[BodyPart](Nil)) - def apply(parts: BodyPart*): MultipartContent = apply(SynchronousProducerFromIterable[BodyPart](parts.toList)) + def apply(parts: BodyPart*): MultipartContent = apply(SynchronousPublisherFromIterable[BodyPart](parts.toList)) def apply(files: Map[String, FormFile]): MultipartContent = apply(files.map(e ⇒ BodyPart(e._2, e._1))(collection.breakOut): _*) @@ -33,12 +33,12 @@ object MultipartContent { * Model for multipart/byteranges content as defined in RFC 2046. * If you are looking for a model for `multipart/form-data` you should be using [[MultipartFormData]]. */ -final case class MultipartByteRanges(parts: Producer[BodyPart]) extends MultipartParts +final case class MultipartByteRanges(parts: Publisher[BodyPart]) extends MultipartParts object MultipartByteRanges { - val Empty = MultipartByteRanges(SynchronousProducerFromIterable[BodyPart](Nil)) + val Empty = MultipartByteRanges(SynchronousPublisherFromIterable[BodyPart](Nil)) - def apply(parts: BodyPart*): MultipartByteRanges = apply(SynchronousProducerFromIterable[BodyPart](parts.toList)) + def apply(parts: BodyPart*): MultipartByteRanges = apply(SynchronousPublisherFromIterable[BodyPart](parts.toList)) } /** diff --git a/akka-http-core/src/main/scala/akka/http/parsing/HttpMessageParser.scala b/akka-http-core/src/main/scala/akka/http/parsing/HttpMessageParser.scala index 7641ce9f3e..86e499a7f6 100644 --- a/akka-http-core/src/main/scala/akka/http/parsing/HttpMessageParser.scala +++ b/akka-http-core/src/main/scala/akka/http/parsing/HttpMessageParser.scala @@ -14,7 +14,7 @@ import akka.http.model.parser.CharacterClasses import akka.http.model._ import headers._ import HttpProtocols._ -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import akka.stream.scaladsl.Flow /** @@ -236,14 +236,14 @@ private[http] abstract class HttpMessageParser[Output >: ParserOutput.MessageOut HttpEntity.Strict(contentType(cth), input.slice(bodyStart, bodyStart + contentLength)) def defaultEntity(cth: Option[`Content-Type`], contentLength: Long, - materializer: FlowMaterializer)(entityParts: Producer[_ <: ParserOutput]): HttpEntity.Regular = { - val data = Flow(entityParts).collect { case ParserOutput.EntityPart(bytes) ⇒ bytes }.toProducer(materializer) + materializer: FlowMaterializer)(entityParts: Publisher[_ <: ParserOutput]): HttpEntity.Regular = { + val data = Flow(entityParts).collect { case ParserOutput.EntityPart(bytes) ⇒ bytes }.toPublisher(materializer) HttpEntity.Default(contentType(cth), contentLength, data) } def chunkedEntity(cth: Option[`Content-Type`], - materializer: FlowMaterializer)(entityChunks: Producer[_ <: ParserOutput]): HttpEntity.Regular = { - val chunks = Flow(entityChunks).collect { case ParserOutput.EntityChunk(chunk) ⇒ chunk }.toProducer(materializer) + materializer: FlowMaterializer)(entityChunks: Publisher[_ <: ParserOutput]): HttpEntity.Regular = { + val chunks = Flow(entityChunks).collect { case ParserOutput.EntityChunk(chunk) ⇒ chunk }.toPublisher(materializer) HttpEntity.Chunked(contentType(cth), chunks) } } \ No newline at end of file diff --git a/akka-http-core/src/main/scala/akka/http/parsing/HttpRequestParser.scala b/akka-http-core/src/main/scala/akka/http/parsing/HttpRequestParser.scala index fdab5975cf..822fa1f3d2 100644 --- a/akka-http-core/src/main/scala/akka/http/parsing/HttpRequestParser.scala +++ b/akka-http-core/src/main/scala/akka/http/parsing/HttpRequestParser.scala @@ -5,7 +5,7 @@ package akka.http.parsing import java.lang.{ StringBuilder ⇒ JStringBuilder } -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import scala.annotation.tailrec import scala.concurrent.ExecutionContext import akka.http.model.parser.CharacterClasses @@ -110,7 +110,7 @@ private[http] class HttpRequestParser(_settings: ParserSettings, clh: Option[`Content-Length`], cth: Option[`Content-Type`], teh: Option[`Transfer-Encoding`], hostHeaderPresent: Boolean, closeAfterResponseCompletion: Boolean): StateResult = if (hostHeaderPresent || protocol == HttpProtocols.`HTTP/1.0`) { - def emitRequestStart(createEntity: Producer[ParserOutput.RequestOutput] ⇒ HttpEntity.Regular) = + def emitRequestStart(createEntity: Publisher[ParserOutput.RequestOutput] ⇒ HttpEntity.Regular) = emit(ParserOutput.RequestStart(method, uri, protocol, headers, createEntity, closeAfterResponseCompletion)) teh match { diff --git a/akka-http-core/src/main/scala/akka/http/parsing/HttpResponseParser.scala b/akka-http-core/src/main/scala/akka/http/parsing/HttpResponseParser.scala index d529ddf62d..5ecf724947 100644 --- a/akka-http-core/src/main/scala/akka/http/parsing/HttpResponseParser.scala +++ b/akka-http-core/src/main/scala/akka/http/parsing/HttpResponseParser.scala @@ -4,7 +4,7 @@ package akka.http.parsing -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import scala.annotation.tailrec import akka.http.model.parser.CharacterClasses import akka.stream.FlowMaterializer @@ -75,7 +75,7 @@ private[http] class HttpResponseParser(_settings: ParserSettings, def parseEntity(headers: List[HttpHeader], protocol: HttpProtocol, input: ByteString, bodyStart: Int, clh: Option[`Content-Length`], cth: Option[`Content-Type`], teh: Option[`Transfer-Encoding`], hostHeaderPresent: Boolean, closeAfterResponseCompletion: Boolean): StateResult = { - def emitResponseStart(createEntity: Producer[ParserOutput.ResponseOutput] ⇒ HttpEntity) = + def emitResponseStart(createEntity: Publisher[ParserOutput.ResponseOutput] ⇒ HttpEntity) = emit(ParserOutput.ResponseStart(statusCode, protocol, headers, createEntity, closeAfterResponseCompletion)) def finishEmptyResponse() = { emitResponseStart(emptyEntity(cth)) @@ -99,7 +99,7 @@ private[http] class HttpResponseParser(_settings: ParserSettings, } case None ⇒ emitResponseStart { entityParts ⇒ - val data = Flow(entityParts).collect { case ParserOutput.EntityPart(bytes) ⇒ bytes }.toProducer(materializer) + val data = Flow(entityParts).collect { case ParserOutput.EntityPart(bytes) ⇒ bytes }.toPublisher(materializer) HttpEntity.CloseDelimited(contentType(cth), data) } parseToCloseBody(input, bodyStart) diff --git a/akka-http-core/src/main/scala/akka/http/parsing/ParserOutput.scala b/akka-http-core/src/main/scala/akka/http/parsing/ParserOutput.scala index 1dcdcbd471..8e62e5a8c0 100644 --- a/akka-http-core/src/main/scala/akka/http/parsing/ParserOutput.scala +++ b/akka-http-core/src/main/scala/akka/http/parsing/ParserOutput.scala @@ -4,7 +4,7 @@ package akka.http.parsing -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import akka.http.model._ import akka.util.ByteString @@ -27,14 +27,14 @@ private[http] object ParserOutput { uri: Uri, protocol: HttpProtocol, headers: List[HttpHeader], - createEntity: Producer[RequestOutput] ⇒ HttpEntity.Regular, + createEntity: Publisher[RequestOutput] ⇒ HttpEntity.Regular, closeAfterResponseCompletion: Boolean) extends MessageStart with RequestOutput final case class ResponseStart( statusCode: StatusCode, protocol: HttpProtocol, headers: List[HttpHeader], - createEntity: Producer[ResponseOutput] ⇒ HttpEntity, + createEntity: Publisher[ResponseOutput] ⇒ HttpEntity, closeAfterResponseCompletion: Boolean) extends MessageStart with ResponseOutput final case class EntityPart(data: ByteString) extends MessageOutput diff --git a/akka-http-core/src/main/scala/akka/http/rendering/HttpRequestRendererFactory.scala b/akka-http-core/src/main/scala/akka/http/rendering/HttpRequestRendererFactory.scala index 99b90684e5..bfa5c38d8b 100644 --- a/akka-http-core/src/main/scala/akka/http/rendering/HttpRequestRendererFactory.scala +++ b/akka-http-core/src/main/scala/akka/http/rendering/HttpRequestRendererFactory.scala @@ -5,14 +5,14 @@ package akka.http.rendering import java.net.InetSocketAddress -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import scala.annotation.tailrec import scala.collection.immutable import akka.event.LoggingAdapter import akka.util.ByteString import akka.stream.scaladsl.Flow import akka.stream.{ FlowMaterializer, Transformer } -import akka.stream.impl.SynchronousProducerFromIterable +import akka.stream.impl.SynchronousPublisherFromIterable import akka.http.model._ import akka.http.util._ import RenderSupport._ @@ -28,9 +28,9 @@ private[http] class HttpRequestRendererFactory(userAgentHeader: Option[headers.` def newRenderer: HttpRequestRenderer = new HttpRequestRenderer - final class HttpRequestRenderer extends Transformer[RequestRenderingContext, Producer[ByteString]] { + final class HttpRequestRenderer extends Transformer[RequestRenderingContext, Publisher[ByteString]] { - def onNext(ctx: RequestRenderingContext): immutable.Seq[Producer[ByteString]] = { + def onNext(ctx: RequestRenderingContext): immutable.Seq[Publisher[ByteString]] = { val r = new ByteStringRendering(requestHeaderSizeHint) import ctx.request._ @@ -97,21 +97,21 @@ private[http] class HttpRequestRendererFactory(userAgentHeader: Option[headers.` r ~~ CrLf } - def completeRequestRendering(): immutable.Seq[Producer[ByteString]] = + def completeRequestRendering(): immutable.Seq[Publisher[ByteString]] = entity match { case HttpEntity.Strict(contentType, data) ⇒ renderContentLength(data.length) - SynchronousProducerFromIterable(r.get :: data :: Nil) :: Nil + SynchronousPublisherFromIterable(r.get :: data :: Nil) :: Nil case HttpEntity.Default(contentType, contentLength, data) ⇒ renderContentLength(contentLength) renderByteStrings(r, - Flow(data).transform(new CheckContentLengthTransformer(contentLength)).toProducer(materializer), + Flow(data).transform(new CheckContentLengthTransformer(contentLength)).toPublisher(materializer), materializer) case HttpEntity.Chunked(contentType, chunks) ⇒ r ~~ `Transfer-Encoding` ~~ Chunked ~~ CrLf ~~ CrLf - renderByteStrings(r, Flow(chunks).transform(new ChunkTransformer).toProducer(materializer), materializer) + renderByteStrings(r, Flow(chunks).transform(new ChunkTransformer).toPublisher(materializer), materializer) } renderRequestLine() @@ -119,7 +119,7 @@ private[http] class HttpRequestRendererFactory(userAgentHeader: Option[headers.` renderEntityContentType(r, entity) if (entity.isKnownEmpty) { renderContentLength(0) - SynchronousProducerFromIterable(r.get :: Nil) :: Nil + SynchronousPublisherFromIterable(r.get :: Nil) :: Nil } else completeRequestRendering() } } diff --git a/akka-http-core/src/main/scala/akka/http/rendering/HttpResponseRendererFactory.scala b/akka-http-core/src/main/scala/akka/http/rendering/HttpResponseRendererFactory.scala index 0a3e59b4f1..aed0134520 100644 --- a/akka-http-core/src/main/scala/akka/http/rendering/HttpResponseRendererFactory.scala +++ b/akka-http-core/src/main/scala/akka/http/rendering/HttpResponseRendererFactory.scala @@ -4,13 +4,13 @@ package akka.http.rendering -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import scala.annotation.tailrec import scala.collection.immutable import akka.event.LoggingAdapter import akka.util.ByteString import akka.stream.scaladsl.Flow -import akka.stream.impl.SynchronousProducerFromIterable +import akka.stream.impl.SynchronousPublisherFromIterable import akka.stream.{ FlowMaterializer, Transformer } import akka.http.model._ import akka.http.util._ @@ -54,12 +54,12 @@ private[http] class HttpResponseRendererFactory(serverHeader: Option[headers.Ser def newRenderer: HttpResponseRenderer = new HttpResponseRenderer - final class HttpResponseRenderer extends Transformer[ResponseRenderingContext, Producer[ByteString]] { + final class HttpResponseRenderer extends Transformer[ResponseRenderingContext, Publisher[ByteString]] { private[this] var close = false // signals whether the connection is to be closed after the current response override def isComplete = close - def onNext(ctx: ResponseRenderingContext): immutable.Seq[Producer[ByteString]] = { + def onNext(ctx: ResponseRenderingContext): immutable.Seq[Publisher[ByteString]] = { val r = new ByteStringRendering(responseHeaderSizeHint) import ctx.response._ @@ -121,23 +121,23 @@ private[http] class HttpResponseRendererFactory(serverHeader: Option[headers.Ser } } - def byteStrings(entityBytes: ⇒ Producer[ByteString]): immutable.Seq[Producer[ByteString]] = + def byteStrings(entityBytes: ⇒ Publisher[ByteString]): immutable.Seq[Publisher[ByteString]] = renderByteStrings(r, entityBytes, materializer, skipEntity = noEntity) - def completeResponseRendering(entity: HttpEntity): immutable.Seq[Producer[ByteString]] = + def completeResponseRendering(entity: HttpEntity): immutable.Seq[Publisher[ByteString]] = entity match { case HttpEntity.Strict(contentType, data) ⇒ renderHeaders(headers.toList) renderEntityContentType(r, entity) r ~~ `Content-Length` ~~ data.length ~~ CrLf ~~ CrLf val entityBytes = if (noEntity) Nil else data :: Nil - SynchronousProducerFromIterable(r.get :: entityBytes) :: Nil + SynchronousPublisherFromIterable(r.get :: entityBytes) :: Nil case HttpEntity.Default(contentType, contentLength, data) ⇒ renderHeaders(headers.toList) renderEntityContentType(r, entity) r ~~ `Content-Length` ~~ contentLength ~~ CrLf ~~ CrLf - byteStrings(Flow(data).transform(new CheckContentLengthTransformer(contentLength)).toProducer(materializer)) + byteStrings(Flow(data).transform(new CheckContentLengthTransformer(contentLength)).toPublisher(materializer)) case HttpEntity.CloseDelimited(contentType, data) ⇒ renderHeaders(headers.toList, alwaysClose = true) @@ -147,14 +147,14 @@ private[http] class HttpResponseRendererFactory(serverHeader: Option[headers.Ser case HttpEntity.Chunked(contentType, chunks) ⇒ if (ctx.requestProtocol == `HTTP/1.0`) - completeResponseRendering(HttpEntity.CloseDelimited(contentType, Flow(chunks).map(_.data).toProducer(materializer))) + completeResponseRendering(HttpEntity.CloseDelimited(contentType, Flow(chunks).map(_.data).toPublisher(materializer))) else { renderHeaders(headers.toList) renderEntityContentType(r, entity) if (!entity.isKnownEmpty || ctx.requestMethod == HttpMethods.HEAD) r ~~ `Transfer-Encoding` ~~ Chunked ~~ CrLf r ~~ CrLf - byteStrings(Flow(chunks).transform(new ChunkTransformer).toProducer(materializer)) + byteStrings(Flow(chunks).transform(new ChunkTransformer).toPublisher(materializer)) } } diff --git a/akka-http-core/src/main/scala/akka/http/rendering/RenderSupport.scala b/akka-http-core/src/main/scala/akka/http/rendering/RenderSupport.scala index eb972f7ab2..ff851f073d 100644 --- a/akka-http-core/src/main/scala/akka/http/rendering/RenderSupport.scala +++ b/akka-http-core/src/main/scala/akka/http/rendering/RenderSupport.scala @@ -4,12 +4,12 @@ package akka.http.rendering -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import scala.collection.immutable import akka.parboiled2.CharUtils import akka.util.ByteString import akka.event.LoggingAdapter -import akka.stream.impl.SynchronousProducerFromIterable +import akka.stream.impl.SynchronousPublisherFromIterable import akka.stream.scaladsl.Flow import akka.stream.{ FlowMaterializer, Transformer } import akka.http.model._ @@ -35,11 +35,11 @@ private object RenderSupport { if (entity.contentType != ContentTypes.NoContentType) r ~~ headers.`Content-Type` ~~ entity.contentType ~~ CrLf - def renderByteStrings(r: ByteStringRendering, entityBytes: ⇒ Producer[ByteString], materializer: FlowMaterializer, - skipEntity: Boolean = false): immutable.Seq[Producer[ByteString]] = { - val messageStart = SynchronousProducerFromIterable(r.get :: Nil) + def renderByteStrings(r: ByteStringRendering, entityBytes: ⇒ Publisher[ByteString], materializer: FlowMaterializer, + skipEntity: Boolean = false): immutable.Seq[Publisher[ByteString]] = { + val messageStart = SynchronousPublisherFromIterable(r.get :: Nil) val messageBytes = - if (!skipEntity) Flow(messageStart).concat(entityBytes).toProducer(materializer) + if (!skipEntity) Flow(messageStart).concat(entityBytes).toPublisher(materializer) else messageStart messageBytes :: Nil } diff --git a/akka-http-core/src/main/scala/akka/http/server/HttpServerPipeline.scala b/akka-http-core/src/main/scala/akka/http/server/HttpServerPipeline.scala index 1f7dd2272f..0ba396f560 100644 --- a/akka-http-core/src/main/scala/akka/http/server/HttpServerPipeline.scala +++ b/akka-http-core/src/main/scala/akka/http/server/HttpServerPipeline.scala @@ -4,7 +4,7 @@ package akka.http.server -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import akka.event.LoggingAdapter import akka.stream.io.StreamTcp import akka.stream.{ FlattenStrategy, Transformer, FlowMaterializer } @@ -34,34 +34,34 @@ private[http] class HttpServerPipeline(settings: ServerSettings, settings.responseHeaderSizeHint, materializer, log) def apply(tcpConn: StreamTcp.IncomingTcpConnection): Http.IncomingConnection = { - val (applicationBypassConsumer, applicationBypassProducer) = - Duct[(RequestOutput, Producer[RequestOutput])] + val (applicationBypassSubscriber, applicationBypassPublisher) = + Duct[(RequestOutput, Publisher[RequestOutput])] .collect[MessageStart with RequestOutput] { case (x: MessageStart, _) ⇒ x } .build(materializer) - val requestProducer = + val requestPublisher = Flow(tcpConn.inputStream) .transform(rootParser.copyWith(warnOnIllegalHeader)) .splitWhen(_.isInstanceOf[MessageStart]) .headAndTail(materializer) - .tee(applicationBypassConsumer) + .tee(applicationBypassSubscriber) .collect { case (RequestStart(method, uri, protocol, headers, createEntity, _), entityParts) ⇒ val effectiveUri = HttpRequest.effectiveUri(uri, headers, securedConnection = false, settings.defaultHostHeader) HttpRequest(method, effectiveUri, headers, createEntity(entityParts), protocol) } - .toProducer(materializer) + .toPublisher(materializer) - val responseConsumer = + val responseSubscriber = Duct[HttpResponse] - .merge(applicationBypassProducer) + .merge(applicationBypassPublisher) .transform(applyApplicationBypass) .transform(responseRendererFactory.newRenderer) .flatten(FlattenStrategy.concat) .transform(errorLogger(log, "Outgoing response stream error")) .produceTo(materializer, tcpConn.outputStream) - Http.IncomingConnection(tcpConn.remoteAddress, requestProducer, responseConsumer) + Http.IncomingConnection(tcpConn.remoteAddress, requestPublisher, responseSubscriber) } /** diff --git a/akka-http-core/src/main/scala/akka/http/util/Rendering.scala b/akka-http-core/src/main/scala/akka/http/util/Rendering.scala index 0fd6c0385b..59dffeeb4f 100644 --- a/akka-http-core/src/main/scala/akka/http/util/Rendering.scala +++ b/akka-http-core/src/main/scala/akka/http/util/Rendering.scala @@ -4,7 +4,8 @@ package akka.http.util -import java.text.DecimalFormat +import java.text.{ DecimalFormatSymbols, DecimalFormat } +import java.util.Locale import scala.annotation.tailrec import scala.collection.{ immutable, LinearSeq } import akka.parboiled2.{ CharPredicate, CharUtils } @@ -202,7 +203,7 @@ private[http] trait Rendering { } private[http] object Rendering { - val floatFormat = new DecimalFormat("0.0##") + val floatFormat = new DecimalFormat("0.0##", DecimalFormatSymbols.getInstance(Locale.ROOT)) val `\"` = CharPredicate('\\', '"') case object `, ` extends SingletonValueRenderable // default separator diff --git a/akka-http-core/src/main/scala/akka/http/util/package.scala b/akka-http-core/src/main/scala/akka/http/util/package.scala index bbdf9084d2..e0cf456e63 100644 --- a/akka-http-core/src/main/scala/akka/http/util/package.scala +++ b/akka-http-core/src/main/scala/akka/http/util/package.scala @@ -9,7 +9,7 @@ import java.net.InetSocketAddress import java.nio.channels.ServerSocketChannel import java.nio.charset.Charset import com.typesafe.config.Config -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import akka.event.LoggingAdapter import akka.util.ByteString import akka.actor.{ ActorRefFactory, ActorContext, ActorSystem } @@ -30,10 +30,10 @@ package object util { private[http] implicit def enhanceConfig(config: Config): EnhancedConfig = new EnhancedConfig(config) private[http] implicit def enhanceString_(s: String): EnhancedString = new EnhancedString(s) - private[http] implicit class FlowWithHeadAndTail[T](val underlying: Flow[Producer[T]]) extends AnyVal { - def headAndTail(materializer: FlowMaterializer): Flow[(T, Producer[T])] = + private[http] implicit class FlowWithHeadAndTail[T](val underlying: Flow[Publisher[T]]) extends AnyVal { + def headAndTail(materializer: FlowMaterializer): Flow[(T, Publisher[T])] = underlying.map { p ⇒ - Flow(p).prefixAndTail(1).map { case (prefix, tail) ⇒ (prefix.head, tail) }.toProducer(materializer) + Flow(p).prefixAndTail(1).map { case (prefix, tail) ⇒ (prefix.head, tail) }.toPublisher(materializer) }.flatten(FlattenStrategy.Concat()) } diff --git a/akka-http-core/src/test/java/akka/http/model/japi/JavaTestServer.java b/akka-http-core/src/test/java/akka/http/model/japi/JavaTestServer.java index b93d952db1..29004c10c3 100644 --- a/akka-http-core/src/test/java/akka/http/model/japi/JavaTestServer.java +++ b/akka-http-core/src/test/java/akka/http/model/japi/JavaTestServer.java @@ -40,7 +40,7 @@ public abstract class JavaTestServer { public void apply(IncomingConnection conn) throws Exception { System.out.println("New incoming connection from " + conn.remoteAddress()); - Flow.create(conn.getRequestProducer()) + Flow.create(conn.getRequestPublisher()) .map(new Function() { @Override public HttpResponse apply(HttpRequest request) throws Exception { @@ -48,7 +48,7 @@ public abstract class JavaTestServer { return JavaApiTestCases.handleRequest(request); } }) - .produceTo(materializer, conn.getResponseConsumer()); + .produceTo(materializer, conn.getResponseSubscriber()); } }).consume(materializer); } diff --git a/akka-http-core/src/test/scala/akka/http/ClientServerSpec.scala b/akka-http-core/src/test/scala/akka/http/ClientServerSpec.scala index 5fdf0d3b19..3d52e2e306 100644 --- a/akka-http-core/src/test/scala/akka/http/ClientServerSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/ClientServerSpec.scala @@ -6,6 +6,7 @@ package akka.http import java.net.Socket import java.io.{ InputStreamReader, BufferedReader, OutputStreamWriter, BufferedWriter } +import akka.stream.testkit.StreamTestKit.{ SubscriberProbe, PublisherProbe } import com.typesafe.config.{ ConfigFactory, Config } import scala.annotation.tailrec import scala.concurrent.duration._ @@ -15,8 +16,8 @@ import akka.actor.ActorSystem import akka.testkit.TestProbe import akka.io.IO import akka.stream.{ FlowMaterializer, MaterializerSettings } -import akka.stream.testkit.{ ProducerProbe, ConsumerProbe, StreamTestKit } -import akka.stream.impl.SynchronousProducerFromIterable +import akka.stream.testkit.StreamTestKit +import akka.stream.impl.SynchronousPublisherFromIterable import akka.stream.scaladsl.Flow import akka.http.server.ServerSettings import akka.http.client.ClientConnectionSettings @@ -48,8 +49,8 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll { localAddress.getHostName shouldEqual hostname localAddress.getPort shouldEqual port - val c = StreamTestKit.consumerProbe[Http.IncomingConnection] - connectionStream.produceTo(c) + val c = StreamTestKit.SubscriberProbe[Http.IncomingConnection]() + connectionStream.subscribe(c) val sub = c.expectSubscription() sub.cancel() @@ -64,14 +65,14 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll { clientOutSub.sendNext(HttpRequest(uri = "/abc") -> 'abcContext) val serverInSub = serverIn.expectSubscription() - serverInSub.requestMore(1) + serverInSub.request(1) serverIn.expectNext().uri shouldEqual Uri(s"http://$hostname:$port/abc") val serverOutSub = serverOut.expectSubscription() serverOutSub.sendNext(HttpResponse(entity = "yeah")) val clientInSub = clientIn.expectSubscription() - clientInSub.requestMore(1) + clientInSub.request(1) val (response, 'abcContext) = clientIn.expectNext() toStrict(response.entity) shouldEqual HttpEntity("yeah") } @@ -82,13 +83,13 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll { val chunks = List(Chunk("abc"), Chunk("defg"), Chunk("hijkl"), LastChunk) val chunkedContentType: ContentType = MediaTypes.`application/base64` - val chunkedEntity = HttpEntity.Chunked(chunkedContentType, SynchronousProducerFromIterable(chunks)) + val chunkedEntity = HttpEntity.Chunked(chunkedContentType, SynchronousPublisherFromIterable(chunks)) val clientOutSub = clientOut.expectSubscription() clientOutSub.sendNext(HttpRequest(POST, "/chunked", List(Accept(MediaRanges.`*/*`)), chunkedEntity) -> 12345678) val serverInSub = serverIn.expectSubscription() - serverInSub.requestMore(1) + serverInSub.request(1) private val HttpRequest(POST, uri, List(`User-Agent`(_), Host(_, _), Accept(Vector(MediaRanges.`*/*`))), Chunked(`chunkedContentType`, chunkStream), HttpProtocols.`HTTP/1.1`) = serverIn.expectNext() uri shouldEqual Uri(s"http://$hostname:$port/chunked") @@ -98,7 +99,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll { serverOutSub.sendNext(HttpResponse(206, List(RawHeader("Age", "42")), chunkedEntity)) val clientInSub = clientIn.expectSubscription() - clientInSub.requestMore(1) + clientInSub.request(1) val (HttpResponse(StatusCodes.PartialContent, List(Date(_), Server(_), RawHeader("Age", "42")), Chunked(`chunkedContentType`, chunkStream2), HttpProtocols.`HTTP/1.1`), 12345678) = clientIn.expectNext() Await.result(Flow(chunkStream2).grouped(1000).toFuture(materializer), 100.millis) shouldEqual chunks @@ -114,37 +115,37 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll { def configOverrides = "" // automatically bind a server - val connectionStream: ConsumerProbe[Http.IncomingConnection] = { + val connectionStream: SubscriberProbe[Http.IncomingConnection] = { val commander = TestProbe() val settings = configOverrides.toOption.map(ServerSettings.apply) commander.send(IO(Http), Http.Bind(hostname, port, serverSettings = settings, materializerSettings = materializerSettings)) - val probe = StreamTestKit.consumerProbe[Http.IncomingConnection] - commander.expectMsgType[Http.ServerBinding].connectionStream.produceTo(probe) + val probe = StreamTestKit.SubscriberProbe[Http.IncomingConnection] + commander.expectMsgType[Http.ServerBinding].connectionStream.subscribe(probe) probe } val connectionStreamSub = connectionStream.expectSubscription() - def openNewClientConnection[T](settings: Option[ClientConnectionSettings] = None): (ProducerProbe[(HttpRequest, T)], ConsumerProbe[(HttpResponse, T)]) = { + def openNewClientConnection[T](settings: Option[ClientConnectionSettings] = None): (PublisherProbe[(HttpRequest, T)], SubscriberProbe[(HttpResponse, T)]) = { val commander = TestProbe() commander.send(IO(Http), Http.Connect(hostname, port, settings = settings, materializerSettings = materializerSettings)) val connection = commander.expectMsgType[Http.OutgoingConnection] connection.remoteAddress.getPort shouldEqual port connection.remoteAddress.getHostName shouldEqual hostname - val requestProducerProbe = StreamTestKit.producerProbe[(HttpRequest, T)] - val responseConsumerProbe = StreamTestKit.consumerProbe[(HttpResponse, T)] - requestProducerProbe.produceTo(connection.processor[T]) - connection.processor[T].produceTo(responseConsumerProbe) - requestProducerProbe -> responseConsumerProbe + val requestPublisherProbe = StreamTestKit.PublisherProbe[(HttpRequest, T)]() + val responseSubscriberProbe = StreamTestKit.SubscriberProbe[(HttpResponse, T)]() + requestPublisherProbe.subscribe(connection.processor[T]) + connection.processor[T].subscribe(responseSubscriberProbe) + requestPublisherProbe -> responseSubscriberProbe } - def acceptConnection(): (ConsumerProbe[HttpRequest], ProducerProbe[HttpResponse]) = { - connectionStreamSub.requestMore(1) - val Http.IncomingConnection(_, requestProducer, responseConsumer) = connectionStream.expectNext() - val requestConsumerProbe = StreamTestKit.consumerProbe[HttpRequest] - val responseProducerProbe = StreamTestKit.producerProbe[HttpResponse] - requestProducer.produceTo(requestConsumerProbe) - responseProducerProbe.produceTo(responseConsumer) - requestConsumerProbe -> responseProducerProbe + def acceptConnection(): (SubscriberProbe[HttpRequest], PublisherProbe[HttpResponse]) = { + connectionStreamSub.request(1) + val Http.IncomingConnection(_, requestPublisher, responseSubscriber) = connectionStream.expectNext() + val requestSubscriberProbe = StreamTestKit.SubscriberProbe[HttpRequest]() + val responsePublisherProbe = StreamTestKit.PublisherProbe[HttpResponse]() + requestPublisher.subscribe(requestSubscriberProbe) + responsePublisherProbe.subscribe(responseSubscriber) + requestSubscriberProbe -> responsePublisherProbe } def openClientSocket() = new Socket(hostname, port) diff --git a/akka-http-core/src/test/scala/akka/http/TestServer.scala b/akka-http-core/src/test/scala/akka/http/TestServer.scala index 564d994dc9..205c8d0074 100644 --- a/akka-http-core/src/test/scala/akka/http/TestServer.scala +++ b/akka-http-core/src/test/scala/akka/http/TestServer.scala @@ -37,9 +37,9 @@ object TestServer extends App { bindingFuture foreach { case Http.ServerBinding(localAddress, connectionStream) ⇒ Flow(connectionStream).foreach { - case Http.IncomingConnection(remoteAddress, requestProducer, responseConsumer) ⇒ + case Http.IncomingConnection(remoteAddress, requestPublisher, responseSubscriber) ⇒ println("Accepted new connection from " + remoteAddress) - Flow(requestProducer).map(requestHandler).produceTo(materializer, responseConsumer) + Flow(requestPublisher).map(requestHandler).produceTo(materializer, responseSubscriber) }.consume(materializer) } diff --git a/akka-http-core/src/test/scala/akka/http/model/HttpEntitySpec.scala b/akka-http-core/src/test/scala/akka/http/model/HttpEntitySpec.scala index c66b96c40c..7ac0eb998c 100644 --- a/akka-http-core/src/test/scala/akka/http/model/HttpEntitySpec.scala +++ b/akka-http-core/src/test/scala/akka/http/model/HttpEntitySpec.scala @@ -9,10 +9,10 @@ import akka.util.ByteString import org.scalatest.matchers.Matcher import akka.stream.scaladsl.Flow import akka.http.model.HttpEntity._ -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import akka.actor.ActorSystem import akka.stream.{ MaterializerSettings, FlowMaterializer } -import akka.stream.impl.SynchronousProducerFromIterable +import akka.stream.impl.SynchronousPublisherFromIterable import com.typesafe.config.{ ConfigFactory, Config } import scala.concurrent.{ Promise, Await } import scala.concurrent.duration._ @@ -40,19 +40,19 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll { Strict(tpe, abc) must collectBytesTo(abc) } "Default" in { - Default(tpe, 11, producer(abc, de, fgh, ijk)) must + Default(tpe, 11, publisher(abc, de, fgh, ijk)) must collectBytesTo(abc, de, fgh, ijk) } "CloseDelimited" in { - CloseDelimited(tpe, producer(abc, de, fgh, ijk)) must + CloseDelimited(tpe, publisher(abc, de, fgh, ijk)) must collectBytesTo(abc, de, fgh, ijk) } "Chunked w/o LastChunk" in { - Chunked(tpe, producer(Chunk(abc), Chunk(fgh), Chunk(ijk))) must + Chunked(tpe, publisher(Chunk(abc), Chunk(fgh), Chunk(ijk))) must collectBytesTo(abc, fgh, ijk) } "Chunked with LastChunk" in { - Chunked(tpe, producer(Chunk(abc), Chunk(fgh), Chunk(ijk), LastChunk)) must + Chunked(tpe, publisher(Chunk(abc), Chunk(fgh), Chunk(ijk), LastChunk)) must collectBytesTo(abc, fgh, ijk) } } @@ -61,24 +61,24 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll { Strict(tpe, abc) must strictifyTo(Strict(tpe, abc)) } "Default" in { - Default(tpe, 11, producer(abc, de, fgh, ijk)) must + Default(tpe, 11, publisher(abc, de, fgh, ijk)) must strictifyTo(Strict(tpe, abc ++ de ++ fgh ++ ijk)) } "CloseDelimited" in { - CloseDelimited(tpe, producer(abc, de, fgh, ijk)) must + CloseDelimited(tpe, publisher(abc, de, fgh, ijk)) must strictifyTo(Strict(tpe, abc ++ de ++ fgh ++ ijk)) } "Chunked w/o LastChunk" in { - Chunked(tpe, producer(Chunk(abc), Chunk(fgh), Chunk(ijk))) must + Chunked(tpe, publisher(Chunk(abc), Chunk(fgh), Chunk(ijk))) must strictifyTo(Strict(tpe, abc ++ fgh ++ ijk)) } "Chunked with LastChunk" in { - Chunked(tpe, producer(Chunk(abc), Chunk(fgh), Chunk(ijk), LastChunk)) must + Chunked(tpe, publisher(Chunk(abc), Chunk(fgh), Chunk(ijk), LastChunk)) must strictifyTo(Strict(tpe, abc ++ fgh ++ ijk)) } "Infinite data stream" in { val neverCompleted = Promise[ByteString]() - val stream: Producer[ByteString] = Flow(neverCompleted.future).toProducer(materializer) + val stream: Publisher[ByteString] = Flow(neverCompleted.future).toPublisher(materializer) intercept[TimeoutException] { Await.result(Default(tpe, 42, stream).toStrict(100.millis, materializer), 150.millis) }.getMessage must be("HttpEntity.toStrict timed out after 100 milliseconds while still waiting for outstanding data") @@ -86,7 +86,7 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll { } } - def producer[T](elems: T*) = SynchronousProducerFromIterable(elems.toList) + def publisher[T](elems: T*) = SynchronousPublisherFromIterable(elems.toList) def collectBytesTo(bytes: ByteString*): Matcher[HttpEntity] = equal(bytes.toVector).matcher[Seq[ByteString]].compose { entity ⇒ diff --git a/akka-http-core/src/test/scala/akka/http/parsing/RequestParserSpec.scala b/akka-http-core/src/test/scala/akka/http/parsing/RequestParserSpec.scala index 0af73b257f..41c8d6c7ac 100644 --- a/akka-http-core/src/test/scala/akka/http/parsing/RequestParserSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/parsing/RequestParserSpec.scala @@ -9,9 +9,9 @@ import scala.concurrent.{ Future, Await } import scala.concurrent.duration._ import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers } import org.scalatest.matchers.Matcher -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import akka.stream.scaladsl.Flow -import akka.stream.impl.SynchronousProducerFromIterable +import akka.stream.impl.SynchronousPublisherFromIterable import akka.stream.{ FlattenStrategy, MaterializerSettings, FlowMaterializer } import akka.util.ByteString import akka.actor.ActorSystem @@ -155,7 +155,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { "request start" in new Test { Seq(start, "rest") should generalMultiParseTo( - Right(baseRequest.withEntity(HttpEntity.Chunked(`application/pdf`, producer()))), + Right(baseRequest.withEntity(HttpEntity.Chunked(`application/pdf`, publisher()))), Left(ParseError(400: StatusCode, ErrorInfo("Illegal character 'r' in chunk start")))) closeAfterResponseCompletion shouldEqual Seq(false) } @@ -174,7 +174,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { |0123456789""", """ABCDEF |dead""") should generalMultiParseTo( - Right(baseRequest.withEntity(HttpEntity.Chunked(`application/pdf`, producer( + Right(baseRequest.withEntity(HttpEntity.Chunked(`application/pdf`, publisher( HttpEntity.Chunk(ByteString("abc")), HttpEntity.Chunk(ByteString("0123456789ABCDEF"), "some=stuff;bla"), HttpEntity.Chunk(ByteString("0123456789ABCDEF"), "foo=bar"), @@ -187,7 +187,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { """0 | |""") should generalMultiParseTo( - Right(baseRequest.withEntity(HttpEntity.Chunked(`application/pdf`, producer(HttpEntity.LastChunk))))) + Right(baseRequest.withEntity(HttpEntity.Chunked(`application/pdf`, publisher(HttpEntity.LastChunk))))) closeAfterResponseCompletion shouldEqual Seq(false) } @@ -200,7 +200,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { | |GE""") should generalMultiParseTo( Right(baseRequest.withEntity(HttpEntity.Chunked(`application/pdf`, - producer(HttpEntity.LastChunk("nice=true", List(RawHeader("Bar", "xyz"), RawHeader("Foo", "pip apo")))))))) + publisher(HttpEntity.LastChunk("nice=true", List(RawHeader("Bar", "xyz"), RawHeader("Foo", "pip apo")))))))) closeAfterResponseCompletion shouldEqual Seq(false) } } @@ -214,7 +214,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { | |""" val baseRequest = HttpRequest(PATCH, "/data", List(Host("ping"), Connection("lalelu")), - HttpEntity.Chunked(`application/octet-stream`, producer())) + HttpEntity.Chunked(`application/octet-stream`, publisher())) "an illegal char after chunk size" in new Test { Seq(start, @@ -369,7 +369,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { case Right(request) ⇒ compactEntity(request.entity).map(x ⇒ Right(request.withEntity(x))) case Left(error) ⇒ Future.successful(Left(error)) } - }.toProducer(materializer) + }.toPublisher(materializer) } .flatten(FlattenStrategy.concat) .grouped(1000).toFuture(materializer) @@ -384,13 +384,13 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { case _ ⇒ entity.toStrict(250.millis, materializer) } - private def compactEntityChunks(data: Producer[ChunkStreamPart]): Future[Producer[ChunkStreamPart]] = + private def compactEntityChunks(data: Publisher[ChunkStreamPart]): Future[Publisher[ChunkStreamPart]] = Flow(data).grouped(1000).toFuture(materializer) - .map(producer(_: _*)) - .recover { case _: NoSuchElementException ⇒ producer[ChunkStreamPart]() } + .map(publisher(_: _*)) + .recover { case _: NoSuchElementException ⇒ publisher[ChunkStreamPart]() } def prep(response: String) = response.stripMarginWithNewline("\r\n") } - def producer[T](elems: T*): Producer[T] = SynchronousProducerFromIterable(elems.toList) + def publisher[T](elems: T*): Publisher[T] = SynchronousPublisherFromIterable(elems.toList) } diff --git a/akka-http-core/src/test/scala/akka/http/parsing/ResponseParserSpec.scala b/akka-http-core/src/test/scala/akka/http/parsing/ResponseParserSpec.scala index c62569928d..15064177df 100644 --- a/akka-http-core/src/test/scala/akka/http/parsing/ResponseParserSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/parsing/ResponseParserSpec.scala @@ -9,9 +9,9 @@ import scala.concurrent.{ Future, Await } import scala.concurrent.duration._ import org.scalatest.{ Tag, BeforeAndAfterAll, FreeSpec, Matchers } import org.scalatest.matchers.Matcher -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import akka.stream.scaladsl.Flow -import akka.stream.impl.SynchronousProducerFromIterable +import akka.stream.impl.SynchronousPublisherFromIterable import akka.stream.{ FlattenStrategy, MaterializerSettings, FlowMaterializer } import akka.util.ByteString import akka.actor.ActorSystem @@ -117,7 +117,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { "response start" in new Test { Seq(start, "rest") should generalMultiParseTo( - Right(baseResponse.withEntity(HttpEntity.Chunked(`application/pdf`, producer()))), + Right(baseResponse.withEntity(HttpEntity.Chunked(`application/pdf`, publisher()))), Left("Illegal character 'r' in chunk start")) closeAfterResponseCompletion shouldEqual Seq(false) } @@ -136,7 +136,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { |0123456789""", """ABCDEF |dead""") should generalMultiParseTo( - Right(baseResponse.withEntity(HttpEntity.Chunked(`application/pdf`, producer( + Right(baseResponse.withEntity(HttpEntity.Chunked(`application/pdf`, publisher( HttpEntity.Chunk(ByteString("abc")), HttpEntity.Chunk(ByteString("0123456789ABCDEF"), "some=stuff;bla"), HttpEntity.Chunk(ByteString("0123456789ABCDEF"), "foo=bar"), @@ -149,7 +149,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { """0 | |""") should generalMultiParseTo( - Right(baseResponse.withEntity(HttpEntity.Chunked(`application/pdf`, producer(HttpEntity.LastChunk))))) + Right(baseResponse.withEntity(HttpEntity.Chunked(`application/pdf`, publisher(HttpEntity.LastChunk))))) closeAfterResponseCompletion shouldEqual Seq(false) } @@ -162,7 +162,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { | |HT""") should generalMultiParseTo( Right(baseResponse.withEntity(HttpEntity.Chunked(`application/pdf`, - producer(HttpEntity.LastChunk("nice=true", List(RawHeader("Bar", "xyz"), RawHeader("Foo", "pip apo")))))))) + publisher(HttpEntity.LastChunk("nice=true", List(RawHeader("Bar", "xyz"), RawHeader("Foo", "pip apo")))))))) closeAfterResponseCompletion shouldEqual Seq(false) } } @@ -227,7 +227,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { case Right(response) ⇒ compactEntity(response.entity).map(x ⇒ Right(response.withEntity(x))) case Left(error) ⇒ Future.successful(Left(error.info.formatPretty)) } - }.toProducer(materializer) + }.toPublisher(materializer) } .flatten(FlattenStrategy.concat) .grouped(1000).toFuture(materializer) @@ -246,15 +246,15 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { case _ ⇒ entity.toStrict(250.millis, materializer) } - private def compactEntityChunks(data: Producer[ChunkStreamPart]): Future[Producer[ChunkStreamPart]] = + private def compactEntityChunks(data: Publisher[ChunkStreamPart]): Future[Publisher[ChunkStreamPart]] = Flow(data).grouped(1000).toFuture(materializer) - .map(producer(_: _*)) + .map(publisher(_: _*)) .recover { - case _: NoSuchElementException ⇒ producer[ChunkStreamPart]() + case _: NoSuchElementException ⇒ publisher[ChunkStreamPart]() } def prep(response: String) = response.stripMarginWithNewline("\r\n") - def producer[T](elems: T*): Producer[T] = SynchronousProducerFromIterable(elems.toList) + def publisher[T](elems: T*): Publisher[T] = SynchronousPublisherFromIterable(elems.toList) } } diff --git a/akka-http-core/src/test/scala/akka/http/rendering/RequestRendererSpec.scala b/akka-http-core/src/test/scala/akka/http/rendering/RequestRendererSpec.scala index c4a073ee9a..afb85976be 100644 --- a/akka-http-core/src/test/scala/akka/http/rendering/RequestRendererSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/rendering/RequestRendererSpec.scala @@ -17,7 +17,7 @@ import akka.http.model.headers._ import akka.http.util._ import akka.stream.scaladsl.Flow import akka.stream.{ MaterializerSettings, FlowMaterializer } -import akka.stream.impl.SynchronousProducerFromIterable +import akka.stream.impl.SynchronousPublisherFromIterable import HttpEntity._ import HttpMethods._ @@ -106,7 +106,7 @@ class RequestRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll "proper render a chunked" - { "PUT request with empty chunk stream and custom Content-Type" in new TestSetup() { - HttpRequest(PUT, "/abc/xyz").withEntity(Chunked(ContentTypes.`text/plain`, producer())) should renderTo { + HttpRequest(PUT, "/abc/xyz").withEntity(Chunked(ContentTypes.`text/plain`, publisher())) should renderTo { """PUT /abc/xyz HTTP/1.1 |Host: test.com:8080 |User-Agent: spray-can/1.0.0 @@ -119,7 +119,7 @@ class RequestRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll "POST request with body" in new TestSetup() { HttpRequest(POST, "/abc/xyz") - .withEntity(Chunked(ContentTypes.`text/plain`, producer("XXXX", "ABCDEFGHIJKLMNOPQRSTUVWXYZ"))) should renderTo { + .withEntity(Chunked(ContentTypes.`text/plain`, publisher("XXXX", "ABCDEFGHIJKLMNOPQRSTUVWXYZ"))) should renderTo { """POST /abc/xyz HTTP/1.1 |Host: test.com:8080 |User-Agent: spray-can/1.0.0 @@ -189,11 +189,11 @@ class RequestRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll def renderTo(expected: String): Matcher[HttpRequest] = equal(expected.stripMarginWithNewline("\r\n")).matcher[String] compose { request ⇒ val renderer = newRenderer - val byteStringProducer :: Nil = renderer.onNext(RequestRenderingContext(request, serverAddress)) - val future = Flow(byteStringProducer).grouped(1000).toFuture(materializer).map(_.reduceLeft(_ ++ _).utf8String) + val byteStringPublisher :: Nil = renderer.onNext(RequestRenderingContext(request, serverAddress)) + val future = Flow(byteStringPublisher).grouped(1000).toFuture(materializer).map(_.reduceLeft(_ ++ _).utf8String) Await.result(future, 250.millis) } } - def producer[T](elems: T*) = SynchronousProducerFromIterable(elems.toList) + def publisher[T](elems: T*) = SynchronousPublisherFromIterable(elems.toList) } diff --git a/akka-http-core/src/test/scala/akka/http/rendering/ResponseRendererSpec.scala b/akka-http-core/src/test/scala/akka/http/rendering/ResponseRendererSpec.scala index 85efe3d944..a988aa7a4a 100644 --- a/akka-http-core/src/test/scala/akka/http/rendering/ResponseRendererSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/rendering/ResponseRendererSpec.scala @@ -17,7 +17,7 @@ import akka.http.util._ import akka.util.ByteString import akka.stream.scaladsl.Flow import akka.stream.{ MaterializerSettings, FlowMaterializer } -import akka.stream.impl.SynchronousProducerFromIterable +import akka.stream.impl.SynchronousPublisherFromIterable import HttpEntity._ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll { @@ -113,7 +113,7 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll "a response with a Default (streamed with explicit content-length body" - { "with status 400, a few headers and a body" in new TestSetup() { HttpResponse(400, List(RawHeader("Age", "30"), Connection("Keep-Alive")), - entity = Default(contentType = ContentTypes.`text/plain(UTF-8)`, 23, producer(ByteString("Small f*ck up overhere!")))) should renderTo { + entity = Default(contentType = ContentTypes.`text/plain(UTF-8)`, 23, publisher(ByteString("Small f*ck up overhere!")))) should renderTo { """HTTP/1.1 400 Bad Request |Age: 30 |Server: akka-http/1.0.0 @@ -127,14 +127,14 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll "with one chunk and incorrect (too large) Content-Length" in new TestSetup() { the[RuntimeException] thrownBy { HttpResponse(200, entity = Default(ContentTypes.`application/json`, 10, - producer(ByteString("body123")))) should renderTo("") + publisher(ByteString("body123")))) should renderTo("") } should have message "HTTP message had declared Content-Length 10 but entity chunk stream amounts to 3 bytes less" } "with one chunk and incorrect (too small) Content-Length" in new TestSetup() { the[RuntimeException] thrownBy { HttpResponse(200, entity = Default(ContentTypes.`application/json`, 5, - producer(ByteString("body123")))) should renderTo("") + publisher(ByteString("body123")))) should renderTo("") } should have message "HTTP message had declared Content-Length 5 but entity chunk stream amounts to more bytes" } @@ -143,7 +143,7 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll "without data" in new TestSetup() { ResponseRenderingContext( HttpResponse(200, entity = CloseDelimited(ContentTypes.`application/json`, - producer(ByteString.empty)))) should renderTo( + publisher(ByteString.empty)))) should renderTo( """HTTP/1.1 200 OK |Server: akka-http/1.0.0 |Date: Thu, 25 Aug 2011 09:10:29 GMT @@ -155,7 +155,7 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll "consisting of two parts" in new TestSetup() { ResponseRenderingContext( HttpResponse(200, entity = CloseDelimited(ContentTypes.`application/json`, - producer(ByteString("abc"), ByteString("defg"))))) should renderTo( + publisher(ByteString("abc"), ByteString("defg"))))) should renderTo( """HTTP/1.1 200 OK |Server: akka-http/1.0.0 |Date: Thu, 25 Aug 2011 09:10:29 GMT @@ -169,7 +169,7 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll "a chunked response" - { "with empty entity" in new TestSetup() { HttpResponse(200, List(RawHeader("Age", "30")), - Chunked(ContentTypes.NoContentType, producer())) should renderTo { + Chunked(ContentTypes.NoContentType, publisher())) should renderTo { """HTTP/1.1 200 OK |Age: 30 |Server: akka-http/1.0.0 @@ -181,7 +181,7 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll "with empty entity but non-default Content-Type" in new TestSetup() { HttpResponse(200, List(RawHeader("Age", "30")), - Chunked(ContentTypes.`application/json`, producer())) should renderTo { + Chunked(ContentTypes.`application/json`, publisher())) should renderTo { """HTTP/1.1 200 OK |Age: 30 |Server: akka-http/1.0.0 @@ -194,7 +194,7 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll "with one chunk and no explicit LastChunk" in new TestSetup() { HttpResponse(entity = Chunked(ContentTypes.`text/plain(UTF-8)`, - producer("Yahoooo"))) should renderTo { + publisher("Yahoooo"))) should renderTo { """HTTP/1.1 200 OK |Server: akka-http/1.0.0 |Date: Thu, 25 Aug 2011 09:10:29 GMT @@ -211,7 +211,7 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll "with one chunk and an explicit LastChunk" in new TestSetup() { HttpResponse(entity = Chunked(ContentTypes.`text/plain(UTF-8)`, - producer(Chunk(ByteString("body123"), """key=value;another="tl;dr""""), + publisher(Chunk(ByteString("body123"), """key=value;another="tl;dr""""), LastChunk("foo=bar", List(RawHeader("Age", "30"), RawHeader("Cache-Control", "public")))))) should renderTo { """HTTP/1.1 200 OK |Server: akka-http/1.0.0 @@ -235,7 +235,7 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll ResponseRenderingContext( requestProtocol = HttpProtocols.`HTTP/1.0`, response = HttpResponse(entity = Chunked(ContentTypes.`application/json`, - producer(Chunk("abc"), Chunk("defg"))))) should renderTo( + publisher(Chunk("abc"), Chunk("defg"))))) should renderTo( """HTTP/1.1 200 OK |Server: akka-http/1.0.0 |Date: Thu, 25 Aug 2011 09:10:29 GMT @@ -248,7 +248,7 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll ResponseRenderingContext( requestProtocol = HttpProtocols.`HTTP/1.0`, response = HttpResponse(entity = Chunked(ContentTypes.`text/plain(UTF-8)`, - producer(Chunk(ByteString("body123"), """key=value;another="tl;dr""""), + publisher(Chunk(ByteString("body123"), """key=value;another="tl;dr""""), LastChunk("foo=bar", List(RawHeader("Age", "30"), RawHeader("Cache-Control", "public"))))))) should renderTo( """HTTP/1.1 200 OK |Server: akka-http/1.0.0 @@ -336,13 +336,13 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll def renderTo(expected: String, close: Boolean): Matcher[ResponseRenderingContext] = equal(expected.stripMarginWithNewline("\r\n") -> close).matcher[(String, Boolean)] compose { ctx ⇒ val renderer = newRenderer - val byteStringProducer :: Nil = renderer.onNext(ctx) - val future = Flow(byteStringProducer).grouped(1000).toFuture(materializer).map(_.reduceLeft(_ ++ _).utf8String) + val byteStringPublisher :: Nil = renderer.onNext(ctx) + val future = Flow(byteStringPublisher).grouped(1000).toFuture(materializer).map(_.reduceLeft(_ ++ _).utf8String) Await.result(future, 250.millis) -> renderer.isComplete } override def dateTime(now: Long) = DateTime(2011, 8, 25, 9, 10, 29) // provide a stable date for testing } - def producer[T](elems: T*) = SynchronousProducerFromIterable(elems.toList) + def publisher[T](elems: T*) = SynchronousPublisherFromIterable(elems.toList) } diff --git a/akka-stream/src/main/scala/akka/persistence/stream/PersistentPublisher.scala b/akka-stream/src/main/scala/akka/persistence/stream/PersistentPublisher.scala index cc07a3996e..27e0e541ba 100644 --- a/akka-stream/src/main/scala/akka/persistence/stream/PersistentPublisher.scala +++ b/akka-stream/src/main/scala/akka/persistence/stream/PersistentPublisher.scala @@ -6,20 +6,17 @@ package akka.persistence.stream import scala.util.control.NonFatal import scala.concurrent.duration._ -import org.reactivestreams.api.Producer -import org.reactivestreams.spi.Subscriber +import org.reactivestreams.{ Publisher, Subscriber } import akka.actor._ import akka.persistence._ import akka.stream._ import akka.stream.impl._ -import akka.stream.impl.Ast.ProducerNode +import akka.stream.impl.Ast.PublisherNode import akka.stream.scaladsl.Flow // ------------------------------------------------------------------------------------------------ // FIXME: move this file to akka-persistence-experimental once going back to project dependencies -// NOTE: "producer" has been changed to "publisher" wherever possible, covering the upcoming -// changes in reactive-streams. // ------------------------------------------------------------------------------------------------ object PersistentFlow { @@ -73,9 +70,9 @@ private object PersistentPublisher { Props(classOf[PersistentPublisherImpl], processorId, publisherSettings, settings).withDispatcher(settings.dispatcher) } -private case class PersistentPublisherNode(processorId: String, publisherSettings: PersistentPublisherSettings) extends ProducerNode[Persistent] { - def createProducer(materializer: ActorBasedFlowMaterializer, flowName: String): Producer[Persistent] = - new ActorProducer(materializer.context.actorOf(PersistentPublisher.props(processorId, publisherSettings, materializer.settings), +private case class PersistentPublisherNode(processorId: String, publisherSettings: PersistentPublisherSettings) extends PublisherNode[Persistent] { + def createPublisher(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[Persistent] = + ActorPublisher[Persistent](materializer.context.actorOf(PersistentPublisher.props(processorId, publisherSettings, materializer.settings), name = s"$flowName-0-persistentPublisher")) } diff --git a/akka-stream/src/main/scala/akka/stream/FlattenStrategy.scala b/akka-stream/src/main/scala/akka/stream/FlattenStrategy.scala index 0ec792f31d..d4701e01af 100644 --- a/akka-stream/src/main/scala/akka/stream/FlattenStrategy.scala +++ b/akka-stream/src/main/scala/akka/stream/FlattenStrategy.scala @@ -3,7 +3,7 @@ */ package akka.stream -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher /** * Strategy that defines how a stream of streams should be flattened into a stream of simple elements. @@ -17,7 +17,7 @@ object FlattenStrategy { * emitting its elements directly to the output until it completes and then taking the next stream. This has the * consequence that if one of the input stream is infinite, no other streams after that will be consumed from. */ - def concat[T]: FlattenStrategy[Producer[T], T] = Concat[T]() + def concat[T]: FlattenStrategy[Publisher[T], T] = Concat[T]() - private[akka] case class Concat[T]() extends FlattenStrategy[Producer[T], T] + private[akka] case class Concat[T]() extends FlattenStrategy[Publisher[T], T] } \ No newline at end of file diff --git a/akka-stream/src/main/scala/akka/stream/FlowMaterializer.scala b/akka-stream/src/main/scala/akka/stream/FlowMaterializer.scala index 079d2137f4..07169abba6 100644 --- a/akka-stream/src/main/scala/akka/stream/FlowMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/FlowMaterializer.scala @@ -7,9 +7,8 @@ import scala.concurrent.duration.FiniteDuration import akka.actor.ActorRefFactory import akka.stream.impl.ActorBasedFlowMaterializer import akka.stream.impl.Ast -import org.reactivestreams.api.Producer +import org.reactivestreams.{ Publisher, Subscriber } import scala.concurrent.duration._ -import org.reactivestreams.api.Consumer import akka.actor.Deploy object FlowMaterializer { @@ -42,7 +41,7 @@ object FlowMaterializer { /** * A FlowMaterializer takes the list of transformations comprising a * [[akka.stream.scaladsl.Flow]] and materializes them in the form of - * [[org.reactivestreams.api.Processor]] instances. How transformation + * [[org.reactivestreams.Processor]] instances. How transformation * steps are split up into asynchronous regions is implementation * dependent. */ @@ -58,17 +57,17 @@ abstract class FlowMaterializer(val settings: MaterializerSettings) { * INTERNAL API * ops are stored in reverse order */ - private[akka] def toProducer[I, O](producerNode: Ast.ProducerNode[I], ops: List[Ast.AstNode]): Producer[O] + private[akka] def toPublisher[I, O](publisherNode: Ast.PublisherNode[I], ops: List[Ast.AstNode]): Publisher[O] /** * INTERNAL API */ - private[akka] def ductProduceTo[In, Out](consumer: Consumer[Out], ops: List[Ast.AstNode]): Consumer[In] + private[akka] def ductProduceTo[In, Out](subscriber: Subscriber[Out], ops: List[Ast.AstNode]): Subscriber[In] /** * INTERNAL API */ - private[akka] def ductBuild[In, Out](ops: List[Ast.AstNode]): (Consumer[In], Producer[Out]) + private[akka] def ductBuild[In, Out](ops: List[Ast.AstNode]): (Subscriber[In], Publisher[Out]) } diff --git a/akka-stream/src/main/scala/akka/stream/OverflowStrategy.scala b/akka-stream/src/main/scala/akka/stream/OverflowStrategy.scala index 4548ff4949..22d277913f 100644 --- a/akka-stream/src/main/scala/akka/stream/OverflowStrategy.scala +++ b/akka-stream/src/main/scala/akka/stream/OverflowStrategy.scala @@ -48,7 +48,7 @@ object OverflowStrategy { def dropBuffer: OverflowStrategy = DropBuffer /** - * If the buffer is full when a new element is available this strategy backpressures the upstream producer until + * If the buffer is full when a new element is available this strategy backpressures the upstream publisher until * space becomes available in the buffer. */ def backpressure: OverflowStrategy = Backpressure diff --git a/akka-stream/src/main/scala/akka/stream/Support.scala b/akka-stream/src/main/scala/akka/stream/Support.scala index 5dfdf97be9..8255da8580 100644 --- a/akka-stream/src/main/scala/akka/stream/Support.scala +++ b/akka-stream/src/main/scala/akka/stream/Support.scala @@ -6,7 +6,7 @@ package akka.stream import scala.util.control.NoStackTrace /** - * This exception must be thrown from a callback-based stream producer to + * This exception must be thrown from a callback-based stream publisher to * signal the end of stream (if the produced stream is not infinite). This is used for example in * [[akka.stream.scaladsl.Flow#apply]] (the variant which takes a closure). */ diff --git a/akka-stream/src/main/scala/akka/stream/Transformer.scala b/akka-stream/src/main/scala/akka/stream/Transformer.scala index 688cd51423..13e26b041c 100644 --- a/akka-stream/src/main/scala/akka/stream/Transformer.scala +++ b/akka-stream/src/main/scala/akka/stream/Transformer.scala @@ -25,7 +25,7 @@ abstract class Transformer[-T, +U] { /** * Invoked after handing off the elements produced from one input element to the - * downstream consumers to determine whether to end stream processing at this point; + * downstream subscribers to determine whether to end stream processing at this point; * in that case the upstream subscription is canceled. */ def isComplete: Boolean = false diff --git a/akka-stream/src/main/scala/akka/stream/actor/ActorProducer.scala b/akka-stream/src/main/scala/akka/stream/actor/ActorPublisher.scala similarity index 73% rename from akka-stream/src/main/scala/akka/stream/actor/ActorProducer.scala rename to akka-stream/src/main/scala/akka/stream/actor/ActorPublisher.scala index 005c221beb..c973caadd2 100644 --- a/akka-stream/src/main/scala/akka/stream/actor/ActorProducer.scala +++ b/akka-stream/src/main/scala/akka/stream/actor/ActorPublisher.scala @@ -4,11 +4,7 @@ package akka.stream.actor import java.util.concurrent.ConcurrentHashMap -import org.reactivestreams.api.Consumer -import org.reactivestreams.api.Producer -import org.reactivestreams.spi.Publisher -import org.reactivestreams.spi.Subscriber -import org.reactivestreams.spi.Subscription +import org.reactivestreams.{ Publisher, Subscriber, Subscription } import akka.actor.Actor import akka.actor.ActorRef import akka.actor.ActorSystem @@ -17,23 +13,23 @@ import akka.actor.Extension import akka.actor.ExtensionId import akka.actor.ExtensionIdProvider -object ActorProducer { +object ActorPublisher { /** - * Create a [[org.reactivestreams.api.Producer]] backed by a [[ActorProducer]] actor. It can be - * attached to a [[org.reactivestreams.api.Consumer]] or be used as an input source for a + * Create a [[org.reactivestreams.Publisher]] backed by a [[ActorPublisher]] actor. It can be + * attached to a [[org.reactivestreams.Subscriber]] or be used as an input source for a * [[akka.stream.Flow]]. */ - def apply[T](ref: ActorRef): Producer[T] = ActorProducerImpl(ref) + def apply[T](ref: ActorRef): Publisher[T] = ActorPublisherImpl(ref) /** - * This message is delivered to the [[ActorProducer]] actor when the stream consumer requests + * This message is delivered to the [[ActorPublisher]] actor when the stream subscriber requests * more elements. */ @SerialVersionUID(1L) case class Request(elements: Int) /** - * This message is delivered to the [[ActorProducer]] actor when the stream consumer cancels the + * This message is delivered to the [[ActorPublisher]] actor when the stream subscriber cancels the * subscription. */ @SerialVersionUID(1L) case object Cancel @@ -55,26 +51,26 @@ object ActorProducer { /** * Extend/mixin this trait in your [[akka.actor.Actor]] to make it a - * stream producer that keeps track of the subscription life cycle and + * stream publisher that keeps track of the subscription life cycle and * requested elements. * - * Create a [[org.reactivestreams.api.Producer]] backed by this actor with [[ActorProducer#apply]]. - * It can be attached to a [[org.reactivestreams.api.Consumer]] or be used as an input source for a - * [[akka.stream.Flow]]. You can only attach one subscriber to this producer. + * Create a [[org.reactivestreams.Publisher]] backed by this actor with [[ActorPublisher#apply]]. + * It can be attached to a [[org.reactivestreams.Subscriber]] or be used as an input source for a + * [[akka.stream.Flow]]. You can only attach one subscriber to this publisher. * * The life cycle state of the subscription is tracked with the following boolean members: * [[#isActive]], [[#isCompleted]], [[#isErrorEmitted]], and [[#isCanceled]]. * * You send elements to the stream by calling [[#onNext]]. You are allowed to send as many - * elements as have been requested by the stream consumer. This amount can be inquired with + * elements as have been requested by the stream subscriber. This amount can be inquired with * [[#totalDemand]]. It is only allowed to use `onNext` when `isActive` and `totalDemand > 0`, * otherwise `onNext` will throw `IllegalStateException`. * - * When the stream consumer requests more elements the [[ActorProducer#Request]] message + * When the stream subscriber requests more elements the [[ActorPublisher#Request]] message * is delivered to this actor, and you can act on that event. The [[#totalDemand]] * is updated automatically. * - * When the stream consumer cancels the subscription the [[ActorProducer#Cancel]] message + * When the stream subscriber cancels the subscription the [[ActorPublisher#Cancel]] message * is delivered to this actor. After that subsequent calls to `onNext` will be ignored. * * You can complete the stream by calling [[#onComplete]]. After that you are not allowed to @@ -86,17 +82,17 @@ object ActorProducer { * If the actor is stopped the stream will be completed, unless it was not already terminated with * failure, completed or canceled. */ -trait ActorProducer[T] extends Actor { - import ActorProducer._ - import ActorProducer.Internal._ +trait ActorPublisher[T] extends Actor { + import ActorPublisher._ + import ActorPublisher.Internal._ - private val state = ActorProducerState(context.system) + private val state = ActorPublisherState(context.system) private var subscriber: Subscriber[Any] = _ private var demand = 0L private var lifecycleState: LifecycleState = PreSubscriber /** - * The state when the producer is active, i.e. before the subscriber is attached + * The state when the publisher is active, i.e. before the subscriber is attached * and when an subscriber is attached. It is allowed to * call [[#onComplete]] and [[#onError]] in this state. It is * allowed to call [[#onNext]] in this state when [[#totalDemand]] @@ -105,7 +101,7 @@ trait ActorProducer[T] extends Actor { final def isActive = lifecycleState == Active || lifecycleState == PreSubscriber /** - * Total number of requested elements from the stream consumer. + * Total number of requested elements from the stream subscriber. * This actor automatically keeps tracks of this amount based on * incoming request messages and outgoing `onNext`. */ @@ -128,15 +124,15 @@ trait ActorProducer[T] extends Actor { final def isErrorEmitted: Boolean = lifecycleState.isInstanceOf[ErrorEmitted] /** - * The state after the stream consumer has canceled the subscription. + * The state after the stream subscriber has canceled the subscription. * It is allowed to call [[#onNext]], [[#onError]], and [[#onComplete]] in * this state, but the calls will not perform anything. */ final def isCanceled: Boolean = lifecycleState == Canceled /** - * Send an element to the stream consumer. You are allowed to send as many elements - * as have been requested by the stream consumer. This amount can be inquired with + * Send an element to the stream subscriber. You are allowed to send as many elements + * as have been requested by the stream subscriber. This amount can be inquired with * [[#totalDemand]]. It is only allowed to use `onNext` when `isActive` and `totalDemand > 0`, * otherwise `onNext` will throw `IllegalStateException`. */ @@ -199,11 +195,11 @@ trait ActorProducer[T] extends Actor { case PreSubscriber ⇒ subscriber = sub lifecycleState = Active - sub.onSubscribe(new ActorProducerSubscription(self)) + sub.onSubscribe(new ActorPublisherSubscription(self)) case ErrorEmitted(cause) ⇒ sub.onError(cause) case Completed ⇒ sub.onComplete() case Active | Canceled ⇒ - sub.onError(new IllegalStateException(s"ActorProducer [$self] can only have one subscriber")) + sub.onError(new IllegalStateException(s"ActorPublisher [$self] can only have one subscriber")) } case Cancel ⇒ @@ -217,7 +213,7 @@ trait ActorProducer[T] extends Actor { protected[akka] override def aroundPreRestart(reason: Throwable, message: Option[Any]): Unit = { // some state must survive restart - state.set(self, ActorProducerState.State(Option(subscriber), demand, lifecycleState)) + state.set(self, ActorPublisherState.State(Option(subscriber), demand, lifecycleState)) super.aroundPreRestart(reason, message) } @@ -243,25 +239,20 @@ trait ActorProducer[T] extends Actor { /** * INTERNAL API */ -private[akka] case class ActorProducerImpl[T](ref: ActorRef) extends Producer[T] with Publisher[T] { - import ActorProducer._ - import ActorProducer.Internal._ - - override def getPublisher: Publisher[T] = this +private[akka] case class ActorPublisherImpl[T](ref: ActorRef) extends Publisher[T] { + import ActorPublisher._ + import ActorPublisher.Internal._ override def subscribe(sub: Subscriber[T]): Unit = ref ! Subscribe(sub.asInstanceOf[Subscriber[Any]]) - - override def produceTo(consumer: Consumer[T]): Unit = - getPublisher.subscribe(consumer.getSubscriber()) } /** * INTERNAL API */ -private[akka] class ActorProducerSubscription[T](ref: ActorRef) extends Subscription { - import ActorProducer._ - override def requestMore(elements: Int): Unit = +private[akka] class ActorPublisherSubscription[T](ref: ActorRef) extends Subscription { + import ActorPublisher._ + override def request(elements: Int): Unit = if (elements <= 0) throw new IllegalArgumentException("The number of requested elements must be > 0") else ref ! Request(elements) override def cancel(): Unit = ref ! Cancel @@ -271,15 +262,15 @@ private[akka] class ActorProducerSubscription[T](ref: ActorRef) extends Subscrip * INTERNAL API * Some state must survive restarts. */ -private[akka] object ActorProducerState extends ExtensionId[ActorProducerState] with ExtensionIdProvider { - import ActorProducer.Internal.LifecycleState +private[akka] object ActorPublisherState extends ExtensionId[ActorPublisherState] with ExtensionIdProvider { + import ActorPublisher.Internal.LifecycleState - override def get(system: ActorSystem): ActorProducerState = super.get(system) + override def get(system: ActorSystem): ActorPublisherState = super.get(system) - override def lookup = ActorProducerState + override def lookup = ActorPublisherState - override def createExtension(system: ExtendedActorSystem): ActorProducerState = - new ActorProducerState + override def createExtension(system: ExtendedActorSystem): ActorPublisherState = + new ActorPublisherState case class State(subscriber: Option[Subscriber[Any]], demand: Long, lifecycleState: LifecycleState) @@ -288,8 +279,8 @@ private[akka] object ActorProducerState extends ExtensionId[ActorProducerState] /** * INTERNAL API */ -private[akka] class ActorProducerState extends Extension { - import ActorProducerState.State +private[akka] class ActorPublisherState extends Extension { + import ActorPublisherState.State private val state = new ConcurrentHashMap[ActorRef, State] def get(ref: ActorRef): Option[State] = Option(state.get(ref)) diff --git a/akka-stream/src/main/scala/akka/stream/actor/ActorConsumer.scala b/akka-stream/src/main/scala/akka/stream/actor/ActorSubscriber.scala similarity index 72% rename from akka-stream/src/main/scala/akka/stream/actor/ActorConsumer.scala rename to akka-stream/src/main/scala/akka/stream/actor/ActorSubscriber.scala index 9bd39fd0f8..f35ba15de5 100644 --- a/akka-stream/src/main/scala/akka/stream/actor/ActorConsumer.scala +++ b/akka-stream/src/main/scala/akka/stream/actor/ActorSubscriber.scala @@ -4,9 +4,7 @@ package akka.stream.actor import java.util.concurrent.ConcurrentHashMap -import org.reactivestreams.api.Consumer -import org.reactivestreams.spi.Subscriber -import org.reactivestreams.spi.Subscription +import org.reactivestreams.{ Publisher, Subscriber, Subscription } import akka.actor.Actor import akka.actor.ActorRef import akka.actor.ActorSystem @@ -15,19 +13,19 @@ import akka.actor.Extension import akka.actor.ExtensionId import akka.actor.ExtensionIdProvider -object ActorConsumer { +object ActorSubscriber { /** - * Attach a [[ActorConsumer]] actor as a [[org.reactivestreams.api.Consumer]] - * to a [[org.reactivestreams.api.Producer]] or [[akka.stream.Flow]]. + * Attach a [[ActorSubscriber]] actor as a [[org.reactivestreams.Subscriber]] + * to a [[org.reactivestreams.Publisher]] or [[akka.stream.Flow]]. */ - def apply[T](ref: ActorRef): Consumer[T] = ActorConsumerImpl(ref) + def apply[T](ref: ActorRef): Subscriber[T] = new ActorSubscriberImpl(ref) /** - * Java API: Attach a [[ActorConsumer]] actor as a [[org.reactivestreams.api.Consumer]] - * to a [[org.reactivestreams.api.Producer]] or [[akka.stream.Flow]]. + * Java API: Attach a [[ActorSubscriber]] actor as a [[org.reactivestreams.Subscriber]] + * to a [[org.reactivestreams.Publisher]] or [[akka.stream.Flow]]. */ - def create[T](ref: ActorRef): Consumer[T] = apply(ref) + def create[T](ref: ActorRef): Subscriber[T] = apply(ref) @SerialVersionUID(1L) case class OnNext(element: Any) @SerialVersionUID(1L) case object OnComplete @@ -39,11 +37,11 @@ object ActorConsumer { @SerialVersionUID(1L) private[akka] case class OnSubscribe(subscription: Subscription) /** - * An [[ActorConsumer]] defines a `RequestStrategy` to control the stream back pressure. + * An [[ActorSubscriber]] defines a `RequestStrategy` to control the stream back pressure. */ trait RequestStrategy { /** - * Invoked by the [[ActorConsumer]] after each incoming message to + * Invoked by the [[ActorSubscriber]] after each incoming message to * determine how many more elements to request from the stream. * * @param remainingRequested current remaining number of elements that @@ -65,7 +63,7 @@ object ActorConsumer { /** * When request is only controlled with manual calls to - * [[ActorConsumer#request]]. + * [[ActorSubscriber#request]]. */ case object ZeroRequestStrategy extends RequestStrategy { def requestDemand(remainingRequested: Int): Int = 0 @@ -122,30 +120,30 @@ object ActorConsumer { /** * Extend/mixin this trait in your [[akka.actor.Actor]] to make it a - * stream consumer with full control of stream back pressure. It will receive - * [[ActorConsumer.OnNext]], [[ActorConsumer.OnComplete]] and [[ActorConsumer.OnError]] + * stream subscriber with full control of stream back pressure. It will receive + * [[ActorSubscriber.OnNext]], [[ActorSubscriber.OnComplete]] and [[ActorSubscriber.OnError]] * messages from the stream. It can also receive other, non-stream messages, in * the same way as any actor. * - * Attach the actor as a [[org.reactivestreams.api.Consumer]] to the stream with - * [[ActorConsumer#apply]]. + * Attach the actor as a [[org.reactivestreams.Subscriber]] to the stream with + * [[ActorSubscriber#apply]]. * * Subclass must define the [[RequestStrategy]] to control stream back pressure. - * After each incoming message the `ActorConsumer` will automatically invoke + * After each incoming message the `ActorSubscriber` will automatically invoke * the [[RequestStrategy#requestDemand]] and propagate the returned demand to the stream. - * The provided [[ActorConsumer.WatermarkRequestStrategy]] is a good strategy if the actor + * The provided [[ActorSubscriber.WatermarkRequestStrategy]] is a good strategy if the actor * performs work itself. - * The provided [[ActorConsumer.MaxInFlightRequestStrategy]] is useful if messages are + * The provided [[ActorSubscriber.MaxInFlightRequestStrategy]] is useful if messages are * queued internally or delegated to other actors. * You can also implement a custom [[RequestStrategy]] or call [[#request]] manually - * together with [[ActorConsumer.ZeroRequestStrategy]] or some other strategy. In that case + * together with [[ActorSubscriber.ZeroRequestStrategy]] or some other strategy. In that case * you must also call [[#request]] when the actor is started or when it is ready, otherwise * it will not receive any elements. */ -trait ActorConsumer extends Actor { - import ActorConsumer._ +trait ActorSubscriber extends Actor { + import ActorSubscriber._ - private val state = ActorConsumerState(context.system) + private val state = ActorSubscriberState(context.system) private var subscription: Option[Subscription] = None private var requested = 0L private var canceled = false @@ -165,7 +163,7 @@ trait ActorConsumer extends Actor { if (canceled) sub.cancel() else if (requested != 0) - sub.requestMore(remainingRequested) + sub.request(remainingRequested) } else sub.cancel() case _: OnError ⇒ @@ -194,7 +192,7 @@ trait ActorConsumer extends Actor { protected[akka] override def aroundPreRestart(reason: Throwable, message: Option[Any]): Unit = { // some state must survive restart - state.set(self, ActorConsumerState.State(subscription, requested, canceled)) + state.set(self, ActorSubscriberState.State(subscription, requested, canceled)) super.aroundPreRestart(reason, message) } @@ -210,7 +208,7 @@ trait ActorConsumer extends Actor { protected def request(elements: Int): Unit = if (elements > 0 && !canceled) { // if we don't have a subscription yet, it will be requested when it arrives - subscription.foreach(_.requestMore(elements)) + subscription.foreach(_.request(elements)) requested += elements } @@ -233,31 +231,24 @@ trait ActorConsumer extends Actor { /** * INTERNAL API */ -private[akka] case class ActorConsumerImpl[T](ref: ActorRef) extends Consumer[T] { - override val getSubscriber: Subscriber[T] = new ActorSubscriber[T](ref) -} - -/** - * INTERNAL API - */ -private[akka] final class ActorSubscriber[T](val impl: ActorRef) extends Subscriber[T] { - override def onError(cause: Throwable): Unit = impl ! ActorConsumer.OnError(cause) - override def onComplete(): Unit = impl ! ActorConsumer.OnComplete - override def onNext(element: T): Unit = impl ! ActorConsumer.OnNext(element) - override def onSubscribe(subscription: Subscription): Unit = impl ! ActorConsumer.OnSubscribe(subscription) +private[akka] final class ActorSubscriberImpl[T](val impl: ActorRef) extends Subscriber[T] { + override def onError(cause: Throwable): Unit = impl ! ActorSubscriber.OnError(cause) + override def onComplete(): Unit = impl ! ActorSubscriber.OnComplete + override def onNext(element: T): Unit = impl ! ActorSubscriber.OnNext(element) + override def onSubscribe(subscription: Subscription): Unit = impl ! ActorSubscriber.OnSubscribe(subscription) } /** * INTERNAL API * Some state must survive restarts. */ -private[akka] object ActorConsumerState extends ExtensionId[ActorConsumerState] with ExtensionIdProvider { - override def get(system: ActorSystem): ActorConsumerState = super.get(system) +private[akka] object ActorSubscriberState extends ExtensionId[ActorSubscriberState] with ExtensionIdProvider { + override def get(system: ActorSystem): ActorSubscriberState = super.get(system) - override def lookup = ActorConsumerState + override def lookup = ActorSubscriberState - override def createExtension(system: ExtendedActorSystem): ActorConsumerState = - new ActorConsumerState + override def createExtension(system: ExtendedActorSystem): ActorSubscriberState = + new ActorSubscriberState case class State(subscription: Option[Subscription], requested: Long, canceled: Boolean) @@ -266,8 +257,8 @@ private[akka] object ActorConsumerState extends ExtensionId[ActorConsumerState] /** * INTERNAL API */ -private[akka] class ActorConsumerState extends Extension { - import ActorConsumerState.State +private[akka] class ActorSubscriberState extends Extension { + import ActorSubscriberState.State private val state = new ConcurrentHashMap[ActorRef, State] def get(ref: ActorRef): Option[State] = Option(state.get(ref)) diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorBasedFlowMaterializer.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorBasedFlowMaterializer.scala index 46899803e5..d28bd373da 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/ActorBasedFlowMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/ActorBasedFlowMaterializer.scala @@ -5,8 +5,7 @@ package akka.stream.impl import scala.annotation.tailrec import scala.collection.immutable -import org.reactivestreams.api.{ Consumer, Processor, Producer } -import org.reactivestreams.spi.Subscriber +import org.reactivestreams.{ Publisher, Subscriber, Processor } import akka.actor.ActorRefFactory import akka.stream.{ OverflowStrategy, MaterializerSettings, FlowMaterializer, Transformer } import scala.util.Try @@ -20,7 +19,6 @@ import akka.actor.ExtensionId import akka.actor.ExtendedActorSystem import akka.actor.ActorSystem import akka.actor.Extension -import akka.stream.actor.ActorConsumer import scala.concurrent.duration.FiniteDuration import akka.stream.TimerTransformer @@ -44,16 +42,16 @@ private[akka] object Ast { case class SplitWhen(p: Any ⇒ Boolean) extends AstNode { override def name = "splitWhen" } - case class Merge(other: Producer[Any]) extends AstNode { + case class Merge(other: Publisher[Any]) extends AstNode { override def name = "merge" } - case class Zip(other: Producer[Any]) extends AstNode { + case class Zip(other: Publisher[Any]) extends AstNode { override def name = "zip" } - case class Concat(next: Producer[Any]) extends AstNode { + case class Concat(next: Publisher[Any]) extends AstNode { override def name = "concat" } - case class Tee(other: Consumer[Any]) extends AstNode { + case class Tee(other: Subscriber[Any]) extends AstNode { override def name = "tee" } case class PrefixAndTail(n: Int) extends AstNode { @@ -75,47 +73,47 @@ private[akka] object Ast { override def name = "buffer" } - trait ProducerNode[I] { - private[akka] def createProducer(materializer: ActorBasedFlowMaterializer, flowName: String): Producer[I] + trait PublisherNode[I] { + private[akka] def createPublisher(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[I] } - final case class ExistingProducer[I](producer: Producer[I]) extends ProducerNode[I] { - def createProducer(materializer: ActorBasedFlowMaterializer, flowName: String) = producer + final case class ExistingPublisher[I](publisher: Publisher[I]) extends PublisherNode[I] { + def createPublisher(materializer: ActorBasedFlowMaterializer, flowName: String) = publisher } - final case class IteratorProducerNode[I](iterator: Iterator[I]) extends ProducerNode[I] { - final def createProducer(materializer: ActorBasedFlowMaterializer, flowName: String): Producer[I] = - if (iterator.isEmpty) EmptyProducer.asInstanceOf[Producer[I]] - else new ActorProducer[I](materializer.context.actorOf(IteratorProducer.props(iterator, materializer.settings), + final case class IteratorPublisherNode[I](iterator: Iterator[I]) extends PublisherNode[I] { + final def createPublisher(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[I] = + if (iterator.isEmpty) EmptyPublisher.asInstanceOf[Publisher[I]] + else ActorPublisher[I](materializer.context.actorOf(IteratorPublisher.props(iterator, materializer.settings), name = s"$flowName-0-iterator")) } - final case class IterableProducerNode[I](iterable: immutable.Iterable[I]) extends ProducerNode[I] { - def createProducer(materializer: ActorBasedFlowMaterializer, flowName: String): Producer[I] = - if (iterable.isEmpty) EmptyProducer.asInstanceOf[Producer[I]] - else new ActorProducer[I](materializer.context.actorOf(IterableProducer.props(iterable, materializer.settings), + final case class IterablePublisherNode[I](iterable: immutable.Iterable[I]) extends PublisherNode[I] { + def createPublisher(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[I] = + if (iterable.isEmpty) EmptyPublisher.asInstanceOf[Publisher[I]] + else ActorPublisher[I](materializer.context.actorOf(IterablePublisher.props(iterable, materializer.settings), name = s"$flowName-0-iterable"), Some(iterable)) } - final case class ThunkProducerNode[I](f: () ⇒ I) extends ProducerNode[I] { - def createProducer(materializer: ActorBasedFlowMaterializer, flowName: String): Producer[I] = - new ActorProducer(materializer.context.actorOf(ActorProducer.props(materializer.settings, f), + final case class ThunkPublisherNode[I](f: () ⇒ I) extends PublisherNode[I] { + def createPublisher(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[I] = + ActorPublisher[I](materializer.context.actorOf(SimpleCallbackPublisher.props(materializer.settings, f), name = s"$flowName-0-thunk")) } - final case class FutureProducerNode[I](future: Future[I]) extends ProducerNode[I] { - def createProducer(materializer: ActorBasedFlowMaterializer, flowName: String): Producer[I] = + final case class FuturePublisherNode[I](future: Future[I]) extends PublisherNode[I] { + def createPublisher(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[I] = future.value match { case Some(Success(element)) ⇒ - new ActorProducer[I](materializer.context.actorOf(IterableProducer.props(List(element), materializer.settings), + ActorPublisher[I](materializer.context.actorOf(IterablePublisher.props(List(element), materializer.settings), name = s"$flowName-0-future"), Some(future)) case Some(Failure(t)) ⇒ - ErrorProducer(t).asInstanceOf[Producer[I]] + ErrorPublisher(t).asInstanceOf[Publisher[I]] case None ⇒ - new ActorProducer[I](materializer.context.actorOf(FutureProducer.props(future, materializer.settings), + ActorPublisher[I](materializer.context.actorOf(FuturePublisher.props(future, materializer.settings), name = s"$flowName-0-future"), Some(future)) } } - final case class TickProducerNode[I](interval: FiniteDuration, tick: () ⇒ I) extends ProducerNode[I] { - def createProducer(materializer: ActorBasedFlowMaterializer, flowName: String): Producer[I] = - new ActorProducer(materializer.context.actorOf(TickProducer.props(interval, tick, materializer.settings), + final case class TickPublisherNode[I](interval: FiniteDuration, tick: () ⇒ I) extends PublisherNode[I] { + def createPublisher(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[I] = + ActorPublisher[I](materializer.context.actorOf(TickPublisher.props(interval, tick, materializer.settings), name = s"$flowName-0-tick")) } } @@ -180,48 +178,43 @@ private[akka] class ActorBasedFlowMaterializer( private def createFlowName(): String = s"$namePrefix-${nextFlowNameCount()}" - @tailrec private def processorChain(topConsumer: Consumer[_], ops: immutable.Seq[AstNode], - flowName: String, n: Int): Consumer[_] = { + @tailrec private def processorChain(topSubscriber: Subscriber[_], ops: immutable.Seq[AstNode], + flowName: String, n: Int): Subscriber[_] = { ops match { case op :: tail ⇒ val opProcessor: Processor[Any, Any] = processorForNode(op, flowName, n) - opProcessor.produceTo(topConsumer.asInstanceOf[Consumer[Any]]) + opProcessor.subscribe(topSubscriber.asInstanceOf[Subscriber[Any]]) processorChain(opProcessor, tail, flowName, n - 1) - case _ ⇒ topConsumer + case _ ⇒ topSubscriber } } // Ops come in reverse order - override def toProducer[I, O](producerNode: ProducerNode[I], ops: List[AstNode]): Producer[O] = { + override def toPublisher[I, O](publisherNode: PublisherNode[I], ops: List[AstNode]): Publisher[O] = { val flowName = createFlowName() - if (ops.isEmpty) producerNode.createProducer(this, flowName).asInstanceOf[Producer[O]] + if (ops.isEmpty) publisherNode.createPublisher(this, flowName).asInstanceOf[Publisher[O]] else { val opsSize = ops.size val opProcessor = processorForNode(ops.head, flowName, opsSize) - val topConsumer = processorChain(opProcessor, ops.tail, flowName, opsSize - 1) - producerNode.createProducer(this, flowName).produceTo(topConsumer.asInstanceOf[Consumer[I]]) - opProcessor.asInstanceOf[Producer[O]] + val topSubscriber = processorChain(opProcessor, ops.tail, flowName, opsSize - 1) + publisherNode.createPublisher(this, flowName).subscribe(topSubscriber.asInstanceOf[Subscriber[I]]) + opProcessor.asInstanceOf[Publisher[O]] } } - private val blackholeTransform = Transform( - new Transformer[Any, Any] { - override def onNext(element: Any) = Nil - }) - private val identityTransform = Transform( new Transformer[Any, Any] { override def onNext(element: Any) = List(element) }) def processorForNode(op: AstNode, flowName: String, n: Int): Processor[Any, Any] = - new ActorProcessor(context.actorOf(ActorProcessor.props(settings, op), + ActorProcessor(context.actorOf(ActorProcessor.props(settings, op), name = s"$flowName-$n-${op.name}")) - override def ductProduceTo[In, Out](consumer: Consumer[Out], ops: List[Ast.AstNode]): Consumer[In] = - processorChain(consumer, ops, createFlowName(), ops.size).asInstanceOf[Consumer[In]] + override def ductProduceTo[In, Out](subscriber: Subscriber[Out], ops: List[Ast.AstNode]): Subscriber[In] = + processorChain(subscriber, ops, createFlowName(), ops.size).asInstanceOf[Subscriber[In]] - override def ductBuild[In, Out](ops: List[Ast.AstNode]): (Consumer[In], Producer[Out]) = { + override def ductBuild[In, Out](ops: List[Ast.AstNode]): (Subscriber[In], Publisher[Out]) = { val flowName = createFlowName() if (ops.isEmpty) { val identityProcessor: Processor[In, Out] = processorForNode(identityTransform, flowName, 1).asInstanceOf[Processor[In, Out]] @@ -229,8 +222,8 @@ private[akka] class ActorBasedFlowMaterializer( } else { val opsSize = ops.size val outProcessor = processorForNode(ops.head, flowName, opsSize).asInstanceOf[Processor[In, Out]] - val topConsumer = processorChain(outProcessor, ops.tail, flowName, opsSize - 1).asInstanceOf[Processor[In, Out]] - (topConsumer, outProcessor) + val topSubscriber = processorChain(outProcessor, ops.tail, flowName, opsSize - 1).asInstanceOf[Processor[In, Out]] + (topSubscriber, outProcessor) } } diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorProcessor.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorProcessor.scala index 29953f47b5..2549dc4b75 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/ActorProcessor.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/ActorProcessor.scala @@ -3,17 +3,11 @@ */ package akka.stream.impl -import org.reactivestreams.api.Processor -import org.reactivestreams.spi.Subscriber +import org.reactivestreams.{ Publisher, Subscriber, Subscription, Processor } import akka.actor._ import akka.stream.MaterializerSettings -import akka.event.LoggingReceive +import akka.stream.actor.ActorSubscriber.{ OnSubscribe, OnNext, OnComplete, OnError } import java.util.Arrays -import scala.util.control.NonFatal -import org.reactivestreams.api.Consumer -import akka.stream.actor.ActorSubscriber -import akka.stream.actor.ActorConsumer.{ OnNext, OnError, OnComplete, OnSubscribe } -import org.reactivestreams.spi.Subscription import akka.stream.TimerTransformer /** @@ -40,13 +34,23 @@ private[akka] object ActorProcessor { case ConcatAll ⇒ Props(new ConcatAllImpl(settings)) case m: MapFuture ⇒ Props(new MapFutureProcessorImpl(settings, m.f)) }).withDispatcher(settings.dispatcher) + + def apply[I, O](impl: ActorRef): ActorProcessor[I, O] = { + val p = new ActorProcessor[I, O](impl) + impl ! ExposedPublisher(p.asInstanceOf[ActorPublisher[Any]]) + p + } } /** * INTERNAL API */ -private[akka] class ActorProcessor[I, O]( final val impl: ActorRef) extends Processor[I, O] with Consumer[I] with ActorProducerLike[O] { - override val getSubscriber: Subscriber[I] = new ActorSubscriber[I](impl) +private[akka] class ActorProcessor[I, O] private (impl: ActorRef) extends ActorPublisher[O](impl, None) + with Processor[I, O] { + override def onSubscribe(s: Subscription): Unit = impl ! OnSubscribe(s) + override def onError(t: Throwable): Unit = impl ! OnError(t) + override def onComplete(): Unit = impl ! OnComplete + override def onNext(t: I): Unit = impl ! OnNext(t) } /** @@ -74,7 +78,7 @@ private[akka] abstract class BatchingInputBuffer(val size: Int, val pump: Pump) batchRemaining -= 1 if (batchRemaining == 0 && !upstreamCompleted) { - upstream.requestMore(requestBatchSize) + upstream.request(requestBatchSize) batchRemaining = requestBatchSize } @@ -120,7 +124,7 @@ private[akka] abstract class BatchingInputBuffer(val size: Int, val pump: Pump) assert(subscription != null) upstream = subscription // Prefetch - upstream.requestMore(inputBuffer.length) + upstream.request(inputBuffer.length) subreceive.become(upstreamRunning) } diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorProducer.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorPublisher.scala similarity index 78% rename from akka-stream/src/main/scala/akka/stream/impl/ActorProducer.scala rename to akka-stream/src/main/scala/akka/stream/impl/ActorPublisher.scala index bf98e90941..9f7ef4414a 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/ActorProducer.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/ActorPublisher.scala @@ -4,64 +4,23 @@ package akka.stream.impl import java.util.concurrent.atomic.AtomicReference + +import akka.actor.{ Actor, ActorLogging, ActorRef, Props, Terminated } +import akka.stream.{ MaterializerSettings, Stop } +import org.reactivestreams.{ Publisher, Subscriber } + import scala.annotation.tailrec import scala.collection.immutable -import org.reactivestreams.api.{ Consumer, Producer } -import org.reactivestreams.spi.{ Publisher, Subscriber } -import akka.actor.ActorRef -import akka.stream.MaterializerSettings -import akka.actor.ActorLogging -import akka.actor.Actor import scala.concurrent.duration.Duration -import scala.util.control.NonFatal -import akka.actor.Props -import scala.util.control.NoStackTrace -import akka.stream.Stop -import akka.actor.Terminated +import scala.util.control.{ NoStackTrace, NonFatal } /** * INTERNAL API */ -private[akka] trait ActorProducerLike[T] extends Producer[T] { - def impl: ActorRef - override val getPublisher: Publisher[T] = { - val a = new ActorPublisher[T](impl) - // Resolve cyclic dependency with actor. This MUST be the first message no matter what. - impl ! ExposedPublisher(a.asInstanceOf[ActorPublisher[Any]]) - a - } - - def produceTo(consumer: Consumer[T]): Unit = - getPublisher.subscribe(consumer.getSubscriber) -} - -/** - * INTERNAL API - * If equalityValue is defined it is used for equals and hashCode, otherwise default reference equality. - */ -private[akka] class ActorProducer[T]( final val impl: ActorRef, val equalityValue: Option[AnyRef] = None) extends ActorProducerLike[T] { - override def equals(o: Any): Boolean = (equalityValue, o) match { - case (Some(v), ActorProducer(_, Some(otherValue))) ⇒ v.equals(otherValue) - case _ ⇒ super.equals(o) - } - - override def hashCode: Int = equalityValue match { - case Some(v) ⇒ v.hashCode - case None ⇒ super.hashCode - } -} - -/** - * INTERNAL API - */ -private[akka] object ActorProducer { +private[akka] object SimpleCallbackPublisher { def props[T](settings: MaterializerSettings, f: () ⇒ T): Props = - Props(new ActorProducerImpl(f, settings)).withDispatcher(settings.dispatcher) + Props(new SimpleCallbackPublisherImpl(f, settings)).withDispatcher(settings.dispatcher) - def unapply(o: Any): Option[(ActorRef, Option[AnyRef])] = o match { - case other: ActorProducer[_] ⇒ Some((other.impl, other.equalityValue)) - case _ ⇒ None - } } /** @@ -70,12 +29,27 @@ private[akka] object ActorProducer { private[akka] object ActorPublisher { class NormalShutdownException extends IllegalStateException("Cannot subscribe to shut-down spi.Publisher") with NoStackTrace val NormalShutdownReason: Option[Throwable] = Some(new NormalShutdownException) + + def apply[T](impl: ActorRef, equalityValue: Option[AnyRef] = None): ActorPublisher[T] = { + val a = new ActorPublisher[T](impl, equalityValue) + // Resolve cyclic dependency with actor. This MUST be the first message no matter what. + impl ! ExposedPublisher(a.asInstanceOf[ActorPublisher[Any]]) + a + } + + def unapply(o: Any): Option[(ActorRef, Option[AnyRef])] = o match { + case other: ActorPublisher[_] ⇒ Some((other.impl, other.equalityValue)) + case _ ⇒ None + } } /** * INTERNAL API + * + * When you instantiate this class, or its subclasses, you MUST send an ExposedPublisher message to the wrapped + * ActorRef! If you don't need to subclass, prefer the apply() method on the companion object which takes care of this. */ -private[akka] final class ActorPublisher[T](val impl: ActorRef) extends Publisher[T] { +private[akka] class ActorPublisher[T](val impl: ActorRef, val equalityValue: Option[AnyRef]) extends Publisher[T] { // The subscriber of an subscription attempt is first placed in this list of pending subscribers. // The actor will call takePendingSubscribers to remove it from the list when it has received the @@ -121,13 +95,23 @@ private[akka] final class ActorPublisher[T](val impl: ActorRef) extends Publishe case Some(e) ⇒ subscriber.onError(e) case None ⇒ subscriber.onComplete() } + + override def equals(o: Any): Boolean = (equalityValue, o) match { + case (Some(v), ActorPublisher(_, Some(otherValue))) ⇒ v.equals(otherValue) + case _ ⇒ super.equals(o) + } + + override def hashCode: Int = equalityValue match { + case Some(v) ⇒ v.hashCode + case None ⇒ super.hashCode + } } /** * INTERNAL API */ private[akka] class ActorSubscription[T]( final val impl: ActorRef, final val subscriber: Subscriber[T]) extends SubscriptionWithCursor[T] { - override def requestMore(elements: Int): Unit = + override def request(elements: Int): Unit = if (elements <= 0) throw new IllegalArgumentException("The number of requested elements must be > 0") else impl ! RequestMore(this, elements) override def cancel(): Unit = impl ! Cancel(this) @@ -154,21 +138,21 @@ private[akka] trait SoftShutdown { this: Actor ⇒ /** * INTERNAL API */ -private[akka] object ActorProducerImpl { +private[akka] object SimpleCallbackPublisherImpl { case object Generate } /** * INTERNAL API */ -private[akka] class ActorProducerImpl[T](f: () ⇒ T, settings: MaterializerSettings) +private[akka] class SimpleCallbackPublisherImpl[T](f: () ⇒ T, settings: MaterializerSettings) extends Actor with ActorLogging with SubscriberManagement[T] with SoftShutdown { - import ActorProducerImpl._ - import ActorBasedFlowMaterializer._ + import akka.stream.impl.ActorBasedFlowMaterializer._ + import akka.stream.impl.SimpleCallbackPublisherImpl._ type S = ActorSubscription[T] var pub: ActorPublisher[T] = _ diff --git a/akka-stream/src/main/scala/akka/stream/impl/BlackholeConsumer.scala b/akka-stream/src/main/scala/akka/stream/impl/BlackholeSubscriber.scala similarity index 68% rename from akka-stream/src/main/scala/akka/stream/impl/BlackholeConsumer.scala rename to akka-stream/src/main/scala/akka/stream/impl/BlackholeSubscriber.scala index 62aa8fa531..a9dde4be20 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/BlackholeConsumer.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/BlackholeSubscriber.scala @@ -3,23 +3,19 @@ */ package akka.stream.impl -import org.reactivestreams.api.Consumer -import org.reactivestreams.spi.Subscription -import org.reactivestreams.spi.Subscriber +import org.reactivestreams.{ Subscriber, Subscription } /** * INTERNAL API */ -private[akka] class BlackholeConsumer[T](highWatermark: Int) extends Consumer[T] with Subscriber[T] { +private[akka] class BlackholeSubscriber[T](highWatermark: Int) extends Subscriber[T] { private val lowWatermark = Math.max(1, highWatermark / 2) private var requested = 0 private var subscription: Subscription = _ - override def getSubscriber: Subscriber[T] = this - override def onSubscribe(sub: Subscription): Unit = { subscription = sub requestMore() @@ -37,7 +33,7 @@ private[akka] class BlackholeConsumer[T](highWatermark: Int) extends Consumer[T] private def requestMore(): Unit = if (requested < lowWatermark) { val amount = highWatermark - requested - subscription.requestMore(amount) + subscription.request(amount) requested += amount } diff --git a/akka-stream/src/main/scala/akka/stream/impl/CompletedPublishers.scala b/akka-stream/src/main/scala/akka/stream/impl/CompletedPublishers.scala new file mode 100644 index 0000000000..97b4fc8586 --- /dev/null +++ b/akka-stream/src/main/scala/akka/stream/impl/CompletedPublishers.scala @@ -0,0 +1,20 @@ +/** + * Copyright (C) 2014 Typesafe Inc. + */ +package akka.stream.impl + +import org.reactivestreams.{ Subscriber, Publisher } + +/** + * INTERNAL API + */ +private[akka] case object EmptyPublisher extends Publisher[Nothing] { + def subscribe(subscriber: Subscriber[Nothing]): Unit = subscriber.onComplete() +} + +/** + * INTERNAL API + */ +private[akka] case class ErrorPublisher(t: Throwable) extends Publisher[Nothing] { + def subscribe(subscriber: Subscriber[Nothing]): Unit = subscriber.onError(t) +} diff --git a/akka-stream/src/main/scala/akka/stream/impl/ConcatAllImpl.scala b/akka-stream/src/main/scala/akka/stream/impl/ConcatAllImpl.scala index 1bf44792e2..fccb2a018f 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/ConcatAllImpl.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/ConcatAllImpl.scala @@ -4,7 +4,7 @@ package akka.stream.impl import akka.stream.MaterializerSettings -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import akka.stream.impl.MultiStreamInputProcessor.SubstreamKey /** @@ -13,8 +13,8 @@ import akka.stream.impl.MultiStreamInputProcessor.SubstreamKey private[akka] class ConcatAllImpl(_settings: MaterializerSettings) extends MultiStreamInputProcessor(_settings) { val takeNextSubstream = TransferPhase(primaryInputs.NeedsInput && primaryOutputs.NeedsDemand) { () ⇒ - val producer = primaryInputs.dequeueInputElement().asInstanceOf[Producer[Any]] - val inputs = createSubstreamInputs(producer) + val publisher = primaryInputs.dequeueInputElement().asInstanceOf[Publisher[Any]] + val inputs = createSubstreamInputs(publisher) nextPhase(streamSubstream(inputs)) } diff --git a/akka-stream/src/main/scala/akka/stream/impl/EmptyProducer.scala b/akka-stream/src/main/scala/akka/stream/impl/EmptyProducer.scala deleted file mode 100644 index d6ffd1631d..0000000000 --- a/akka-stream/src/main/scala/akka/stream/impl/EmptyProducer.scala +++ /dev/null @@ -1,37 +0,0 @@ -/** - * Copyright (C) 2014 Typesafe Inc. - */ -package akka.stream.impl - -import org.reactivestreams.api.Consumer -import org.reactivestreams.api.Producer -import org.reactivestreams.spi.Publisher -import org.reactivestreams.spi.Subscriber - -/** - * INTERNAL API - */ -private[akka] case object EmptyProducer extends Producer[Nothing] with Publisher[Nothing] { - def getPublisher: Publisher[Nothing] = this - - def subscribe(subscriber: Subscriber[Nothing]): Unit = - subscriber.onComplete() - - def produceTo(consumer: Consumer[Nothing]): Unit = - getPublisher.subscribe(consumer.getSubscriber) - -} - -/** - * INTERNAL API - */ -private[akka] case class ErrorProducer(t: Throwable) extends Producer[Nothing] with Publisher[Nothing] { - def getPublisher: Publisher[Nothing] = this - - def subscribe(subscriber: Subscriber[Nothing]): Unit = - subscriber.onError(t) - - def produceTo(consumer: Consumer[Nothing]): Unit = - getPublisher.subscribe(consumer.getSubscriber) - -} diff --git a/akka-stream/src/main/scala/akka/stream/impl/FlowImpl.scala b/akka-stream/src/main/scala/akka/stream/impl/FlowImpl.scala index 0b0e3aad78..ce2000bc56 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/FlowImpl.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/FlowImpl.scala @@ -6,15 +6,13 @@ package akka.stream.impl import scala.collection.immutable import scala.concurrent.{ Future, Promise } import scala.util.Try -import org.reactivestreams.api.Consumer -import org.reactivestreams.api.Producer +import org.reactivestreams.{ Publisher, Subscriber } import Ast.{ AstNode, Transform } import akka.stream.{ OverflowStrategy, FlowMaterializer, Transformer } import akka.stream.{ FlattenStrategy, FlowMaterializer, Transformer } import akka.stream.scaladsl.Flow import scala.util.Success import scala.util.Failure -import org.reactivestreams.api.Consumer import akka.stream.scaladsl.Duct import scala.concurrent.duration.FiniteDuration import akka.stream.TimerTransformer @@ -23,7 +21,7 @@ import akka.util.Collections.EmptyImmutableSeq /** * INTERNAL API */ -private[akka] case class FlowImpl[I, O](producerNode: Ast.ProducerNode[I], ops: List[Ast.AstNode]) extends Flow[O] with Builder[O] { +private[akka] case class FlowImpl[I, O](publisherNode: Ast.PublisherNode[I], ops: List[Ast.AstNode]) extends Flow[O] with Builder[O] { import FlowImpl._ import Ast._ @@ -51,7 +49,7 @@ private[akka] case class FlowImpl[I, O](producerNode: Ast.ProducerNode[I], ops: } override def consume(materializer: FlowMaterializer): Unit = - produceTo(materializer, new BlackholeConsumer(materializer.settings.maximumInputBufferSize)) + produceTo(materializer, new BlackholeSubscriber(materializer.settings.maximumInputBufferSize)) override def onComplete(materializer: FlowMaterializer)(callback: Try[Unit] ⇒ Unit): Unit = transform(new Transformer[O, Unit] { @@ -66,10 +64,10 @@ private[akka] case class FlowImpl[I, O](producerNode: Ast.ProducerNode[I], ops: } }).consume(materializer) - override def toProducer(materializer: FlowMaterializer): Producer[O] = materializer.toProducer(producerNode, ops) + override def toPublisher(materializer: FlowMaterializer): Publisher[O] = materializer.toPublisher(publisherNode, ops) - override def produceTo(materializer: FlowMaterializer, consumer: Consumer[_ >: O]) = - toProducer(materializer).produceTo(consumer.asInstanceOf[Consumer[O]]) + override def produceTo(materializer: FlowMaterializer, subscriber: Subscriber[_ >: O]): Unit = + toPublisher(materializer).subscribe(subscriber.asInstanceOf[Subscriber[O]]) } /** @@ -88,13 +86,13 @@ private[akka] case class DuctImpl[In, Out](ops: List[Ast.AstNode]) extends Duct[ override def appendJava[U](duct: akka.stream.javadsl.Duct[_ >: Out, U]): Duct[In, U] = copy(ops = duct.ops ++: ops) - override def produceTo(materializer: FlowMaterializer, consumer: Consumer[Out]): Consumer[In] = - materializer.ductProduceTo(consumer, ops) + override def produceTo(materializer: FlowMaterializer, subscriber: Subscriber[Out]): Subscriber[In] = + materializer.ductProduceTo(subscriber, ops) - override def consume(materializer: FlowMaterializer): Consumer[In] = - produceTo(materializer, new BlackholeConsumer(materializer.settings.maximumInputBufferSize)) + override def consume(materializer: FlowMaterializer): Subscriber[In] = + produceTo(materializer, new BlackholeSubscriber(materializer.settings.maximumInputBufferSize)) - override def onComplete(materializer: FlowMaterializer)(callback: Try[Unit] ⇒ Unit): Consumer[In] = + override def onComplete(materializer: FlowMaterializer)(callback: Try[Unit] ⇒ Unit): Subscriber[In] = transform(new Transformer[Out, Unit] { override def onNext(in: Out) = Nil override def onError(e: Throwable) = { @@ -107,7 +105,7 @@ private[akka] case class DuctImpl[In, Out](ops: List[Ast.AstNode]) extends Duct[ } }).consume(materializer) - override def build(materializer: FlowMaterializer): (Consumer[In], Producer[Out]) = + override def build(materializer: FlowMaterializer): (Subscriber[In], Publisher[Out]) = materializer.ductBuild(ops) } @@ -253,7 +251,7 @@ private[akka] trait Builder[Out] { override def name = "takeWithin" }) - def prefixAndTail(n: Int): Thing[(immutable.Seq[Out], Producer[Out])] = andThen(PrefixAndTail(n)) + def prefixAndTail(n: Int): Thing[(immutable.Seq[Out], Publisher[Out])] = andThen(PrefixAndTail(n)) def grouped(n: Int): Thing[immutable.Seq[Out]] = transform(new Transformer[Out, immutable.Seq[Out]] { @@ -305,17 +303,17 @@ private[akka] trait Builder[Out] { def transform[U](transformer: Transformer[Out, U]): Thing[U] = andThen(Transform(transformer.asInstanceOf[Transformer[Any, Any]])) - def zip[O2](other: Producer[O2]): Thing[(Out, O2)] = andThen(Zip(other.asInstanceOf[Producer[Any]])) + def zip[O2](other: Publisher[O2]): Thing[(Out, O2)] = andThen(Zip(other.asInstanceOf[Publisher[Any]])) - def concat[U >: Out](next: Producer[U]): Thing[U] = andThen(Concat(next.asInstanceOf[Producer[Any]])) + def concat[U >: Out](next: Publisher[U]): Thing[U] = andThen(Concat(next.asInstanceOf[Publisher[Any]])) - def merge[U >: Out](other: Producer[_ <: U]): Thing[U] = andThen(Merge(other.asInstanceOf[Producer[Any]])) + def merge[U >: Out](other: Publisher[_ <: U]): Thing[U] = andThen(Merge(other.asInstanceOf[Publisher[Any]])) - def splitWhen(p: (Out) ⇒ Boolean): Thing[Producer[Out]] = andThen(SplitWhen(p.asInstanceOf[Any ⇒ Boolean])) + def splitWhen(p: (Out) ⇒ Boolean): Thing[Publisher[Out]] = andThen(SplitWhen(p.asInstanceOf[Any ⇒ Boolean])) - def groupBy[K](f: (Out) ⇒ K): Thing[(K, Producer[Out])] = andThen(GroupBy(f.asInstanceOf[Any ⇒ Any])) + def groupBy[K](f: (Out) ⇒ K): Thing[(K, Publisher[Out])] = andThen(GroupBy(f.asInstanceOf[Any ⇒ Any])) - def tee(other: Consumer[_ >: Out]): Thing[Out] = andThen(Tee(other.asInstanceOf[Consumer[Any]])) + def tee(other: Subscriber[_ >: Out]): Thing[Out] = andThen(Tee(other.asInstanceOf[Subscriber[Any]])) def conflate[S](seed: Out ⇒ S, aggregate: (S, Out) ⇒ S): Thing[S] = andThen(Conflate(seed.asInstanceOf[Any ⇒ Any], aggregate.asInstanceOf[(Any, Any) ⇒ Any])) diff --git a/akka-stream/src/main/scala/akka/stream/impl/FutureProducer.scala b/akka-stream/src/main/scala/akka/stream/impl/FuturePublisher.scala similarity index 83% rename from akka-stream/src/main/scala/akka/stream/impl/FutureProducer.scala rename to akka-stream/src/main/scala/akka/stream/impl/FuturePublisher.scala index 12b7bf3a6f..7a509bb048 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/FutureProducer.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/FuturePublisher.scala @@ -3,27 +3,21 @@ */ package akka.stream.impl -import scala.concurrent.Future -import scala.concurrent.duration.Duration -import scala.util.Failure -import scala.util.Success -import scala.util.Try -import org.reactivestreams.spi.Subscriber -import org.reactivestreams.spi.Subscription -import akka.actor.Actor -import akka.actor.ActorRef -import akka.actor.Props -import akka.actor.Status -import akka.actor.SupervisorStrategy +import akka.actor.{ Actor, ActorRef, Props, Status, SupervisorStrategy } import akka.pattern.pipe import akka.stream.MaterializerSettings +import org.reactivestreams.{ Subscriber, Subscription } + +import scala.concurrent.Future +import scala.concurrent.duration.Duration +import scala.util.{ Failure, Success, Try } /** * INTERNAL API */ -private[akka] object FutureProducer { +private[akka] object FuturePublisher { def props(future: Future[Any], settings: MaterializerSettings): Props = - Props(new FutureProducer(future, settings)).withDispatcher(settings.dispatcher) + Props(new FuturePublisher(future, settings)).withDispatcher(settings.dispatcher) object FutureSubscription { case class Cancel(subscription: FutureSubscription) @@ -31,9 +25,9 @@ private[akka] object FutureProducer { } class FutureSubscription(ref: ActorRef) extends Subscription { - import FutureSubscription._ + import akka.stream.impl.FuturePublisher.FutureSubscription._ def cancel(): Unit = ref ! Cancel(this) - def requestMore(elements: Int): Unit = + def request(elements: Int): Unit = if (elements <= 0) throw new IllegalArgumentException("The number of requested elements must be > 0") else ref ! RequestMore(this) override def toString = "FutureSubscription" @@ -43,10 +37,9 @@ private[akka] object FutureProducer { /** * INTERNAL API */ -private[akka] class FutureProducer(future: Future[Any], settings: MaterializerSettings) extends Actor with SoftShutdown { - import FutureProducer.FutureSubscription - import FutureProducer.FutureSubscription.Cancel - import FutureProducer.FutureSubscription.RequestMore +private[akka] class FuturePublisher(future: Future[Any], settings: MaterializerSettings) extends Actor with SoftShutdown { + import akka.stream.impl.FuturePublisher.FutureSubscription + import akka.stream.impl.FuturePublisher.FutureSubscription.{ Cancel, RequestMore } var exposedPublisher: ActorPublisher[Any] = _ var subscribers = Map.empty[Subscriber[Any], FutureSubscription] diff --git a/akka-stream/src/main/scala/akka/stream/impl/IterableProducer.scala b/akka-stream/src/main/scala/akka/stream/impl/IterablePublisher.scala similarity index 76% rename from akka-stream/src/main/scala/akka/stream/impl/IterableProducer.scala rename to akka-stream/src/main/scala/akka/stream/impl/IterablePublisher.scala index 0d78f3521d..3d6e9b6078 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/IterableProducer.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/IterablePublisher.scala @@ -3,25 +3,21 @@ */ package akka.stream.impl +import akka.actor.{ Actor, ActorRef, Props, SupervisorStrategy, Terminated } +import akka.stream.MaterializerSettings +import org.reactivestreams.{ Subscriber, Subscription } + import scala.annotation.tailrec import scala.collection.immutable -import scala.util.control.NonFatal -import org.reactivestreams.spi.Subscriber -import org.reactivestreams.spi.Subscription -import akka.actor.Actor -import akka.actor.ActorRef -import akka.actor.Props -import akka.actor.SupervisorStrategy -import akka.actor.Terminated -import akka.stream.MaterializerSettings import scala.concurrent.duration.Duration +import scala.util.control.NonFatal /** * INTERNAL API */ -private[akka] object IterableProducer { +private[akka] object IterablePublisher { def props(iterable: immutable.Iterable[Any], settings: MaterializerSettings): Props = - Props(new IterableProducer(iterable, settings)).withDispatcher(settings.dispatcher) + Props(new IterablePublisher(iterable, settings)).withDispatcher(settings.dispatcher) object BasicActorSubscription { case object Cancel @@ -30,9 +26,9 @@ private[akka] object IterableProducer { class BasicActorSubscription(ref: ActorRef) extends Subscription { - import BasicActorSubscription._ + import akka.stream.impl.IterablePublisher.BasicActorSubscription._ def cancel(): Unit = ref ! Cancel - def requestMore(elements: Int): Unit = + def request(elements: Int): Unit = if (elements <= 0) throw new IllegalArgumentException("The number of requested elements must be > 0") else ref ! RequestMore(elements) override def toString = "BasicActorSubscription" @@ -43,15 +39,14 @@ private[akka] object IterableProducer { * INTERNAL API * * Elements are produced from the iterator of the iterable. Each subscriber - * makes use of its own iterable, i.e. each consumer will receive the elements from the + * makes use of its own iterable, i.e. each subscriber will receive the elements from the * beginning of the iterable and it can consume the elements in its own pace. */ -private[akka] class IterableProducer(iterable: immutable.Iterable[Any], settings: MaterializerSettings) extends Actor with SoftShutdown { - import IterableProducer.BasicActorSubscription - import IterableProducer.BasicActorSubscription.Cancel - import ActorBasedFlowMaterializer._ +private[akka] class IterablePublisher(iterable: immutable.Iterable[Any], settings: MaterializerSettings) extends Actor with SoftShutdown { + import akka.stream.impl.ActorBasedFlowMaterializer._ + import akka.stream.impl.IterablePublisher.BasicActorSubscription - require(iterable.nonEmpty, "Use EmptyProducer for empty iterable") + require(iterable.nonEmpty, "Use EmptyPublisher for empty iterable") var exposedPublisher: ActorPublisher[Any] = _ var subscribers = Set.empty[Subscriber[Any]] @@ -81,7 +76,7 @@ private[akka] class IterableProducer(iterable: immutable.Iterable[Any], settings case Terminated(worker) ⇒ workerFinished(worker) - case IterableProducerWorker.Finished ⇒ + case IterablePublisherWorker.Finished ⇒ context.unwatch(sender) workerFinished(sender) } @@ -101,7 +96,7 @@ private[akka] class IterableProducer(iterable: immutable.Iterable[Any], settings subscriber.onError(new IllegalStateException(s"Cannot subscribe $subscriber twice")) else { val iterator = withCtx(context)(iterable.iterator) - val worker = context.watch(context.actorOf(IterableProducerWorker.props(iterator, subscriber, + val worker = context.watch(context.actorOf(IterablePublisherWorker.props(iterator, subscriber, settings.maximumInputBufferSize).withDispatcher(context.props.dispatcher))) val subscription = new BasicActorSubscription(worker) subscribers += subscriber @@ -120,9 +115,9 @@ private[akka] class IterableProducer(iterable: immutable.Iterable[Any], settings /** * INTERNAL API */ -private[akka] object IterableProducerWorker { +private[akka] object IterablePublisherWorker { def props(iterator: Iterator[Any], subscriber: Subscriber[Any], maxPush: Int): Props = - Props(new IterableProducerWorker(iterator, subscriber, maxPush)) + Props(new IterablePublisherWorker(iterator, subscriber, maxPush)) private object PushMore case object Finished @@ -135,11 +130,11 @@ private[akka] object IterableProducerWorker { * subscriber immediately when it receives demand, but to allow cancel before * pushing everything it sends a PushMore to itself after a batch of elements. */ -private[akka] class IterableProducerWorker(iterator: Iterator[Any], subscriber: Subscriber[Any], maxPush: Int) +private[akka] class IterablePublisherWorker(iterator: Iterator[Any], subscriber: Subscriber[Any], maxPush: Int) extends Actor with SoftShutdown { - import IterableProducerWorker._ - import IterableProducer.BasicActorSubscription._ - import ActorBasedFlowMaterializer._ + import akka.stream.impl.ActorBasedFlowMaterializer._ + import akka.stream.impl.IterablePublisher.BasicActorSubscription._ + import akka.stream.impl.IterablePublisherWorker._ require(iterator.hasNext, "Iterator must not be empty") diff --git a/akka-stream/src/main/scala/akka/stream/impl/IteratorProducer.scala b/akka-stream/src/main/scala/akka/stream/impl/IteratorPublisher.scala similarity index 81% rename from akka-stream/src/main/scala/akka/stream/impl/IteratorProducer.scala rename to akka-stream/src/main/scala/akka/stream/impl/IteratorPublisher.scala index 2c5a3f6885..9e3e80378f 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/IteratorProducer.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/IteratorPublisher.scala @@ -10,13 +10,13 @@ import akka.stream.Stop /** * INTERNAL API */ -private[akka] object IteratorProducer { +private[akka] object IteratorPublisher { def props(iterator: Iterator[Any], settings: MaterializerSettings): Props = { def f(): Any = { if (!iterator.hasNext) throw Stop iterator.next() } - ActorProducer.props(settings, f) + SimpleCallbackPublisher.props(settings, f) } } \ No newline at end of file diff --git a/akka-stream/src/main/scala/akka/stream/impl/Messages.scala b/akka-stream/src/main/scala/akka/stream/impl/Messages.scala index d78544f794..fc1b55b7bc 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/Messages.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/Messages.scala @@ -3,7 +3,7 @@ */ package akka.stream.impl -import org.reactivestreams.spi.Subscription +import org.reactivestreams.Subscription /** * INTERNAL API diff --git a/akka-stream/src/main/scala/akka/stream/impl/PrefixAndTailImpl.scala b/akka-stream/src/main/scala/akka/stream/impl/PrefixAndTailImpl.scala index 7488c37852..58365761a4 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/PrefixAndTailImpl.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/PrefixAndTailImpl.scala @@ -38,7 +38,7 @@ private[akka] class PrefixAndTailImpl(_settings: MaterializerSettings, val takeM } def emitEmptyTail(): Unit = { - primaryOutputs.enqueueOutputElement((taken, EmptyProducer)) + primaryOutputs.enqueueOutputElement((taken, EmptyPublisher)) nextPhase(completedPhase) } diff --git a/akka-stream/src/main/scala/akka/stream/impl/SplitWhenProcessorImpl.scala b/akka-stream/src/main/scala/akka/stream/impl/SplitWhenProcessorImpl.scala index d257f3ed64..d0f499a924 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/SplitWhenProcessorImpl.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/SplitWhenProcessorImpl.scala @@ -3,11 +3,8 @@ */ package akka.stream.impl -import akka.actor.{ Props, ActorRef } -import org.reactivestreams.spi.Subscription -import akka.stream.impl._ +import akka.actor.ActorRef import akka.stream.MaterializerSettings -import akka.actor.Terminated /** * INTERNAL API diff --git a/akka-stream/src/main/scala/akka/stream/impl/StaticFanins.scala b/akka-stream/src/main/scala/akka/stream/impl/StaticFanins.scala index 450edd721b..6c8f22fe83 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/StaticFanins.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/StaticFanins.scala @@ -4,13 +4,13 @@ package akka.stream.impl import akka.stream.MaterializerSettings -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import scala.concurrent.forkjoin.ThreadLocalRandom /** * INTERNAL API */ -private[akka] class MergeImpl(_settings: MaterializerSettings, _other: Producer[Any]) +private[akka] class MergeImpl(_settings: MaterializerSettings, _other: Publisher[Any]) extends TwoStreamInputProcessor(_settings, _other) { val runningPhase = TransferPhase( @@ -29,7 +29,7 @@ private[akka] class MergeImpl(_settings: MaterializerSettings, _other: Producer[ /** * INTERNAL API */ -private[akka] class ZipImpl(_settings: MaterializerSettings, _other: Producer[Any]) +private[akka] class ZipImpl(_settings: MaterializerSettings, _other: Publisher[Any]) extends TwoStreamInputProcessor(_settings, _other) { val runningPhase = TransferPhase(primaryInputs.NeedsInput && secondaryInputs.NeedsInput && primaryOutputs.NeedsDemand) { () ⇒ @@ -43,7 +43,7 @@ private[akka] class ZipImpl(_settings: MaterializerSettings, _other: Producer[An /** * INTERNAL API */ -private[akka] class ConcatImpl(_settings: MaterializerSettings, _other: Producer[Any]) +private[akka] class ConcatImpl(_settings: MaterializerSettings, _other: Publisher[Any]) extends TwoStreamInputProcessor(_settings, _other) { val processingPrimary = TransferPhase(primaryInputs.NeedsInputOrComplete && primaryOutputs.NeedsDemand) { () ⇒ diff --git a/akka-stream/src/main/scala/akka/stream/impl/StaticFanouts.scala b/akka-stream/src/main/scala/akka/stream/impl/StaticFanouts.scala index 18951e745a..edc8430bd5 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/StaticFanouts.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/StaticFanouts.scala @@ -3,16 +3,13 @@ */ package akka.stream.impl -import org.reactivestreams.api.Producer import akka.stream.MaterializerSettings -import org.reactivestreams.api.Consumer -import org.reactivestreams.spi.Subscriber -import org.reactivestreams.spi.Subscription +import org.reactivestreams.{ Subscriber, Subscription, Publisher } /** * INTERNAL API */ -private[akka] class TeeImpl(_settings: MaterializerSettings, other: Consumer[Any]) +private[akka] class TeeImpl(_settings: MaterializerSettings, other: Subscriber[Any]) extends ActorProcessorImpl(_settings) { override val primaryOutputs = new FanoutOutputs(settings.maxFanOutBufferSize, settings.initialFanOutBufferSize, self, pump = this) { @@ -20,7 +17,7 @@ private[akka] class TeeImpl(_settings: MaterializerSettings, other: Consumer[Any override def registerSubscriber(subscriber: Subscriber[Any]): Unit = { if (!secondarySubscribed) { - super.registerSubscriber(other.getSubscriber) + super.registerSubscriber(other) secondarySubscribed = true } super.registerSubscriber(subscriber) diff --git a/akka-stream/src/main/scala/akka/stream/impl/StreamOfStreamProcessors.scala b/akka-stream/src/main/scala/akka/stream/impl/StreamOfStreamProcessors.scala index 6795f9213d..e87486681e 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/StreamOfStreamProcessors.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/StreamOfStreamProcessors.scala @@ -5,9 +5,8 @@ package akka.stream.impl import akka.stream.MaterializerSettings import akka.actor.{ Actor, Terminated, ActorRef } -import org.reactivestreams.spi.{ Subscriber, Subscription } -import org.reactivestreams.api.Producer -import akka.stream.actor.ActorConsumer.{ OnNext, OnError, OnComplete, OnSubscribe } +import org.reactivestreams.{ Publisher, Subscriber, Subscription } +import akka.stream.actor.ActorSubscriber.{ OnNext, OnError, OnComplete, OnSubscribe } /** * INTERNAL API @@ -17,7 +16,7 @@ private[akka] object MultiStreamOutputProcessor { case class SubstreamCancel(substream: ActorRef) class SubstreamSubscription(val parent: ActorRef, val substream: ActorRef) extends Subscription { - override def requestMore(elements: Int): Unit = + override def request(elements: Int): Unit = if (elements <= 0) throw new IllegalArgumentException("The number of requested elements must be > 0") else parent ! SubstreamRequestMore(substream, elements) override def cancel(): Unit = parent ! SubstreamCancel(substream) @@ -44,7 +43,7 @@ private[akka] abstract class MultiStreamOutputProcessor(_settings: MaterializerS val substream = context.watch(context.actorOf( IdentityProcessorImpl.props(settings) .withDispatcher(context.props.dispatcher))) - val processor = new ActorProcessor[AnyRef, AnyRef](substream) + val processor = ActorProcessor[AnyRef, AnyRef](substream) override def isClosed: Boolean = completed override def complete(): Unit = { @@ -135,7 +134,7 @@ private[akka] object TwoStreamInputProcessor { /** * INTERNAL API */ -private[akka] abstract class TwoStreamInputProcessor(_settings: MaterializerSettings, val other: Producer[Any]) +private[akka] abstract class TwoStreamInputProcessor(_settings: MaterializerSettings, val other: Publisher[Any]) extends ActorProcessorImpl(_settings) { import TwoStreamInputProcessor._ @@ -162,7 +161,7 @@ private[akka] abstract class TwoStreamInputProcessor(_settings: MaterializerSett override def receive = secondaryInputs.subreceive orElse primaryInputs.subreceive orElse primaryOutputs.subreceive - other.getPublisher.subscribe(new OtherActorSubscriber(self)) + other.subscribe(new OtherActorSubscriber(self)) override def shutdownHooks(): Unit = { secondaryInputs.cancel() @@ -224,10 +223,10 @@ private[akka] abstract class MultiStreamInputProcessor(_settings: MaterializerSe } - def createSubstreamInputs(p: Producer[Any]): SubstreamInputs = { + def createSubstreamInputs(p: Publisher[Any]): SubstreamInputs = { val key = SubstreamKey(nextId) val inputs = new SubstreamInputs(key) - p.getPublisher.subscribe(new SubstreamSubscriber(self, key)) + p.subscribe(new SubstreamSubscriber(self, key)) substreamInputs(key) = inputs nextId += 1 inputs diff --git a/akka-stream/src/main/scala/akka/stream/impl/SubscriberManagement.scala b/akka-stream/src/main/scala/akka/stream/impl/SubscriberManagement.scala index df05e5e195..a8027d2346 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/SubscriberManagement.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/SubscriberManagement.scala @@ -4,8 +4,7 @@ package akka.stream.impl import scala.annotation.tailrec -import org.reactivestreams.api -import org.reactivestreams.spi +import org.reactivestreams.{ Subscriber, Subscription } import SubscriberManagement.ShutDown import ResizableMultiReaderRingBuffer.NothingToReadException @@ -15,19 +14,19 @@ import ResizableMultiReaderRingBuffer.NothingToReadException private[akka] object SubscriberManagement { sealed trait EndOfStream { - def apply[T](subscriber: spi.Subscriber[T]): Unit + def apply[T](subscriber: Subscriber[T]): Unit } object NotReached extends EndOfStream { - def apply[T](subscriber: spi.Subscriber[T]): Unit = throw new IllegalStateException("Called apply on NotReached") + def apply[T](subscriber: Subscriber[T]): Unit = throw new IllegalStateException("Called apply on NotReached") } object Completed extends EndOfStream { - def apply[T](subscriber: spi.Subscriber[T]): Unit = subscriber.onComplete() + def apply[T](subscriber: Subscriber[T]): Unit = subscriber.onComplete() } case class ErrorCompleted(cause: Throwable) extends EndOfStream { - def apply[T](subscriber: spi.Subscriber[T]): Unit = subscriber.onError(cause) + def apply[T](subscriber: Subscriber[T]): Unit = subscriber.onError(cause) } val ShutDown = new ErrorCompleted(new IllegalStateException("Cannot subscribe to shut-down spi.Publisher")) @@ -36,8 +35,8 @@ private[akka] object SubscriberManagement { /** * INTERNAL API */ -private[akka] trait SubscriptionWithCursor[T] extends spi.Subscription with ResizableMultiReaderRingBuffer.Cursor { - def subscriber: spi.Subscriber[T] +private[akka] trait SubscriptionWithCursor[T] extends Subscription with ResizableMultiReaderRingBuffer.Cursor { + def subscriber: Subscriber[T] def dispatch(element: T): Unit = subscriber.onNext(element) @@ -77,7 +76,7 @@ private[akka] trait SubscriberManagement[T] extends ResizableMultiReaderRingBuff /** * Use to register a subscriber */ - protected def createSubscription(subscriber: spi.Subscriber[T]): S + protected def createSubscription(subscriber: Subscriber[T]): S private[this] val buffer = new ResizableMultiReaderRingBuffer[T](initialBufferSize, maxBufferSize, this) @@ -199,7 +198,7 @@ private[akka] trait SubscriberManagement[T] extends ResizableMultiReaderRingBuff /** * Register a new subscriber. */ - protected def registerSubscriber(subscriber: spi.Subscriber[T]): Unit = endOfStream match { + protected def registerSubscriber(subscriber: Subscriber[T]): Unit = endOfStream match { case NotReached if subscriptions.exists(_.subscriber eq subscriber) ⇒ subscriber.onError(new IllegalStateException(s"Cannot subscribe $subscriber twice")) case NotReached ⇒ diff --git a/akka-stream/src/main/scala/akka/stream/impl/SynchronousProducerFromIterable.scala b/akka-stream/src/main/scala/akka/stream/impl/SynchronousPublisherFromIterable.scala similarity index 62% rename from akka-stream/src/main/scala/akka/stream/impl/SynchronousProducerFromIterable.scala rename to akka-stream/src/main/scala/akka/stream/impl/SynchronousPublisherFromIterable.scala index 6816d75a7c..f36038153e 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/SynchronousProducerFromIterable.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/SynchronousPublisherFromIterable.scala @@ -3,22 +3,19 @@ */ package akka.stream.impl -import scala.collection.immutable -import org.reactivestreams.api.Consumer -import org.reactivestreams.spi.Subscription +import org.reactivestreams.{ Publisher, Subscriber, Subscription } + import scala.annotation.tailrec -import org.reactivestreams.spi.Subscriber -import org.reactivestreams.spi.Publisher -import org.reactivestreams.api.Producer +import scala.collection.immutable import scala.util.control.NonFatal /** * INTERNAL API */ -private[akka] object SynchronousProducerFromIterable { - def apply[T](iterable: immutable.Iterable[T]): Producer[T] = - if (iterable.isEmpty) EmptyProducer.asInstanceOf[Producer[T]] - else new SynchronousProducerFromIterable(iterable) +private[akka] object SynchronousPublisherFromIterable { + def apply[T](iterable: immutable.Iterable[T]): Publisher[T] = + if (iterable.isEmpty) EmptyPublisher.asInstanceOf[Publisher[T]] + else new SynchronousPublisherFromIterable(iterable) private class IteratorSubscription[T](subscriber: Subscriber[T], iterator: Iterator[T]) extends Subscription { var done = false @@ -28,7 +25,7 @@ private[akka] object SynchronousProducerFromIterable { override def cancel(): Unit = done = true - override def requestMore(elements: Int): Unit = { + override def request(elements: Int): Unit = { @tailrec def pushNext(): Unit = { if (!done) if (iterator.isEmpty) { @@ -60,7 +57,7 @@ private[akka] object SynchronousProducerFromIterable { /** * INTERNAL API - * Producer that will push all requested elements from the iterator of the iterable + * Publisher that will push all requested elements from the iterator of the iterable * to the subscriber in the calling thread of `requestMore`. * * It is only intended to be used with iterators over static collections. @@ -70,27 +67,21 @@ private[akka] object SynchronousProducerFromIterable { * It is the responsibility of the subscriber to provide necessary memory visibility * if calls to `requestMore` and `cancel` are performed from different threads. * For example, usage from an actor is fine. Concurrent calls to the subscription is not allowed. - * Reentrant calls to `requestMore` directly from `onNext` are supported by this producer. + * Reentrant calls to `requestMore` directly from `onNext` are supported by this publisher. */ -private[akka] class SynchronousProducerFromIterable[T](private val iterable: immutable.Iterable[T]) - extends Producer[T] with Publisher[T] { +private[akka] class SynchronousPublisherFromIterable[T](private val iterable: immutable.Iterable[T]) extends Publisher[T] { - import SynchronousProducerFromIterable.IteratorSubscription - - override def getPublisher: Publisher[T] = this + import akka.stream.impl.SynchronousPublisherFromIterable.IteratorSubscription override def subscribe(subscriber: Subscriber[T]): Unit = subscriber.onSubscribe(new IteratorSubscription(subscriber, iterable.iterator)) - override def produceTo(consumer: Consumer[T]): Unit = - getPublisher.subscribe(consumer.getSubscriber) - override def equals(o: Any): Boolean = o match { - case other: SynchronousProducerFromIterable[T] ⇒ iterable == other.iterable - case _ ⇒ false + case other: SynchronousPublisherFromIterable[T] ⇒ iterable == other.iterable + case _ ⇒ false } override def hashCode: Int = iterable.hashCode - override def toString: String = s"SynchronousProducerFromIterable(${iterable.mkString(", ")})" + override def toString: String = s"SynchronousPublisherFromIterable(${iterable.mkString(", ")})" } diff --git a/akka-stream/src/main/scala/akka/stream/impl/TickProducer.scala b/akka-stream/src/main/scala/akka/stream/impl/TickPublisher.scala similarity index 76% rename from akka-stream/src/main/scala/akka/stream/impl/TickProducer.scala rename to akka-stream/src/main/scala/akka/stream/impl/TickPublisher.scala index 99a4e53f61..cc0e23fba5 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/TickProducer.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/TickPublisher.scala @@ -3,39 +3,33 @@ */ package akka.stream.impl -import scala.collection.mutable -import scala.concurrent.duration.Duration -import scala.concurrent.duration.FiniteDuration -import org.reactivestreams.spi.Subscriber -import org.reactivestreams.spi.Subscription -import akka.actor.Actor -import akka.actor.ActorRef -import akka.actor.Props -import akka.actor.SupervisorStrategy +import akka.actor.{ Actor, ActorRef, Cancellable, Props, SupervisorStrategy } import akka.stream.MaterializerSettings +import org.reactivestreams.{ Subscriber, Subscription } + +import scala.collection.mutable +import scala.concurrent.duration.{ Duration, FiniteDuration } import scala.util.control.NonFatal -import akka.actor.Cancellable /** * INTERNAL API */ -private[akka] object TickProducer { +private[akka] object TickPublisher { def props(interval: FiniteDuration, tick: () ⇒ Any, settings: MaterializerSettings): Props = - Props(new TickProducer(interval, tick, settings)).withDispatcher(settings.dispatcher) + Props(new TickPublisher(interval, tick, settings)).withDispatcher(settings.dispatcher) - object TickProducerSubscription { + object TickPublisherSubscription { case class Cancel(subscriber: Subscriber[Any]) case class RequestMore(elements: Int, subscriber: Subscriber[Any]) } - class TickProducerSubscription(ref: ActorRef, subscriber: Subscriber[Any]) - extends Subscription { - import TickProducerSubscription._ + class TickPublisherSubscription(ref: ActorRef, subscriber: Subscriber[Any]) extends Subscription { + import akka.stream.impl.TickPublisher.TickPublisherSubscription._ def cancel(): Unit = ref ! Cancel(subscriber) - def requestMore(elements: Int): Unit = + def request(elements: Int): Unit = if (elements <= 0) throw new IllegalArgumentException("The number of requested elements must be > 0") else ref ! RequestMore(elements, subscriber) - override def toString = "TickProducerSubscription" + override def toString = "TickPublisherSubscription" } private case object Tick @@ -48,9 +42,9 @@ private[akka] object TickProducer { * Each subscriber will receive the tick element if it has requested any elements, * otherwise the tick element is dropped for that subscriber. */ -private[akka] class TickProducer(interval: FiniteDuration, tick: () ⇒ Any, settings: MaterializerSettings) extends Actor with SoftShutdown { - import TickProducer._ - import TickProducer.TickProducerSubscription._ +private[akka] class TickPublisher(interval: FiniteDuration, tick: () ⇒ Any, settings: MaterializerSettings) extends Actor with SoftShutdown { + import akka.stream.impl.TickPublisher.TickPublisherSubscription._ + import akka.stream.impl.TickPublisher._ var exposedPublisher: ActorPublisher[Any] = _ val demand = mutable.Map.empty[Subscriber[Any], Long] @@ -100,8 +94,7 @@ private[akka] class TickProducer(interval: FiniteDuration, tick: () ⇒ Any, set case Some(d) ⇒ demand(subscriber) = d + elements case None ⇒ // canceled } - case Cancel ⇒ - softShutdown() + case Cancel(subscriber) ⇒ unregisterSubscriber(subscriber) case SubscribePending ⇒ exposedPublisher.takePendingSubscribers() foreach registerSubscriber @@ -112,7 +105,7 @@ private[akka] class TickProducer(interval: FiniteDuration, tick: () ⇒ Any, set if (demand.contains(subscriber)) subscriber.onError(new IllegalStateException(s"Cannot subscribe $subscriber twice")) else { - val subscription = new TickProducerSubscription(self, subscriber) + val subscription = new TickPublisherSubscription(self, subscriber) demand(subscriber) = 0 subscriber.onSubscribe(subscription) } diff --git a/akka-stream/src/main/scala/akka/stream/impl/Transfer.scala b/akka-stream/src/main/scala/akka/stream/impl/Transfer.scala index 2ea4549049..a7119b0f48 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/Transfer.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/Transfer.scala @@ -3,8 +3,6 @@ */ package akka.stream.impl -import org.reactivestreams.spi.{ Subscriber, Subscription } -import java.util.Arrays import scala.util.control.NonFatal import akka.actor.{ Actor, ActorRefFactory } diff --git a/akka-stream/src/main/scala/akka/stream/io/StreamIO.scala b/akka-stream/src/main/scala/akka/stream/io/StreamIO.scala index 073364adf7..63e04cf72e 100644 --- a/akka-stream/src/main/scala/akka/stream/io/StreamIO.scala +++ b/akka-stream/src/main/scala/akka/stream/io/StreamIO.scala @@ -4,7 +4,7 @@ package akka.stream.io import akka.util.ByteString -import org.reactivestreams.api.{ Processor, Producer, Consumer } +import org.reactivestreams.{ Processor, Publisher, Subscriber } import java.net.InetSocketAddress import akka.actor._ import scala.collection._ @@ -26,19 +26,19 @@ object StreamTcp extends ExtensionId[StreamTcpExt] with ExtensionIdProvider { case class OutgoingTcpConnection(remoteAddress: InetSocketAddress, localAddress: InetSocketAddress, processor: Processor[ByteString, ByteString]) { - def outputStream: Consumer[ByteString] = processor - def inputStream: Producer[ByteString] = processor + def outputStream: Subscriber[ByteString] = processor + def inputStream: Publisher[ByteString] = processor } case class TcpServerBinding(localAddress: InetSocketAddress, - connectionStream: Producer[IncomingTcpConnection]) + connectionStream: Publisher[IncomingTcpConnection]) case class IncomingTcpConnection(remoteAddress: InetSocketAddress, - inputStream: Producer[ByteString], - outputStream: Consumer[ByteString]) { + inputStream: Publisher[ByteString], + outputStream: Subscriber[ByteString]) { def handleWith(processor: Processor[ByteString, ByteString]): Unit = { - processor.produceTo(outputStream) - inputStream.produceTo(processor) + processor.subscribe(outputStream) + inputStream.subscribe(processor) } } @@ -220,14 +220,14 @@ private[akka] class StreamTcpManager extends Actor { Tcp.Connect(remoteAddress, localAddress, options, connTimeout, pullMode = true), requester = sender(), settings), name = encName("client", remoteAddress)) - processorActor ! ExposedProcessor(new ActorProcessor[ByteString, ByteString](processorActor)) + processorActor ! ExposedProcessor(ActorProcessor[ByteString, ByteString](processorActor)) case StreamTcp.Bind(settings, localAddress, backlog, options, idleTimeout) ⇒ val publisherActor = context.actorOf(TcpListenStreamActor.props( Tcp.Bind(context.system.deadLetters, localAddress, backlog, options, pullMode = true), requester = sender(), settings), name = encName("server", localAddress)) - publisherActor ! ExposedPublisher(new ActorPublisher(publisherActor)) + publisherActor ! ExposedPublisher(ActorPublisher[Any](publisherActor)) } } diff --git a/akka-stream/src/main/scala/akka/stream/io/TcpConnectionStream.scala b/akka-stream/src/main/scala/akka/stream/io/TcpConnectionStream.scala index be6976ee8d..8cd0a56146 100644 --- a/akka-stream/src/main/scala/akka/stream/io/TcpConnectionStream.scala +++ b/akka-stream/src/main/scala/akka/stream/io/TcpConnectionStream.scala @@ -10,7 +10,7 @@ import akka.stream.impl._ import akka.util.ByteString import akka.io.Tcp._ import akka.stream.MaterializerSettings -import org.reactivestreams.api.Processor +import org.reactivestreams.Processor /** * INTERNAL API diff --git a/akka-stream/src/main/scala/akka/stream/io/TcpListenStreamActor.scala b/akka-stream/src/main/scala/akka/stream/io/TcpListenStreamActor.scala index 6cf2b5142b..70e9d858d1 100644 --- a/akka-stream/src/main/scala/akka/stream/io/TcpListenStreamActor.scala +++ b/akka-stream/src/main/scala/akka/stream/io/TcpListenStreamActor.scala @@ -3,15 +3,15 @@ */ package akka.stream.io -import scala.util.control.NoStackTrace import akka.actor._ -import akka.stream.impl._ -import akka.io.{ IO, Tcp } import akka.io.Tcp._ -import akka.util.ByteString -import org.reactivestreams.api.{ Consumer, Producer } -import org.reactivestreams.spi.Publisher +import akka.io.{ IO, Tcp } import akka.stream.MaterializerSettings +import akka.stream.impl._ +import akka.util.ByteString +import org.reactivestreams.Publisher + +import scala.util.control.NoStackTrace /** * INTERNAL API @@ -22,12 +22,6 @@ private[akka] object TcpListenStreamActor { def props(bindCmd: Tcp.Bind, requester: ActorRef, settings: MaterializerSettings): Props = Props(new TcpListenStreamActor(bindCmd, requester, settings)).withDispatcher(settings.dispatcher) - case class ConnectionProducer(getPublisher: Publisher[StreamTcp.IncomingTcpConnection]) - extends Producer[StreamTcp.IncomingTcpConnection] { - - def produceTo(consumer: Consumer[StreamTcp.IncomingTcpConnection]): Unit = - getPublisher.subscribe(consumer.getSubscriber) - } } /** @@ -35,7 +29,7 @@ private[akka] object TcpListenStreamActor { */ private[akka] class TcpListenStreamActor(bindCmd: Tcp.Bind, requester: ActorRef, val settings: MaterializerSettings) extends Actor with Pump { - import TcpListenStreamActor._ + import akka.stream.io.TcpListenStreamActor._ import context.system object primaryOutputs extends FanoutOutputs(settings.maxFanOutBufferSize, settings.initialFanOutBufferSize, self, pump = this) { @@ -72,7 +66,7 @@ private[akka] class TcpListenStreamActor(bindCmd: Tcp.Bind, requester: ActorRef, listener ! ResumeAccepting(1) requester ! StreamTcp.TcpServerBinding( localAddress, - ConnectionProducer(primaryOutputs.getExposedPublisher.asInstanceOf[Publisher[StreamTcp.IncomingTcpConnection]])) + primaryOutputs.getExposedPublisher.asInstanceOf[Publisher[StreamTcp.IncomingTcpConnection]]) subreceive.become(running) case f: CommandFailed ⇒ val ex = new TcpListenStreamException("Bind failed") @@ -112,7 +106,7 @@ private[akka] class TcpListenStreamActor(bindCmd: Tcp.Bind, requester: ActorRef, def runningPhase = TransferPhase(primaryOutputs.NeedsDemand && incomingConnections.NeedsInput) { () ⇒ val (connected: Connected, connection: ActorRef) = incomingConnections.dequeueInputElement() val tcpStreamActor = context.actorOf(TcpStreamActor.inboundProps(connection, settings)) - val processor = new ActorProcessor[ByteString, ByteString](tcpStreamActor) + val processor = ActorProcessor[ByteString, ByteString](tcpStreamActor) primaryOutputs.enqueueOutputElement(StreamTcp.IncomingTcpConnection(connected.remoteAddress, processor, processor)) } diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Duct.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Duct.scala index 3354b55bec..1e0768a1aa 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Duct.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Duct.scala @@ -7,8 +7,7 @@ import scala.collection.immutable import scala.collection.JavaConverters._ import scala.util.Failure import scala.util.Success -import org.reactivestreams.api.Consumer -import org.reactivestreams.api.Producer +import org.reactivestreams.{ Publisher, Subscriber } import akka.japi.Function import akka.japi.Function2 import akka.japi.Pair @@ -28,7 +27,7 @@ object Duct { /** * Create an empty [[Duct]]. The transformation steps are executed by a series - * of [[org.reactivestreams.api.Processor]] instances that mediate the flow of + * of [[org.reactivestreams.Processor]] instances that mediate the flow of * elements downstream and the propagation of back-pressure upstream. */ def create[In](inputType: Class[In]): Duct[In, In] = new DuctAdapter(SDuct.apply[In]) @@ -40,8 +39,8 @@ object Duct { * The difference is that it is not attached to an input source. * * The pipeline must be materialized by calling the [[#produceTo]], [[#consume]] or [[#build]] - * methods on it and then attach the `Consumer` representing the input side of the `Duct` to an - * upstream `Producer`. + * methods on it and then attach the `Subscriber` representing the input side of the `Duct` to an + * upstream `Publisher`. * */ abstract class Duct[In, Out] { @@ -79,8 +78,8 @@ abstract class Duct[In, Out] { * Invoke the given procedure for each received element and produce a Unit value * upon reaching the normal end of the stream. Please note that also in this case * the `Duct` needs to be materialized (e.g. using [[#consume]] and attaching the - * the `Consumer` representing the input side of the `Duct` to an upstream - * `Producer`) to initiate its execution. + * the `Subscriber` representing the input side of the `Duct` to an upstream + * `Publisher`) to initiate its execution. */ def foreach(c: Procedure[Out]): Duct[In, Void] @@ -103,17 +102,17 @@ abstract class Duct[In, Out] { def dropWithin(d: FiniteDuration): Duct[In, Out] /** - * Terminate processing (and cancel the upstream producer) after the given + * Terminate processing (and cancel the upstream publisher) after the given * number of elements. Due to input buffering some elements may have been - * requested from upstream producers that will then not be processed downstream + * requested from upstream publishers that will then not be processed downstream * of this step. */ def take(n: Int): Duct[In, Out] /** - * Terminate processing (and cancel the upstream producer) after the given + * Terminate processing (and cancel the upstream publisher) after the given * duration. Due to input buffering some elements may have been - * requested from upstream producers that will then not be processed downstream + * requested from upstream pubilshers that will then not be processed downstream * of this step. * * Note that this can be combined with [[#take]] to limit the number of elements @@ -147,9 +146,9 @@ abstract class Duct[In, Out] { * function is invoked and expecting a (possibly empty) sequence of output elements * to be produced. * After handing off the elements produced from one input element to the downstream - * consumers, the [[akka.stream.Transformer#isComplete]] predicate determines whether to end + * subscribers, the [[akka.stream.Transformer#isComplete]] predicate determines whether to end * stream processing at this point; in that case the upstream subscription is - * canceled. Before signaling normal completion to the downstream consumers, + * canceled. Before signaling normal completion to the downstream subscribers, * the [[akka.stream.Transformer#onComplete]] function is invoked to produce a (possibly empty) * sequence of elements in response to the end-of-stream event. * @@ -170,20 +169,20 @@ abstract class Duct[In, Out] { * and a stream representing the remaining elements. If ''n'' is zero or negative, then this will return a pair * of an empty collection and a stream containing the whole upstream unchanged. */ - def prefixAndTail(n: Int): Duct[In, Pair[java.util.List[Out], Producer[Out]]] + def prefixAndTail(n: Int): Duct[In, Pair[java.util.List[Out], Publisher[Out]]] /** * This operation demultiplexes the incoming stream into separate output * streams, one for each element key. The key is computed for each element * using the given function. When a new key is encountered for the first time - * it is emitted to the downstream consumer together with a fresh - * producer that will eventually produce all the elements of the substream + * it is emitted to the downstream subscriber together with a fresh + * publisher that will eventually produce all the elements of the substream * for that key. Not consuming the elements from the created streams will * stop this processor from processing more elements, therefore you must take * care to unblock (or cancel) all of the produced streams even if you want * to consume only one of them. */ - def groupBy[K](f: Function[Out, K]): Duct[In, Pair[K, Producer[Out]]] + def groupBy[K](f: Function[Out, K]): Duct[In, Pair[K, Publisher[Out]]] /** * This operation applies the given predicate to all incoming elements and @@ -198,40 +197,40 @@ abstract class Duct[In, Out] { * true, false, false // elements go into third substream * }}} */ - def splitWhen(p: Predicate[Out]): Duct[In, Producer[Out]] + def splitWhen(p: Predicate[Out]): Duct[In, Publisher[Out]] /** - * Merge this stream with the one emitted by the given producer, taking + * Merge this stream with the one emitted by the given publisher, taking * elements as they arrive from either side (picking randomly when both * have elements ready). */ - def merge[U >: Out](other: Producer[U]): Duct[In, U] + def merge[U >: Out](other: Publisher[U]): Duct[In, U] /** - * Zip this stream together with the one emitted by the given producer. + * Zip this stream together with the one emitted by the given publisher. * This transformation finishes when either input stream reaches its end, * cancelling the subscription to the other one. */ - def zip[U](other: Producer[U]): Duct[In, Pair[Out, U]] + def zip[U](other: Publisher[U]): Duct[In, Pair[Out, U]] /** * Concatenate the given other stream to this stream so that the first element - * emitted by the given producer is emitted after the last element of this + * emitted by the given publisher is emitted after the last element of this * stream. */ - def concat[U >: Out](next: Producer[U]): Duct[In, U] + def concat[U >: Out](next: Publisher[U]): Duct[In, U] /** - * Fan-out the stream to another consumer. Each element is produced to - * the `other` consumer as well as to downstream consumers. It will + * Fan-out the stream to another subscriber. Each element is produced to + * the `other` subscriber as well as to downstream subscribers. It will * not shutdown until the subscriptions for `other` and at least - * one downstream consumer have been established. + * one downstream subscriber have been established. */ - def tee(other: Consumer[_ >: Out]): Duct[In, Out] + def tee(other: Subscriber[_ >: Out]): Duct[In, Out] /** * Transforms a stream of streams into a contiguous stream of elements using the provided flattening strategy. - * This operation can be used on a stream of element type [[Producer]]. + * This operation can be used on a stream of element type [[Publisher]]. */ def flatten[U](strategy: FlattenStrategy[Out, U]): Duct[In, U] @@ -241,9 +240,9 @@ abstract class Duct[In, Out] { def append[U](duct: Duct[_ >: Out, U]): Duct[In, U] /** - * Allows a faster upstream to progress independently of a slower consumer by conflating elements into a summary - * until the consumer is ready to accept them. For example a conflate step might average incoming numbers if the - * upstream producer is faster. + * Allows a faster upstream to progress independently of a slower subscriber by conflating elements into a summary + * until the subscriber is ready to accept them. For example a conflate step might average incoming numbers if the + * upstream publisher is faster. * * This element only rolls up elements if the upstream is faster, but if the downstream is faster it will not * duplicate elements. @@ -254,13 +253,13 @@ abstract class Duct[In, Out] { def conflate[S](seed: Function[Out, S], aggregate: Function2[S, Out, S]): Duct[In, S] /** - * Allows a faster downstream to progress independently of a slower producer by extrapolating elements from an older + * Allows a faster downstream to progress independently of a slower publisher by extrapolating elements from an older * element until new element comes from the upstream. For example an expand step might repeat the last element for - * the consumer until it receives an update from upstream. + * the subscriber until it receives an update from upstream. * * This element will never "drop" upstream elements as all elements go through at least one extrapolation step. * This means that if the upstream is actually faster than the upstream it will be backpressured by the downstream - * consumer. + * subscriber. * * @param seed Provides the first state for extrapolation using the first unconsumed element * @param extrapolate Takes the current extrapolation state to produce an output element and the next extrapolation @@ -279,52 +278,52 @@ abstract class Duct[In, Out] { def buffer(size: Int, overflowStrategy: OverflowStrategy): Duct[In, Out] /** - * Materialize this `Duct` by attaching it to the specified downstream `consumer` - * and return a `Consumer` representing the input side of the `Duct`. - * The returned `Consumer` can later be connected to an upstream `Producer`. + * Materialize this `Duct` by attaching it to the specified downstream `subscriber` + * and return a `Subscriber` representing the input side of the `Duct`. + * The returned `Subscriber` can later be connected to an upstream `Publisher`. * * *This will materialize the flow and initiate its execution.* * * The given FlowMaterializer decides how the flow’s logical structure is * broken down into individual processing steps. */ - def produceTo(materializer: FlowMaterializer, consumer: Consumer[Out]): Consumer[In] + def produceTo(materializer: FlowMaterializer, subscriber: Subscriber[Out]): Subscriber[In] /** - * Attaches a consumer to this stream which will just discard all received - * elements. The returned `Consumer` represents the input side of the `Duct` and can - * later be connected to an upstream `Producer`. + * Attaches a subscriber to this stream which will just discard all received + * elements. The returned `Subscriber` represents the input side of the `Duct` and can + * later be connected to an upstream `Publisher`. * * *This will materialize the flow and initiate its execution.* * * The given FlowMaterializer decides how the flow’s logical structure is * broken down into individual processing steps. */ - def consume(materializer: FlowMaterializer): Consumer[In] + def consume(materializer: FlowMaterializer): Subscriber[In] /** * When this flow is completed, either through an error or normal * completion, apply the provided function with [[scala.util.Success]] - * or [[scala.util.Failure]]. The returned `Consumer` represents the input side of - * the `Duct` and can later be connected to an upstream `Producer`. + * or [[scala.util.Failure]]. The returned `Subscriber` represents the input side of + * the `Duct` and can later be connected to an upstream `Publisher`. * * *This operation materializes the flow and initiates its execution.* */ - def onComplete(materializer: FlowMaterializer)(callback: OnCompleteCallback): Consumer[In] + def onComplete(materializer: FlowMaterializer)(callback: OnCompleteCallback): Subscriber[In] /** - * Materialize this `Duct` into a `Consumer` representing the input side of the `Duct` - * and a `Producer`representing the output side of the the `Duct`. + * Materialize this `Duct` into a `Subscriber` representing the input side of the `Duct` + * and a `Publisher` representing the output side of the the `Duct`. * - * The returned `Producer` can later be connected to an downstream `Consumer`. - * The returned `Consumer` can later be connected to an upstream `Producer`. + * The returned `Publisher` can later be connected to an downstream `Subscriber`. + * The returned `Subscriber` can later be connected to an upstream `Publisher`. * * *This will materialize the flow and initiate its execution.* * * The given FlowMaterializer decides how the flow’s logical structure is * broken down into individual processing steps. */ - def build(materializer: FlowMaterializer): Pair[Consumer[In], Producer[Out]] + def build(materializer: FlowMaterializer): Pair[Subscriber[In], Publisher[Out]] /** * INTERNAL API @@ -376,25 +375,25 @@ private[akka] class DuctAdapter[In, T](delegate: SDuct[In, T]) extends Duct[In, * Takes up to n elements from the stream and returns a pair containing a strict sequence of the taken element * and a stream representing the remaining elements. */ - override def prefixAndTail(n: Int): Duct[In, Pair[java.util.List[T], Producer[T]]] = + override def prefixAndTail(n: Int): Duct[In, Pair[java.util.List[T], Publisher[T]]] = new DuctAdapter(delegate.prefixAndTail(n).map { case (taken, tail) ⇒ Pair(taken.asJava, tail) }) - override def groupBy[K](f: Function[T, K]): Duct[In, Pair[K, Producer[T]]] = + override def groupBy[K](f: Function[T, K]): Duct[In, Pair[K, Publisher[T]]] = new DuctAdapter(delegate.groupBy(f.apply).map { case (k, p) ⇒ Pair(k, p) }) // FIXME optimize to one step - override def splitWhen(p: Predicate[T]): Duct[In, Producer[T]] = + override def splitWhen(p: Predicate[T]): Duct[In, Publisher[T]] = new DuctAdapter(delegate.splitWhen(p.test)) - override def merge[U >: T](other: Producer[U]): Duct[In, U] = + override def merge[U >: T](other: Publisher[U]): Duct[In, U] = new DuctAdapter(delegate.merge(other)) - override def zip[U](other: Producer[U]): Duct[In, Pair[T, U]] = + override def zip[U](other: Publisher[U]): Duct[In, Pair[T, U]] = new DuctAdapter(delegate.zip(other).map { case (k, p) ⇒ Pair(k, p) }) // FIXME optimize to one step - override def concat[U >: T](next: Producer[U]): Duct[In, U] = + override def concat[U >: T](next: Publisher[U]): Duct[In, U] = new DuctAdapter(delegate.concat(next)) - override def tee(other: Consumer[_ >: T]): Duct[In, T] = + override def tee(other: Subscriber[_ >: T]): Duct[In, T] = new DuctAdapter(delegate.tee(other)) override def buffer(size: Int, overflowStrategy: OverflowStrategy): Duct[In, T] = @@ -415,19 +414,19 @@ private[akka] class DuctAdapter[In, T](delegate: SDuct[In, T]) extends Duct[In, override def append[U](duct: Duct[_ >: T, U]): Duct[In, U] = new DuctAdapter(delegate.appendJava(duct)) - override def produceTo(materializer: FlowMaterializer, consumer: Consumer[T]): Consumer[In] = - delegate.produceTo(materializer, consumer) + override def produceTo(materializer: FlowMaterializer, subscriber: Subscriber[T]): Subscriber[In] = + delegate.produceTo(materializer, subscriber) - override def consume(materializer: FlowMaterializer): Consumer[In] = + override def consume(materializer: FlowMaterializer): Subscriber[In] = delegate.consume(materializer) - override def onComplete(materializer: FlowMaterializer)(callback: OnCompleteCallback): Consumer[In] = + override def onComplete(materializer: FlowMaterializer)(callback: OnCompleteCallback): Subscriber[In] = delegate.onComplete(materializer) { case Success(_) ⇒ callback.onComplete(null) case Failure(e) ⇒ callback.onComplete(e) } - override def build(materializer: FlowMaterializer): Pair[Consumer[In], Producer[T]] = { + override def build(materializer: FlowMaterializer): Pair[Subscriber[In], Publisher[T]] = { val (in, out) = delegate.build(materializer) Pair(in, out) } diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala index ef5a33495d..f1599d79ec 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala @@ -9,7 +9,7 @@ import scala.collection.immutable import scala.concurrent.Future import scala.util.Failure import scala.util.Success -import org.reactivestreams.api.Producer +import org.reactivestreams.{ Publisher, Subscriber } import akka.japi.Function import akka.japi.Function2 import akka.japi.Pair @@ -18,7 +18,6 @@ import akka.japi.Procedure import akka.japi.Util.immutableSeq import akka.stream.{ FlattenStrategy, OverflowStrategy, FlowMaterializer, Transformer } import akka.stream.scaladsl.{ Flow ⇒ SFlow } -import org.reactivestreams.api.Consumer import scala.concurrent.duration.FiniteDuration /** @@ -27,12 +26,12 @@ import scala.concurrent.duration.FiniteDuration object Flow { /** - * Construct a transformation of the given producer. The transformation steps - * are executed by a series of [[org.reactivestreams.api.Processor]] instances + * Construct a transformation of the given publisher. The transformation steps + * are executed by a series of [[org.reactivestreams.Processor]] instances * that mediate the flow of elements downstream and the propagation of * back-pressure upstream. */ - def create[T](producer: Producer[T]): Flow[T] = new FlowAdapter(SFlow.apply(producer)) + def create[T](publisher: Publisher[T]): Flow[T] = new FlowAdapter(SFlow.apply(publisher)) /** * Start a new flow from the given Iterator. The produced stream of elements @@ -46,7 +45,7 @@ object Flow { /** * Start a new flow from the given Iterable. This is like starting from an - * Iterator, but every Consumer directly attached to the Producer of this + * Iterator, but every Subscriber directly attached to the Publisher of this * stream will see an individual flow of elements (always starting from the * beginning) regardless of when they subscribed. */ @@ -67,8 +66,8 @@ object Flow { /** * Elements are produced from the tick `Callable` periodically with the specified interval. - * The tick element will be delivered to downstream consumers that has requested any elements. - * If a consumer has not requested any elements at the point in time when the tick + * The tick element will be delivered to downstream subscribers that has requested any elements. + * If a subscriber has not requested any elements at the point in time when the tick * element is produced it will not receive that tick element later. It will * receive new tick elements as soon as it has requested more elements. */ @@ -80,14 +79,14 @@ object Flow { /** * Java API: The Flow DSL allows the formulation of stream transformations based on some * input. The starting point can be a collection, an iterator, a block of code - * which is evaluated repeatedly or a [[org.reactivestreams.api.Producer]]. + * which is evaluated repeatedly or a [[org.reactivestreams.Publisher]]. * * See Reactive Streams for details. * * Each DSL element produces a new Flow that can be further transformed, building * up a description of the complete transformation pipeline. In order to execute * this pipeline the Flow must be materialized by calling the [[#toFuture]], [[#consume]], - * [[#onComplete]], or [[#toProducer]] methods on it. + * [[#onComplete]], or [[#toPublisher]] methods on it. * * It should be noted that the streams modeled by this library are “hot”, * meaning that they asynchronously flow through a series of processors without @@ -103,7 +102,7 @@ object Flow { * to enable full pipelining of the chained set of computations. This behavior * is determined by the [[akka.stream.FlowMaterializer]] which is required * by those methods that materialize the Flow into a series of - * [[org.reactivestreams.api.Processor]] instances. The returned reactive stream + * [[org.reactivestreams.Processor]] instances. The returned reactive stream * is fully started and active. */ abstract class Flow[T] { @@ -164,17 +163,17 @@ abstract class Flow[T] { def dropWithin(d: FiniteDuration): Flow[T] /** - * Terminate processing (and cancel the upstream producer) after the given + * Terminate processing (and cancel the upstream publisher) after the given * number of elements. Due to input buffering some elements may have been - * requested from upstream producers that will then not be processed downstream + * requested from upstream publishers that will then not be processed downstream * of this step. */ def take(n: Int): Flow[T] /** - * Terminate processing (and cancel the upstream producer) after the given + * Terminate processing (and cancel the upstream publisher) after the given * duration. Due to input buffering some elements may have been - * requested from upstream producers that will then not be processed downstream + * requested from upstream publishers that will then not be processed downstream * of this step. * * Note that this can be combined with [[#take]] to limit the number of elements @@ -208,9 +207,9 @@ abstract class Flow[T] { * function is invoked and expecting a (possibly empty) sequence of output elements * to be produced. * After handing off the elements produced from one input element to the downstream - * consumers, the [[akka.stream.Transformer#isComplete]] predicate determines whether to end + * subscribers, the [[akka.stream.Transformer#isComplete]] predicate determines whether to end * stream processing at this point; in that case the upstream subscription is - * canceled. Before signaling normal completion to the downstream consumers, + * canceled. Before signaling normal completion to the downstream subscribers, * the [[akka.stream.Transformer#onComplete]] function is invoked to produce a (possibly empty) * sequence of elements in response to the end-of-stream event. * @@ -231,20 +230,20 @@ abstract class Flow[T] { * and a stream representing the remaining elements. If ''n'' is zero or negative, then this will return a pair * of an empty collection and a stream containing the whole upstream unchanged. */ - def prefixAndTail(n: Int): Flow[Pair[java.util.List[T], Producer[T]]] + def prefixAndTail(n: Int): Flow[Pair[java.util.List[T], Publisher[T]]] /** * This operation demultiplexes the incoming stream into separate output * streams, one for each element key. The key is computed for each element * using the given function. When a new key is encountered for the first time - * it is emitted to the downstream consumer together with a fresh - * producer that will eventually produce all the elements of the substream + * it is emitted to the downstream subscriber together with a fresh + * publisher that will eventually produce all the elements of the substream * for that key. Not consuming the elements from the created streams will * stop this processor from processing more elements, therefore you must take * care to unblock (or cancel) all of the produced streams even if you want * to consume only one of them. */ - def groupBy[K](f: Function[T, K]): Flow[Pair[K, Producer[T]]] + def groupBy[K](f: Function[T, K]): Flow[Pair[K, Publisher[T]]] /** * This operation applies the given predicate to all incoming elements and @@ -259,36 +258,36 @@ abstract class Flow[T] { * true, false, false // elements go into third substream * }}} */ - def splitWhen(p: Predicate[T]): Flow[Producer[T]] + def splitWhen(p: Predicate[T]): Flow[Publisher[T]] /** - * Merge this stream with the one emitted by the given producer, taking + * Merge this stream with the one emitted by the given publisher, taking * elements as they arrive from either side (picking randomly when both * have elements ready). */ - def merge[U >: T](other: Producer[U]): Flow[U] + def merge[U >: T](other: Publisher[U]): Flow[U] /** - * Zip this stream together with the one emitted by the given producer. + * Zip this stream together with the one emitted by the given publisher. * This transformation finishes when either input stream reaches its end, * cancelling the subscription to the other one. */ - def zip[U](other: Producer[U]): Flow[Pair[T, U]] + def zip[U](other: Publisher[U]): Flow[Pair[T, U]] /** * Concatenate the given other stream to this stream so that the first element - * emitted by the given producer is emitted after the last element of this + * emitted by the given publisher is emitted after the last element of this * stream. */ - def concat[U >: T](next: Producer[U]): Flow[U] + def concat[U >: T](next: Publisher[U]): Flow[U] /** - * Fan-out the stream to another consumer. Each element is produced to - * the `other` consumer as well as to downstream consumers. It will + * Fan-out the stream to another subscriber. Each element is produced to + * the `other` subscriber as well as to downstream subscribers. It will * not shutdown until the subscriptions for `other` and at least - * one downstream consumer have been established. + * one downstream subscriber have been established. */ - def tee(other: Consumer[_ >: T]): Flow[T] + def tee(other: Subscriber[_ >: T]): Flow[T] /** * Append the operations of a [[Duct]] to this flow. @@ -297,14 +296,14 @@ abstract class Flow[T] { /** * Transforms a stream of streams into a contiguous stream of elements using the provided flattening strategy. - * This operation can be used on a stream of element type [[Producer]]. + * This operation can be used on a stream of element type [[Publisher]]. */ def flatten[U](strategy: FlattenStrategy[T, U]): Flow[U] /** - * Allows a faster upstream to progress independently of a slower consumer by conflating elements into a summary - * until the consumer is ready to accept them. For example a conflate step might average incoming numbers if the - * upstream producer is faster. + * Allows a faster upstream to progress independently of a slower subscriber by conflating elements into a summary + * until the subscriber is ready to accept them. For example a conflate step might average incoming numbers if the + * upstream publisher is faster. * * This element only rolls up elements if the upstream is faster, but if the downstream is faster it will not * duplicate elements. @@ -315,13 +314,13 @@ abstract class Flow[T] { def conflate[S](seed: Function[T, S], aggregate: Function2[S, T, S]): Flow[S] /** - * Allows a faster downstream to progress independently of a slower producer by extrapolating elements from an older + * Allows a faster downstream to progress independently of a slower publisher by extrapolating elements from an older * element until new element comes from the upstream. For example an expand step might repeat the last element for - * the consumer until it receives an update from upstream. + * the subscriber until it receives an update from upstream. * * This element will never "drop" upstream elements as all elements go through at least one extrapolation step. * This means that if the upstream is actually faster than the upstream it will be backpressured by the downstream - * consumer. + * subscriber. * * @param seed Provides the first state for extrapolation using the first unconsumed element * @param extrapolate Takes the current extrapolation state to produce an output element and the next extrapolation @@ -353,7 +352,7 @@ abstract class Flow[T] { def toFuture(materializer: FlowMaterializer): Future[T] /** - * Attaches a consumer to this stream which will just discard all received + * Attaches a subscriber to this stream which will just discard all received * elements. *This will materialize the flow and initiate its execution.* * * The given FlowMaterializer decides how the flow’s logical structure is @@ -371,25 +370,25 @@ abstract class Flow[T] { /** * Materialize this flow and return the downstream-most - * [[org.reactivestreams.api.Producer]] interface. The stream will not have - * any consumers attached at this point, which means that after prefetching + * [[org.reactivestreams.Publisher]] interface. The stream will not have + * any subscribers attached at this point, which means that after prefetching * elements to fill the internal buffers it will assert back-pressure until - * a consumer connects and creates demand for elements to be emitted. + * a subscriber connects and creates demand for elements to be emitted. * * The given FlowMaterializer decides how the flow’s logical structure is * broken down into individual processing steps. */ - def toProducer(materializer: FlowMaterializer): Producer[T] + def toPublisher(materializer: FlowMaterializer): Publisher[T] /** - * Attaches a consumer to this stream. + * Attaches a subscriber to this stream. * * *This will materialize the flow and initiate its execution.* * * The given FlowMaterializer decides how the flow’s logical structure is * broken down into individual processing steps. */ - def produceTo(materializer: FlowMaterializer, consumer: Consumer[_ >: T]): Unit + def produceTo(materializer: FlowMaterializer, subscriber: Subscriber[_ >: T]): Unit } @@ -443,25 +442,25 @@ private[akka] class FlowAdapter[T](delegate: SFlow[T]) extends Flow[T] { override def transform[U](transformer: Transformer[T, U]): Flow[U] = new FlowAdapter(delegate.transform(transformer)) - override def prefixAndTail(n: Int): Flow[Pair[java.util.List[T], Producer[T]]] = + override def prefixAndTail(n: Int): Flow[Pair[java.util.List[T], Publisher[T]]] = new FlowAdapter(delegate.prefixAndTail(n).map { case (taken, tail) ⇒ Pair(taken.asJava, tail) }) - override def groupBy[K](f: Function[T, K]): Flow[Pair[K, Producer[T]]] = + override def groupBy[K](f: Function[T, K]): Flow[Pair[K, Publisher[T]]] = new FlowAdapter(delegate.groupBy(f.apply).map { case (k, p) ⇒ Pair(k, p) }) // FIXME optimize to one step - override def splitWhen(p: Predicate[T]): Flow[Producer[T]] = + override def splitWhen(p: Predicate[T]): Flow[Publisher[T]] = new FlowAdapter(delegate.splitWhen(p.test)) - override def merge[U >: T](other: Producer[U]): Flow[U] = + override def merge[U >: T](other: Publisher[U]): Flow[U] = new FlowAdapter(delegate.merge(other)) - override def zip[U](other: Producer[U]): Flow[Pair[T, U]] = + override def zip[U](other: Publisher[U]): Flow[Pair[T, U]] = new FlowAdapter(delegate.zip(other).map { case (k, p) ⇒ Pair(k, p) }) // FIXME optimize to one step - override def concat[U >: T](next: Producer[U]): Flow[U] = + override def concat[U >: T](next: Publisher[U]): Flow[U] = new FlowAdapter(delegate.concat(next)) - override def tee(other: Consumer[_ >: T]): Flow[T] = + override def tee(other: Subscriber[_ >: T]): Flow[T] = new FlowAdapter(delegate.tee(other)) override def flatten[U](strategy: FlattenStrategy[T, U]): Flow[U] = @@ -494,10 +493,10 @@ private[akka] class FlowAdapter[T](delegate: SFlow[T]) extends Flow[T] { case Failure(e) ⇒ callback.onComplete(e) } - override def toProducer(materializer: FlowMaterializer): Producer[T] = - delegate.toProducer(materializer) + override def toPublisher(materializer: FlowMaterializer): Publisher[T] = + delegate.toPublisher(materializer) - override def produceTo(materializer: FlowMaterializer, consumer: Consumer[_ >: T]): Unit = - delegate.produceTo(materializer, consumer) + override def produceTo(materializer: FlowMaterializer, subsriber: Subscriber[_ >: T]): Unit = + delegate.produceTo(materializer, subsriber) } \ No newline at end of file diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Duct.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Duct.scala index 32fc5171d3..6315c1095a 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Duct.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Duct.scala @@ -6,8 +6,7 @@ package akka.stream.scaladsl import scala.annotation.unchecked.uncheckedVariance import scala.collection.immutable import scala.util.Try -import org.reactivestreams.api.Consumer -import org.reactivestreams.api.Producer +import org.reactivestreams.{ Publisher, Subscriber } import akka.stream.{ FlattenStrategy, OverflowStrategy, FlowMaterializer, Transformer } import akka.stream.impl.DuctImpl import akka.stream.impl.Ast @@ -20,7 +19,7 @@ object Duct { /** * Create an empty [[Duct]]. The transformation steps are executed by a series - * of [[org.reactivestreams.api.Processor]] instances that mediate the flow of + * of [[org.reactivestreams.Processor]] instances that mediate the flow of * elements downstream and the propagation of back-pressure upstream. */ def apply[In]: Duct[In, In] = empty.asInstanceOf[Duct[In, In]] @@ -32,8 +31,8 @@ object Duct { * The difference is that it is not attached to an input source. * * The pipeline must be materialized by calling the [[#produceTo]], [[#consume]] or [[#build]] - * methods on it and then attach the `Consumer` representing the input side of the `Duct` to an - * upstream `Producer`. + * methods on it and then attach the `Subscriber` representing the input side of the `Duct` to an + * upstream `Publisher`. * */ trait Duct[In, +Out] { @@ -68,8 +67,8 @@ trait Duct[In, +Out] { * Invoke the given procedure for each received element and produce a Unit value * upon reaching the normal end of the stream. Please note that also in this case * the `Duct` needs to be materialized (e.g. using [[#consume]] and attaching the - * the `Consumer` representing the input side of the `Duct` to an upstream - * `Producer`) to initiate its execution. + * the `Subscriber` representing the input side of the `Duct` to an upstream + * `Publisher`) to initiate its execution. */ def foreach(c: Out ⇒ Unit): Duct[In, Unit] @@ -92,17 +91,17 @@ trait Duct[In, +Out] { def dropWithin(d: FiniteDuration): Duct[In, Out] /** - * Terminate processing (and cancel the upstream producer) after the given + * Terminate processing (and cancel the upstream publisher) after the given * number of elements. Due to input buffering some elements may have been - * requested from upstream producers that will then not be processed downstream + * requested from upstream publishers that will then not be processed downstream * of this step. */ def take(n: Int): Duct[In, Out] /** - * Terminate processing (and cancel the upstream producer) after the given + * Terminate processing (and cancel the upstream publisher) after the given * duration. Due to input buffering some elements may have been - * requested from upstream producers that will then not be processed downstream + * requested from upstream publishers that will then not be processed downstream * of this step. * * Note that this can be combined with [[#take]] to limit the number of elements @@ -136,9 +135,9 @@ trait Duct[In, +Out] { * function is invoked and expecting a (possibly empty) sequence of output elements * to be produced. * After handing off the elements produced from one input element to the downstream - * consumers, the [[Transformer#isComplete]] predicate determines whether to end + * subscribers, the [[Transformer#isComplete]] predicate determines whether to end * stream processing at this point; in that case the upstream subscription is - * canceled. Before signaling normal completion to the downstream consumers, + * canceled. Before signaling normal completion to the downstream subscribers, * the [[Transformer#onComplete]] function is invoked to produce a (possibly empty) * sequence of elements in response to the end-of-stream event. * @@ -159,20 +158,20 @@ trait Duct[In, +Out] { * and a stream representing the remaining elements. If ''n'' is zero or negative, then this will return a pair * of an empty collection and a stream containing the whole upstream unchanged. */ - def prefixAndTail(n: Int): Duct[In, (immutable.Seq[Out], Producer[Out @uncheckedVariance])] + def prefixAndTail(n: Int): Duct[In, (immutable.Seq[Out], Publisher[Out @uncheckedVariance])] /** * This operation demultiplexes the incoming stream into separate output * streams, one for each element key. The key is computed for each element * using the given function. When a new key is encountered for the first time - * it is emitted to the downstream consumer together with a fresh - * producer that will eventually produce all the elements of the substream + * it is emitted to the downstream subscriber together with a fresh + * publisher that will eventually produce all the elements of the substream * for that key. Not consuming the elements from the created streams will * stop this processor from processing more elements, therefore you must take * care to unblock (or cancel) all of the produced streams even if you want * to consume only one of them. */ - def groupBy[K](f: Out ⇒ K): Duct[In, (K, Producer[Out @uncheckedVariance])] + def groupBy[K](f: Out ⇒ K): Duct[In, (K, Publisher[Out @uncheckedVariance])] /** * This operation applies the given predicate to all incoming elements and @@ -187,47 +186,47 @@ trait Duct[In, +Out] { * true, false, false // elements go into third substream * }}} */ - def splitWhen(p: Out ⇒ Boolean): Duct[In, Producer[Out @uncheckedVariance]] + def splitWhen(p: Out ⇒ Boolean): Duct[In, Publisher[Out @uncheckedVariance]] /** - * Merge this stream with the one emitted by the given producer, taking + * Merge this stream with the one emitted by the given publisher, taking * elements as they arrive from either side (picking randomly when both * have elements ready). */ - def merge[U >: Out](other: Producer[_ <: U]): Duct[In, U] + def merge[U >: Out](other: Publisher[_ <: U]): Duct[In, U] /** - * Zip this stream together with the one emitted by the given producer. + * Zip this stream together with the one emitted by the given publisher. * This transformation finishes when either input stream reaches its end, * cancelling the subscription to the other one. */ - def zip[U](other: Producer[U]): Duct[In, (Out, U)] + def zip[U](other: Publisher[U]): Duct[In, (Out, U)] /** * Concatenate the given other stream to this stream so that the first element - * emitted by the given producer is emitted after the last element of this + * emitted by the given publisher is emitted after the last element of this * stream. */ - def concat[U >: Out](next: Producer[U]): Duct[In, U] + def concat[U >: Out](next: Publisher[U]): Duct[In, U] /** - * Fan-out the stream to another consumer. Each element is produced to - * the `other` consumer as well as to downstream consumers. It will + * Fan-out the stream to another subscriber. Each element is produced to + * the `other` subscriber as well as to downstream subscribers. It will * not shutdown until the subscriptions for `other` and at least - * one downstream consumer have been established. + * one downstream subscriber have been established. */ - def tee(other: Consumer[_ >: Out]): Duct[In, Out] + def tee(other: Subscriber[_ >: Out]): Duct[In, Out] /** * Transforms a stream of streams into a contiguous stream of elements using the provided flattening strategy. - * This operation can be used on a stream of element type [[Producer]]. + * This operation can be used on a stream of element type [[Publisher]]. */ def flatten[U](strategy: FlattenStrategy[Out, U]): Duct[In, U] /** - * Allows a faster upstream to progress independently of a slower consumer by conflating elements into a summary - * until the consumer is ready to accept them. For example a conflate step might average incoming numbers if the - * upstream producer is faster. + * Allows a faster upstream to progress independently of a slower subscriber by conflating elements into a summary + * until the subscriber is ready to accept them. For example a conflate step might average incoming numbers if the + * upstream publisher is faster. * * This element only rolls up elements if the upstream is faster, but if the downstream is faster it will not * duplicate elements. @@ -238,13 +237,13 @@ trait Duct[In, +Out] { def conflate[S](seed: Out ⇒ S, aggregate: (S, Out) ⇒ S): Duct[In, S] /** - * Allows a faster downstream to progress independently of a slower producer by extrapolating elements from an older + * Allows a faster downstream to progress independently of a slower publisher by extrapolating elements from an older * element until new element comes from the upstream. For example an expand step might repeat the last element for - * the consumer until it receives an update from upstream. + * the subscriber until it receives an update from upstream. * * This element will never "drop" upstream elements as all elements go through at least one extrapolation step. * This means that if the upstream is actually faster than the upstream it will be backpressured by the downstream - * consumer. + * subscriber. * * @param seed Provides the first state for extrapolation using the first unconsumed element * @param extrapolate Takes the current extrapolation state to produce an output element and the next extrapolation @@ -273,52 +272,52 @@ trait Duct[In, +Out] { private[akka] def appendJava[U](duct: akka.stream.javadsl.Duct[_ >: Out, U]): Duct[In, U] /** - * Materialize this `Duct` by attaching it to the specified downstream `consumer` - * and return a `Consumer` representing the input side of the `Duct`. - * The returned `Consumer` can later be connected to an upstream `Producer`. + * Materialize this `Duct` by attaching it to the specified downstream `subscriber` + * and return a `Subscriber` representing the input side of the `Duct`. + * The returned `Subscriber` can later be connected to an upstream `Publisher`. * * *This will materialize the flow and initiate its execution.* * * The given FlowMaterializer decides how the flow’s logical structure is * broken down into individual processing steps. */ - def produceTo(materializer: FlowMaterializer, consumer: Consumer[Out] @uncheckedVariance): Consumer[In] + def produceTo(materializer: FlowMaterializer, subscriber: Subscriber[Out] @uncheckedVariance): Subscriber[In] /** - * Attaches a consumer to this stream which will just discard all received - * elements. The returned `Consumer` represents the input side of the `Duct` and can - * later be connected to an upstream `Producer`. + * Attaches a subscriber to this stream which will just discard all received + * elements. The returned `Subscriber` represents the input side of the `Duct` and can + * later be connected to an upstream `Publisher`. * * *This will materialize the flow and initiate its execution.* * * The given FlowMaterializer decides how the flow’s logical structure is * broken down into individual processing steps. */ - def consume(materializer: FlowMaterializer): Consumer[In] + def consume(materializer: FlowMaterializer): Subscriber[In] /** * When this flow is completed, either through an error or normal * completion, apply the provided function with [[scala.util.Success]] - * or [[scala.util.Failure]]. The returned `Consumer` represents the input side of - * the `Duct` and can later be connected to an upstream `Producer`. + * or [[scala.util.Failure]]. The returned `Subscriber` represents the input side of + * the `Duct` and can later be connected to an upstream `Publisher`. * * *This operation materializes the flow and initiates its execution.* */ - def onComplete(materializer: FlowMaterializer)(callback: Try[Unit] ⇒ Unit): Consumer[In] + def onComplete(materializer: FlowMaterializer)(callback: Try[Unit] ⇒ Unit): Subscriber[In] /** - * Materialize this `Duct` into a `Consumer` representing the input side of the `Duct` - * and a `Producer`representing the output side of the the `Duct`. + * Materialize this `Duct` into a `Subscriber` representing the input side of the `Duct` + * and a `Publisher`representing the output side of the the `Duct`. * - * The returned `Producer` can later be connected to an downstream `Consumer`. - * The returned `Consumer` can later be connected to an upstream `Producer`. + * The returned `Publisher` can later be connected to an downstream `Subscriber`. + * The returned `Subscriber` can later be connected to an upstream `Publisher`. * * *This will materialize the flow and initiate its execution.* * * The given FlowMaterializer decides how the flow’s logical structure is * broken down into individual processing steps. */ - def build(materializer: FlowMaterializer): (Consumer[In], Producer[Out] @uncheckedVariance) + def build(materializer: FlowMaterializer): (Subscriber[In], Publisher[Out] @uncheckedVariance) /** * INTERNAL API diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala index 384f113fcf..0bdee671ac 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala @@ -7,13 +7,12 @@ import scala.annotation.unchecked.uncheckedVariance import scala.collection.immutable import scala.concurrent.Future import scala.util.Try -import org.reactivestreams.api.Consumer -import org.reactivestreams.api.Producer +import org.reactivestreams.{ Publisher, Subscriber } import akka.stream.{ FlattenStrategy, OverflowStrategy, FlowMaterializer, Transformer } -import akka.stream.impl.Ast.{ ExistingProducer, IterableProducerNode, IteratorProducerNode, ThunkProducerNode } -import akka.stream.impl.Ast.FutureProducerNode +import akka.stream.impl.Ast.{ ExistingPublisher, IterablePublisherNode, IteratorPublisherNode, ThunkPublisherNode } +import akka.stream.impl.Ast.FuturePublisherNode import akka.stream.impl.FlowImpl -import akka.stream.impl.Ast.TickProducerNode +import akka.stream.impl.Ast.TickPublisherNode import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration @@ -22,12 +21,12 @@ import scala.concurrent.duration.FiniteDuration */ object Flow { /** - * Construct a transformation of the given producer. The transformation steps - * are executed by a series of [[org.reactivestreams.api.Processor]] instances + * Construct a transformation of the given publisher. The transformation steps + * are executed by a series of [[org.reactivestreams.Processor]] instances * that mediate the flow of elements downstream and the propagation of * back-pressure upstream. */ - def apply[T](producer: Producer[T]): Flow[T] = FlowImpl(ExistingProducer(producer), Nil) + def apply[T](publisher: Publisher[T]): Flow[T] = FlowImpl(ExistingPublisher(publisher), Nil) /** * Start a new flow from the given Iterator. The produced stream of elements @@ -36,15 +35,15 @@ object Flow { * in accordance with the demand coming from the downstream transformation * steps. */ - def apply[T](iterator: Iterator[T]): Flow[T] = FlowImpl(IteratorProducerNode(iterator), Nil) + def apply[T](iterator: Iterator[T]): Flow[T] = FlowImpl(IteratorPublisherNode(iterator), Nil) /** * Start a new flow from the given Iterable. This is like starting from an - * Iterator, but every Consumer directly attached to the Producer of this + * Iterator, but every Subscriber directly attached to the Publisher of this * stream will see an individual flow of elements (always starting from the * beginning) regardless of when they subscribed. */ - def apply[T](iterable: immutable.Iterable[T]): Flow[T] = FlowImpl(IterableProducerNode(iterable), Nil) + def apply[T](iterable: immutable.Iterable[T]): Flow[T] = FlowImpl(IterablePublisherNode(iterable), Nil) /** * Define the sequence of elements to be produced by the given closure. @@ -52,7 +51,7 @@ object Flow { * a [[akka.stream.Stop]] exception being thrown; it ends exceptionally * when any other exception is thrown. */ - def apply[T](f: () ⇒ T): Flow[T] = FlowImpl(ThunkProducerNode(f), Nil) + def apply[T](f: () ⇒ T): Flow[T] = FlowImpl(ThunkPublisherNode(f), Nil) /** * Start a new flow from the given `Future`. The stream will consist of @@ -60,30 +59,30 @@ object Flow { * may happen before or after materializing the `Flow`. * The stream terminates with an error if the `Future` is completed with a failure. */ - def apply[T](future: Future[T]): Flow[T] = FlowImpl(FutureProducerNode(future), Nil) + def apply[T](future: Future[T]): Flow[T] = FlowImpl(FuturePublisherNode(future), Nil) /** * Elements are produced from the tick closure periodically with the specified interval. - * The tick element will be delivered to downstream consumers that has requested any elements. - * If a consumer has not requested any elements at the point in time when the tick + * The tick element will be delivered to downstream subscribers that has requested any elements. + * If a subscriber has not requested any elements at the point in time when the tick * element is produced it will not receive that tick element later. It will * receive new tick elements as soon as it has requested more elements. */ - def apply[T](interval: FiniteDuration, tick: () ⇒ T): Flow[T] = FlowImpl(TickProducerNode(interval, tick), Nil) + def apply[T](interval: FiniteDuration, tick: () ⇒ T): Flow[T] = FlowImpl(TickPublisherNode(interval, tick), Nil) } /** * Scala API: The Flow DSL allows the formulation of stream transformations based on some * input. The starting point can be a collection, an iterator, a block of code - * which is evaluated repeatedly or a [[org.reactivestreams.api.Producer]]. + * which is evaluated repeatedly or a [[org.reactivestreams.Publisher]]. * * See Reactive Streams for details. * * Each DSL element produces a new Flow that can be further transformed, building * up a description of the complete transformation pipeline. In order to execute * this pipeline the Flow must be materialized by calling the [[#toFuture]], [[#consume]], - * [[#onComplete]], or [[#toProducer]] methods on it. + * [[#onComplete]], or [[#toPublisher]] methods on it. * * It should be noted that the streams modeled by this library are “hot”, * meaning that they asynchronously flow through a series of processors without @@ -99,7 +98,7 @@ object Flow { * to enable full pipelining of the chained set of computations. This behavior * is determined by the [[akka.stream.FlowMaterializer]] which is required * by those methods that materialize the Flow into a series of - * [[org.reactivestreams.api.Processor]] instances. The returned reactive stream + * [[org.reactivestreams.Processor]] instances. The returned reactive stream * is fully started and active. */ trait Flow[+T] { @@ -158,17 +157,17 @@ trait Flow[+T] { def dropWithin(d: FiniteDuration): Flow[T] /** - * Terminate processing (and cancel the upstream producer) after the given + * Terminate processing (and cancel the upstream publisher) after the given * number of elements. Due to input buffering some elements may have been - * requested from upstream producers that will then not be processed downstream + * requested from upstream publishers that will then not be processed downstream * of this step. */ def take(n: Int): Flow[T] /** - * Terminate processing (and cancel the upstream producer) after the given + * Terminate processing (and cancel the upstream publisher) after the given * duration. Due to input buffering some elements may have been - * requested from upstream producers that will then not be processed downstream + * requested from upstream publishers that will then not be processed downstream * of this step. * * Note that this can be combined with [[#take]] to limit the number of elements @@ -202,9 +201,9 @@ trait Flow[+T] { * function is invoked, expecting a (possibly empty) sequence of output elements * to be produced. * After handing off the elements produced from one input element to the downstream - * consumers, the [[akka.stream.Transformer#isComplete]] predicate determines whether to end + * subscribers, the [[akka.stream.Transformer#isComplete]] predicate determines whether to end * stream processing at this point; in that case the upstream subscription is - * canceled. Before signaling normal completion to the downstream consumers, + * canceled. Before signaling normal completion to the downstream subscribers, * the [[akka.stream.Transformer#onComplete]] function is invoked to produce a (possibly empty) * sequence of elements in response to the end-of-stream event. * @@ -227,20 +226,20 @@ trait Flow[+T] { * and a stream representing the remaining elements. If ''n'' is zero or negative, then this will return a pair * of an empty collection and a stream containing the whole upstream unchanged. */ - def prefixAndTail(n: Int): Flow[(immutable.Seq[T], Producer[T @uncheckedVariance])] + def prefixAndTail(n: Int): Flow[(immutable.Seq[T], Publisher[T @uncheckedVariance])] /** * This operation demultiplexes the incoming stream into separate output * streams, one for each element key. The key is computed for each element * using the given function. When a new key is encountered for the first time - * it is emitted to the downstream consumer together with a fresh - * producer that will eventually produce all the elements of the substream + * it is emitted to the downstream subscriber together with a fresh + * publisher that will eventually produce all the elements of the substream * for that key. Not consuming the elements from the created streams will * stop this processor from processing more elements, therefore you must take * care to unblock (or cancel) all of the produced streams even if you want * to consume only one of them. */ - def groupBy[K](f: T ⇒ K): Flow[(K, Producer[T @uncheckedVariance])] + def groupBy[K](f: T ⇒ K): Flow[(K, Publisher[T @uncheckedVariance])] /** * This operation applies the given predicate to all incoming elements and @@ -255,47 +254,47 @@ trait Flow[+T] { * true, false, false // elements go into third substream * }}} */ - def splitWhen(p: T ⇒ Boolean): Flow[Producer[T @uncheckedVariance]] + def splitWhen(p: T ⇒ Boolean): Flow[Publisher[T @uncheckedVariance]] /** - * Merge this stream with the one emitted by the given producer, taking + * Merge this stream with the one emitted by the given publisher, taking * elements as they arrive from either side (picking randomly when both * have elements ready). */ - def merge[U >: T](other: Producer[_ <: U]): Flow[U] + def merge[U >: T](other: Publisher[_ <: U]): Flow[U] /** - * Zip this stream together with the one emitted by the given producer. + * Zip this stream together with the one emitted by the given publisher. * This transformation finishes when either input stream reaches its end, * cancelling the subscription to the other one. */ - def zip[U](other: Producer[U]): Flow[(T, U)] + def zip[U](other: Publisher[U]): Flow[(T, U)] /** * Concatenate the given other stream to this stream so that the first element - * emitted by the given producer is emitted after the last element of this + * emitted by the given publisher is emitted after the last element of this * stream. */ - def concat[U >: T](next: Producer[U]): Flow[U] + def concat[U >: T](next: Publisher[U]): Flow[U] /** - * Fan-out the stream to another consumer. Each element is produced to - * the `other` consumer as well as to downstream consumers. It will + * Fan-out the stream to another subscriber. Each element is produced to + * the `other` subscriber as well as to downstream subscribers. It will * not shutdown until the subscriptions for `other` and at least - * one downstream consumer have been established. + * one downstream subscriber have been established. */ - def tee(other: Consumer[_ >: T]): Flow[T] + def tee(other: Subscriber[_ >: T]): Flow[T] /** * Transforms a stream of streams into a contiguous stream of elements using the provided flattening strategy. - * This operation can be used on a stream of element type [[Producer]]. + * This operation can be used on a stream of element type [[Publisher]]. */ def flatten[U](strategy: FlattenStrategy[T, U]): Flow[U] /** - * Allows a faster upstream to progress independently of a slower consumer by conflating elements into a summary - * until the consumer is ready to accept them. For example a conflate step might average incoming numbers if the - * upstream producer is faster. + * Allows a faster upstream to progress independently of a slower subscriber by conflating elements into a summary + * until the subscriber is ready to accept them. For example a conflate step might average incoming numbers if the + * upstream publisher is faster. * * This element only rolls up elements if the upstream is faster, but if the downstream is faster it will not * duplicate elements. @@ -306,13 +305,13 @@ trait Flow[+T] { def conflate[S](seed: T ⇒ S, aggregate: (S, T) ⇒ S): Flow[S] /** - * Allows a faster downstream to progress independently of a slower producer by extrapolating elements from an older + * Allows a faster downstream to progress independently of a slower publisher by extrapolating elements from an older * element until new element comes from the upstream. For example an expand step might repeat the last element for - * the consumer until it receives an update from upstream. + * the subscriber until it receives an update from upstream. * * This element will never "drop" upstream elements as all elements go through at least one extrapolation step. * This means that if the upstream is actually faster than the upstream it will be backpressured by the downstream - * consumer. + * subscriber. * * @param seed Provides the first state for extrapolation using the first unconsumed element * @param extrapolate Takes the current extrapolation state to produce an output element and the next extrapolation @@ -354,7 +353,7 @@ trait Flow[+T] { def toFuture(materializer: FlowMaterializer): Future[T] /** - * Attaches a consumer to this stream which will just discard all received + * Attaches a subscriber to this stream which will just discard all received * elements. *This will materialize the flow and initiate its execution.* * * The given FlowMaterializer decides how the flow’s logical structure is @@ -373,25 +372,25 @@ trait Flow[+T] { /** * Materialize this flow and return the downstream-most - * [[org.reactivestreams.api.Producer]] interface. The stream will not have - * any consumers attached at this point, which means that after prefetching + * [[org.reactivestreams.Publisher]] interface. The stream will not have + * any subscribers attached at this point, which means that after prefetching * elements to fill the internal buffers it will assert back-pressure until - * a consumer connects and creates demand for elements to be emitted. + * a subscriber connects and creates demand for elements to be emitted. * * The given FlowMaterializer decides how the flow’s logical structure is * broken down into individual processing steps. */ - def toProducer(materializer: FlowMaterializer): Producer[T @uncheckedVariance] + def toPublisher(materializer: FlowMaterializer): Publisher[T @uncheckedVariance] /** - * Attaches a consumer to this stream. + * Attaches a subscriber to this stream. * * *This will materialize the flow and initiate its execution.* * * The given FlowMaterializer decides how the flow’s logical structure is * broken down into individual processing steps. */ - def produceTo(materializer: FlowMaterializer, consumer: Consumer[_ >: T]): Unit + def produceTo(materializer: FlowMaterializer, subscriber: Subscriber[_ >: T]): Unit } diff --git a/akka-stream/src/test/java/akka/stream/javadsl/DuctTest.java b/akka-stream/src/test/java/akka/stream/javadsl/DuctTest.java index 2cf64f19fc..fba9608405 100644 --- a/akka-stream/src/test/java/akka/stream/javadsl/DuctTest.java +++ b/akka-stream/src/test/java/akka/stream/javadsl/DuctTest.java @@ -4,8 +4,8 @@ import java.util.Arrays; import java.util.concurrent.TimeUnit; import org.junit.ClassRule; import org.junit.Test; -import org.reactivestreams.api.Consumer; -import org.reactivestreams.api.Producer; +import org.reactivestreams.Publisher; +import org.reactivestreams.Subscriber; import scala.concurrent.duration.FiniteDuration; import scala.concurrent.Future; import akka.actor.ActorRef; @@ -37,7 +37,7 @@ public class DuctTest { final JavaTestKit probe = new JavaTestKit(system); final String[] lookup = { "a", "b", "c", "d", "e", "f" }; - Consumer inputConsumer = Duct.create(Integer.class).drop(2).take(3).map(new Function() { + Subscriber inputSubscriber = Duct.create(Integer.class).drop(2).take(3).map(new Function() { public String apply(Integer elem) { return lookup[elem]; } @@ -60,17 +60,17 @@ public class DuctTest { }).consume(materializer); final java.util.Iterator input = Arrays.asList(0, 1, 2, 3, 4, 5).iterator(); - Producer producer = Flow.create(input).toProducer(materializer); + Publisher publisher = Flow.create(input).toPublisher(materializer); - producer.produceTo(inputConsumer); + publisher.subscribe(inputSubscriber); probe.expectMsgEquals("de"); } @Test - public void mustMaterializeIntoProducerConsumer() { + public void mustMaterializeIntoPublisherSubscriber() { final JavaTestKit probe = new JavaTestKit(system); - Pair, Producer> inOutPair = Duct.create(String.class).build(materializer); + Pair, Publisher> inOutPair = Duct.create(String.class).build(materializer); Flow.create(inOutPair.second()).foreach(new Procedure() { public void apply(String elem) { @@ -79,29 +79,29 @@ public class DuctTest { }).consume(materializer); probe.expectNoMsg(FiniteDuration.create(200, TimeUnit.MILLISECONDS)); - Producer producer = Flow.create(Arrays.asList("a", "b", "c")).toProducer(materializer); - producer.produceTo(inOutPair.first()); + Publisher publisher = Flow.create(Arrays.asList("a", "b", "c")).toPublisher(materializer); + publisher.subscribe(inOutPair.first()); probe.expectMsgEquals("a"); probe.expectMsgEquals("b"); probe.expectMsgEquals("c"); } @Test - public void mustProduceToConsumer() { + public void mustProduceToSubscriber() { final JavaTestKit probe = new JavaTestKit(system); - Consumer consumer = Duct.create(String.class).foreach(new Procedure() { + Subscriber subscriber = Duct.create(String.class).foreach(new Procedure() { public void apply(String elem) { probe.getRef().tell(elem, ActorRef.noSender()); } }).consume(materializer); - Consumer inConsumer = Duct.create(String.class).produceTo(materializer, consumer); + Subscriber inSubscriber = Duct.create(String.class).produceTo(materializer, subscriber); probe.expectNoMsg(FiniteDuration.create(200, TimeUnit.MILLISECONDS)); - Producer producer = Flow.create(Arrays.asList("a", "b", "c")).toProducer(materializer); - producer.produceTo(inConsumer); + Publisher publisher = Flow.create(Arrays.asList("a", "b", "c")).toPublisher(materializer); + publisher.subscribe(inSubscriber); probe.expectMsgEquals("a"); probe.expectMsgEquals("b"); probe.expectMsgEquals("c"); @@ -146,7 +146,7 @@ public class DuctTest { } }); - Consumer ductInConsumer = Duct.create(Integer.class).map(new Function() { + Subscriber ductInSubscriber = Duct.create(Integer.class).map(new Function() { public String apply(Integer elem) { return Integer.toString(elem * 2); } @@ -160,7 +160,7 @@ public class DuctTest { } }).consume(materializer); - Flow.create(Arrays.asList(1, 2, 3)).produceTo(materializer, ductInConsumer); + Flow.create(Arrays.asList(1, 2, 3)).produceTo(materializer, ductInSubscriber); probe.expectMsgEquals("elem-12"); probe.expectMsgEquals("elem-14"); @@ -171,7 +171,7 @@ public class DuctTest { public void mustCallOnCompleteCallbackWhenDone() { final JavaTestKit probe = new JavaTestKit(system); - Consumer inConsumer = Duct.create(Integer.class).map(new Function() { + Subscriber inSubscriber = Duct.create(Integer.class).map(new Function() { public String apply(Integer elem) { return elem.toString(); } @@ -185,15 +185,15 @@ public class DuctTest { } }); - Producer producer = Flow.create(Arrays.asList(1, 2, 3)).toProducer(materializer); - producer.produceTo(inConsumer); + Publisher publisher = Flow.create(Arrays.asList(1, 2, 3)).toPublisher(materializer); + publisher.subscribe(inSubscriber); probe.expectMsgEquals("done"); } @Test public void mustBeAbleToUseMapFuture() throws Exception { final JavaTestKit probe = new JavaTestKit(system); - Consumer c = Duct.create(String.class).mapFuture(new Function>() { + Subscriber c = Duct.create(String.class).mapFuture(new Function>() { public Future apply(String elem) { return Futures.successful(elem.toUpperCase()); } diff --git a/akka-stream/src/test/java/akka/stream/javadsl/FlowTest.java b/akka-stream/src/test/java/akka/stream/javadsl/FlowTest.java index 82da8fd005..1590fe453b 100644 --- a/akka-stream/src/test/java/akka/stream/javadsl/FlowTest.java +++ b/akka-stream/src/test/java/akka/stream/javadsl/FlowTest.java @@ -14,7 +14,7 @@ import akka.stream.OverflowStrategy; import org.junit.ClassRule; import org.junit.Test; import static org.junit.Assert.assertEquals; -import org.reactivestreams.api.Producer; +import org.reactivestreams.Publisher; import scala.Option; import scala.concurrent.Await; import scala.concurrent.Future; @@ -213,8 +213,8 @@ public class FlowTest { public String apply(String elem) { return elem.substring(0, 1); } - }).foreach(new Procedure>>() { - public void apply(final Pair> pair) { + }).foreach(new Procedure>>() { + public void apply(final Pair> pair) { Flow.create(pair.second()).foreach(new Procedure() { public void apply(String elem) { probe.getRef().tell(new Pair(pair.first(), elem), ActorRef.noSender()); @@ -246,9 +246,9 @@ public class FlowTest { public boolean test(String elem) { return elem.equals("\n"); } - }).foreach(new Procedure>() { - public void apply(Producer subProducer) { - Flow.create(subProducer).filter(new Predicate() { + }).foreach(new Procedure>() { + public void apply(Publisher subPublisher) { + Flow.create(subPublisher).filter(new Predicate() { public boolean test(String elem) { return !elem.equals("\n"); } @@ -280,7 +280,7 @@ public class FlowTest { final JavaTestKit probe = new JavaTestKit(system); final java.lang.Iterable input1 = Arrays.asList("A", "B", "C"); final java.lang.Iterable input2 = Arrays.asList("D", "E", "F"); - Flow.create(input1).merge(Flow.create(input2).toProducer(materializer)).foreach(new Procedure() { + Flow.create(input1).merge(Flow.create(input2).toPublisher(materializer)).foreach(new Procedure() { public void apply(String elem) { probe.getRef().tell(elem, ActorRef.noSender()); } @@ -295,7 +295,7 @@ public class FlowTest { final JavaTestKit probe = new JavaTestKit(system); final java.lang.Iterable input1 = Arrays.asList("A", "B", "C"); final java.lang.Iterable input2 = Arrays.asList(1, 2, 3); - Flow.create(input1).zip(Flow.create(input2).toProducer(materializer)) + Flow.create(input1).zip(Flow.create(input2).toPublisher(materializer)) .foreach(new Procedure>() { public void apply(Pair elem) { probe.getRef().tell(elem, ActorRef.noSender()); @@ -314,7 +314,7 @@ public class FlowTest { final JavaTestKit probe = new JavaTestKit(system); final java.lang.Iterable input1 = Arrays.asList("A", "B", "C"); final java.lang.Iterable input2 = Arrays.asList("D", "E", "F"); - Flow.create(input1).concat(Flow.create(input2).toProducer(materializer)).foreach(new Procedure() { + Flow.create(input1).concat(Flow.create(input2).toPublisher(materializer)).foreach(new Procedure() { public void apply(String elem) { probe.getRef().tell(elem, ActorRef.noSender()); } @@ -402,8 +402,8 @@ public class FlowTest { public void mustBeAbleToUsePrefixAndTail() throws Exception { final JavaTestKit probe = new JavaTestKit(system); final java.lang.Iterable input = Arrays.asList(1, 2, 3, 4, 5, 6); - Future, Producer>> future = Flow.create(input).prefixAndTail(3).toFuture(materializer); - Pair, Producer> result = + Future, Publisher>> future = Flow.create(input).prefixAndTail(3).toFuture(materializer); + Pair, Publisher> result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS))); assertEquals(Arrays.asList(1, 2, 3), result.first()); @@ -419,9 +419,9 @@ public class FlowTest { final java.lang.Iterable input1 = Arrays.asList(1, 2, 3); final java.lang.Iterable input2 = Arrays.asList(4, 5); - final List> mainInputs = Arrays.asList( - Flow.create(input1).toProducer(materializer), - Flow.create(input2).toProducer(materializer) + final List> mainInputs = Arrays.asList( + Flow.create(input1).toPublisher(materializer), + Flow.create(input2).toPublisher(materializer) ); Future> future = diff --git a/akka-stream/src/test/scala/akka/persistence/stream/PersistentPublisherSpec.scala b/akka-stream/src/test/scala/akka/persistence/stream/PersistentPublisherSpec.scala index 39d71dd386..04cd21dc00 100644 --- a/akka-stream/src/test/scala/akka/persistence/stream/PersistentPublisherSpec.scala +++ b/akka-stream/src/test/scala/akka/persistence/stream/PersistentPublisherSpec.scala @@ -24,7 +24,7 @@ object PersistentPublisherSpec { } } -class PersistentPublisherSpec extends AkkaSpec(PersistenceSpec.config("leveldb", "ViewProducerSpec", serialization = "off")) with PersistenceSpec { +class PersistentPublisherSpec extends AkkaSpec(PersistenceSpec.config("leveldb", "ViewPublisherSpec", serialization = "off")) with PersistenceSpec { import PersistentPublisherSpec._ val numMessages = 10 @@ -65,7 +65,7 @@ class PersistentPublisherSpec extends AkkaSpec(PersistenceSpec.config("leveldb", super.afterEach() } - "A view producer" must { + "A view publisher" must { "pull existing messages from a processor's journal" in { val streamProbe = TestProbe() @@ -108,28 +108,28 @@ class PersistentPublisherSpec extends AkkaSpec(PersistenceSpec.config("leveldb", } } - "A view producer" can { - "have several consumers" in { + "A view publisher" can { + "have several subscribers" in { val streamProbe1 = TestProbe() val streamProbe2 = TestProbe() - val producer = PersistentFlow.fromProcessor(processorId(1), publisherSettings).toProducer(materializer) + val publisher = PersistentFlow.fromProcessor(processorId(1), publisherSettings).toPublisher(materializer) - Flow(producer).foreach { + Flow(publisher).foreach { case Persistent(payload, sequenceNr) ⇒ streamProbe1.ref ! s"${payload}-${sequenceNr}" }.consume(materializer) - // let consumer consume all existing messages + // let subscriber consume all existing messages 1 to numMessages foreach { i ⇒ streamProbe1.expectMsg(s"a-${i}") } - // subscribe another consumer - Flow(producer).foreach { + // subscribe another subscriber + Flow(publisher).foreach { case Persistent(payload, sequenceNr) ⇒ streamProbe2.ref ! s"${payload}-${sequenceNr}" }.consume(materializer) - // produce new messages and let both consumers handle them + // produce new messages and let both subscribers handle them 1 to 2 foreach { i ⇒ processor1 ! Persistent("a") streamProbe1.expectMsg(s"a-${numMessages + i}") @@ -138,18 +138,18 @@ class PersistentPublisherSpec extends AkkaSpec(PersistenceSpec.config("leveldb", } } - "A consumer" can { - "consume from several view producers" in { + "A subscriber" can { + "consume from several view publishers" in { val streamProbe1 = TestProbe() val streamProbe2 = TestProbe() val fromSequenceNr1 = 7L val fromSequenceNr2 = 3L - val producer1 = PersistentFlow.fromProcessor(processorId(1), publisherSettings.copy(fromSequenceNr = fromSequenceNr1)).toProducer(materializer) - val producer2 = PersistentFlow.fromProcessor(processorId(2), publisherSettings.copy(fromSequenceNr = fromSequenceNr2)).toProducer(materializer) + val publisher1 = PersistentFlow.fromProcessor(processorId(1), publisherSettings.copy(fromSequenceNr = fromSequenceNr1)).toPublisher(materializer) + val publisher2 = PersistentFlow.fromProcessor(processorId(2), publisherSettings.copy(fromSequenceNr = fromSequenceNr2)).toPublisher(materializer) - Flow(producer1).merge(producer2).foreach { + Flow(publisher1).merge(publisher2).foreach { case Persistent(payload: String, sequenceNr) if (payload.startsWith("a")) ⇒ streamProbe1.ref ! s"${payload}-${sequenceNr}" case Persistent(payload: String, sequenceNr) if (payload.startsWith("b")) ⇒ streamProbe2.ref ! s"${payload}-${sequenceNr}" }.consume(materializer) diff --git a/akka-stream/src/test/scala/akka/stream/ActorProducerTest.scala b/akka-stream/src/test/scala/akka/stream/ActorProducerTest.scala deleted file mode 100644 index 306303c974..0000000000 --- a/akka-stream/src/test/scala/akka/stream/ActorProducerTest.scala +++ /dev/null @@ -1,47 +0,0 @@ -/** - * Copyright (C) 2014 Typesafe Inc. - */ -package akka.stream - -import org.scalatest.testng.TestNGSuiteLike -import org.reactivestreams.spi.Publisher -import org.reactivestreams.tck.{ TestEnvironment, PublisherVerification } -import org.reactivestreams.api.Producer -import akka.stream.scaladsl.Flow -import akka.actor.ActorSystem -import akka.stream.testkit.AkkaSpec - -class ActorProducerTest(_system: ActorSystem, env: TestEnvironment, publisherShutdownTimeout: Long) - extends PublisherVerification[Int](env, publisherShutdownTimeout) - with WithActorSystem with TestNGSuiteLike { - - implicit val system = _system - import system.dispatcher - - def this(system: ActorSystem) { - this(system, new TestEnvironment(Timeouts.defaultTimeoutMillis(system)), Timeouts.publisherShutdownTimeoutMillis) - } - - def this() { - this(ActorSystem(classOf[ActorProducerTest].getSimpleName, AkkaSpec.testConf)) - } - - private val materializer = FlowMaterializer(MaterializerSettings(dispatcher = "akka.test.stream-dispatcher")) - - private def createProducer(elements: Int): Producer[Int] = { - val iter = Iterator from 1000 - val iter2 = if (elements > 0) iter take elements else iter - Flow(() ⇒ if (iter2.hasNext) iter2.next() else throw Stop).toProducer(materializer) - } - - def createPublisher(elements: Int): Publisher[Int] = createProducer(elements).getPublisher - - override def createCompletedStatePublisher(): Publisher[Int] = { - val pub = createProducer(1) - Flow(pub).consume(materializer) - Thread.sleep(100) - pub.getPublisher - } - - override def createErrorStatePublisher(): Publisher[Int] = null // ignore error-state tests -} \ No newline at end of file diff --git a/akka-stream/src/test/scala/akka/stream/ActorPublisherTest.scala b/akka-stream/src/test/scala/akka/stream/ActorPublisherTest.scala new file mode 100644 index 0000000000..56c677100d --- /dev/null +++ b/akka-stream/src/test/scala/akka/stream/ActorPublisherTest.scala @@ -0,0 +1,14 @@ +/** + * Copyright (C) 2014 Typesafe Inc. + */ +package akka.stream + +import org.scalatest.testng.TestNGSuiteLike +import akka.stream.scaladsl.Flow +import akka.actor.ActorSystem +import akka.stream.testkit.AkkaSpec + +class ActorPublisherTest(_system: ActorSystem, /*env: TestEnvironment,*/ publisherShutdownTimeout: Long) { + // FIXME: Needs new TCK version + // Original code available in 82734877d080577cf538c2a47d60c117e078ac1c +} \ No newline at end of file diff --git a/akka-stream/src/test/scala/akka/stream/DuctSpec.scala b/akka-stream/src/test/scala/akka/stream/DuctSpec.scala index 346756c8f7..027c7145bd 100644 --- a/akka-stream/src/test/scala/akka/stream/DuctSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/DuctSpec.scala @@ -4,8 +4,7 @@ package akka.stream import scala.concurrent.duration._ -import org.reactivestreams.api.Consumer -import org.reactivestreams.api.Producer +import org.reactivestreams.{ Publisher, Subscriber } import akka.stream.scaladsl.Duct import akka.stream.scaladsl.Flow import akka.stream.testkit.AkkaSpec @@ -20,36 +19,36 @@ class DuctSpec extends AkkaSpec { "A Duct" must { - "materialize into Producer/Consumer" in { + "materialize into Publisher/Subscriber" in { val duct: Duct[String, String] = Duct[String] - val (ductIn: Consumer[String], ductOut: Producer[String]) = duct.build(materializer) + val (ductIn: Subscriber[String], ductOut: Publisher[String]) = duct.build(materializer) - val c1 = StreamTestKit.consumerProbe[String] - ductOut.produceTo(c1) + val c1 = StreamTestKit.SubscriberProbe[String]() + ductOut.subscribe(c1) - val source: Producer[String] = Flow(List("1", "2", "3")).toProducer(materializer) - source.produceTo(ductIn) + val source: Publisher[String] = Flow(List("1", "2", "3")).toPublisher(materializer) + source.subscribe(ductIn) val sub1 = c1.expectSubscription - sub1.requestMore(3) + sub1.request(3) c1.expectNext("1") c1.expectNext("2") c1.expectNext("3") c1.expectComplete } - "materialize into Producer/Consumer and transformation processor" in { + "materialize into Publisher/Subscriber and transformation processor" in { val duct: Duct[Int, String] = Duct[Int].map((i: Int) ⇒ i.toString) - val (ductIn: Consumer[Int], ductOut: Producer[String]) = duct.build(materializer) + val (ductIn: Subscriber[Int], ductOut: Publisher[String]) = duct.build(materializer) - val c1 = StreamTestKit.consumerProbe[String] - ductOut.produceTo(c1) + val c1 = StreamTestKit.SubscriberProbe[String]() + ductOut.subscribe(c1) val sub1 = c1.expectSubscription - sub1.requestMore(3) + sub1.request(3) c1.expectNoMsg(200.millis) - val source: Producer[Int] = Flow(List(1, 2, 3)).toProducer(materializer) - source.produceTo(ductIn) + val source: Publisher[Int] = Flow(List(1, 2, 3)).toPublisher(materializer) + source.subscribe(ductIn) c1.expectNext("1") c1.expectNext("2") @@ -57,18 +56,18 @@ class DuctSpec extends AkkaSpec { c1.expectComplete } - "materialize into Producer/Consumer and multiple transformation processors" in { + "materialize into Publisher/Subscriber and multiple transformation processors" in { val duct = Duct[Int].map(_.toString).map("elem-" + _) val (ductIn, ductOut) = duct.build(materializer) - val c1 = StreamTestKit.consumerProbe[String] - ductOut.produceTo(c1) + val c1 = StreamTestKit.SubscriberProbe[String]() + ductOut.subscribe(c1) val sub1 = c1.expectSubscription - sub1.requestMore(3) + sub1.request(3) c1.expectNoMsg(200.millis) - val source: Producer[Int] = Flow(List(1, 2, 3)).toProducer(materializer) - source.produceTo(ductIn) + val source: Publisher[Int] = Flow(List(1, 2, 3)).toPublisher(materializer) + source.subscribe(ductIn) c1.expectNext("elem-1") c1.expectNext("elem-2") @@ -76,15 +75,15 @@ class DuctSpec extends AkkaSpec { c1.expectComplete } - "produceTo Consumer" in { + "subscribe Subscriber" in { val duct: Duct[String, String] = Duct[String] - val c1 = StreamTestKit.consumerProbe[String] - val c2: Consumer[String] = duct.produceTo(materializer, c1) - val source: Producer[String] = Flow(List("1", "2", "3")).toProducer(materializer) - source.produceTo(c2) + val c1 = StreamTestKit.SubscriberProbe[String]() + val c2: Subscriber[String] = duct.produceTo(materializer, c1) + val source: Publisher[String] = Flow(List("1", "2", "3")).toPublisher(materializer) + source.subscribe(c2) val sub1 = c1.expectSubscription - sub1.requestMore(3) + sub1.request(3) c1.expectNext("1") c1.expectNext("2") c1.expectNext("3") @@ -95,8 +94,8 @@ class DuctSpec extends AkkaSpec { val duct = Duct[Int].map(i ⇒ { testActor ! i.toString; i.toString }) val c = duct.consume(materializer) - val source = Flow(List(1, 2, 3)).toProducer(materializer) - source.produceTo(c) + val source = Flow(List(1, 2, 3)).toPublisher(materializer) + source.subscribe(c) expectMsg("1") expectMsg("2") @@ -107,25 +106,25 @@ class DuctSpec extends AkkaSpec { val duct = Duct[Int].map(_.toString).map("elem-" + _).foreach(testActor ! _) val c = duct.consume(materializer) - val source = Flow(List(1, 2, 3)).toProducer(materializer) - source.produceTo(c) + val source = Flow(List(1, 2, 3)).toPublisher(materializer) + source.subscribe(c) expectMsg("elem-1") expectMsg("elem-2") expectMsg("elem-3") } - "perform transformation operation and produceTo Consumer" in { + "perform transformation operation and subscribe Subscriber" in { val duct = Duct[Int].map(_.toString) - val c1 = StreamTestKit.consumerProbe[String] - val c2: Consumer[Int] = duct.produceTo(materializer, c1) + val c1 = StreamTestKit.SubscriberProbe[String]() + val c2: Subscriber[Int] = duct.produceTo(materializer, c1) val sub1 = c1.expectSubscription - sub1.requestMore(3) + sub1.request(3) c1.expectNoMsg(200.millis) - val source: Producer[Int] = Flow(List(1, 2, 3)).toProducer(materializer) - source.produceTo(c2) + val source: Publisher[Int] = Flow(List(1, 2, 3)).toPublisher(materializer) + source.subscribe(c2) c1.expectNext("1") c1.expectNext("2") @@ -133,17 +132,17 @@ class DuctSpec extends AkkaSpec { c1.expectComplete } - "perform multiple transformation operations and produceTo Consumer" in { + "perform multiple transformation operations and subscribe Subscriber" in { val duct = Duct[Int].map(_.toString).map("elem-" + _) - val c1 = StreamTestKit.consumerProbe[String] + val c1 = StreamTestKit.SubscriberProbe[String]() val c2 = duct.produceTo(materializer, c1) val sub1 = c1.expectSubscription - sub1.requestMore(3) + sub1.request(3) c1.expectNoMsg(200.millis) - val source: Producer[Int] = Flow(List(1, 2, 3)).toProducer(materializer) - source.produceTo(c2) + val source: Publisher[Int] = Flow(List(1, 2, 3)).toPublisher(materializer) + source.subscribe(c2) c1.expectNext("elem-1") c1.expectNext("elem-2") @@ -158,8 +157,8 @@ class DuctSpec extends AkkaSpec { case Failure(e) ⇒ testActor ! e } - val source = Flow(List(1, 2, 3)).toProducer(materializer) - source.produceTo(c) + val source = Flow(List(1, 2, 3)).toPublisher(materializer) + source.subscribe(c) expectMsg("1") expectMsg("2") @@ -168,12 +167,12 @@ class DuctSpec extends AkkaSpec { } "be appendable to a Flow" in { - val c = StreamTestKit.consumerProbe[String] + val c = StreamTestKit.SubscriberProbe[String]() val duct = Duct[Int].map(_ + 10).map(_.toString) Flow(List(1, 2, 3)).map(_ * 2).append(duct).map((s: String) ⇒ "elem-" + s).produceTo(materializer, c) val sub = c.expectSubscription - sub.requestMore(3) + sub.request(3) c.expectNext("elem-12") c.expectNext("elem-14") c.expectNext("elem-16") @@ -181,18 +180,18 @@ class DuctSpec extends AkkaSpec { } "be appendable to a Duct" in { - val c = StreamTestKit.consumerProbe[String] + val c = StreamTestKit.SubscriberProbe[String]() val duct1 = Duct[String].map(Integer.parseInt) - val ductInConsumer = Duct[Int] + val ductInSubscriber = Duct[Int] .map { i ⇒ (i * 2).toString } .append(duct1) .map { i ⇒ "elem-" + (i + 10) } .produceTo(materializer, c) - Flow(List(1, 2, 3)).produceTo(materializer, ductInConsumer) + Flow(List(1, 2, 3)).produceTo(materializer, ductInSubscriber) val sub = c.expectSubscription - sub.requestMore(3) + sub.request(3) c.expectNext("elem-12") c.expectNext("elem-14") c.expectNext("elem-16") diff --git a/akka-stream/src/test/scala/akka/stream/FlowBufferSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowBufferSpec.scala index 2048996bfc..d35e4138e5 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowBufferSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowBufferSpec.scala @@ -44,108 +44,108 @@ class FlowBufferSpec extends AkkaSpec { } "accept elements that fit in the buffer while downstream is silent" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[Int] + val publisher = StreamTestKit.PublisherProbe[Int]() + val subscriber = StreamTestKit.SubscriberProbe[Int]() - Flow(producer).buffer(100, overflowStrategy = OverflowStrategy.backpressure).produceTo(materializer, consumer) + Flow(publisher).buffer(100, overflowStrategy = OverflowStrategy.backpressure).produceTo(materializer, subscriber) - val autoProducer = new StreamTestKit.AutoProducer(producer) - val sub = consumer.expectSubscription() + val autoPublisher = new StreamTestKit.AutoPublisher(publisher) + val sub = subscriber.expectSubscription() // Fill up buffer - for (i ← 1 to 100) autoProducer.sendNext(i) + for (i ← 1 to 100) autoPublisher.sendNext(i) // drain for (i ← 1 to 100) { - sub.requestMore(1) - consumer.expectNext(i) + sub.request(1) + subscriber.expectNext(i) } sub.cancel() } "drop head elements if buffer is full and configured so" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[Int] + val publisher = StreamTestKit.PublisherProbe[Int]() + val subscriber = StreamTestKit.SubscriberProbe[Int]() - Flow(producer).buffer(100, overflowStrategy = OverflowStrategy.dropHead).produceTo(materializer, consumer) + Flow(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropHead).produceTo(materializer, subscriber) - val autoProducer = new StreamTestKit.AutoProducer(producer) - val sub = consumer.expectSubscription() + val autoPublisher = new StreamTestKit.AutoPublisher(publisher) + val sub = subscriber.expectSubscription() // Fill up buffer - for (i ← 1 to 200) autoProducer.sendNext(i) + for (i ← 1 to 200) autoPublisher.sendNext(i) // drain for (i ← 101 to 200) { - sub.requestMore(1) - consumer.expectNext(i) + sub.request(1) + subscriber.expectNext(i) } - sub.requestMore(1) - consumer.expectNoMsg(1.seconds) + sub.request(1) + subscriber.expectNoMsg(1.seconds) - autoProducer.sendNext(-1) - sub.requestMore(1) - consumer.expectNext(-1) + autoPublisher.sendNext(-1) + sub.request(1) + subscriber.expectNext(-1) sub.cancel() } "drop tail elements if buffer is full and configured so" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[Int] + val publisher = StreamTestKit.PublisherProbe[Int]() + val subscriber = StreamTestKit.SubscriberProbe[Int]() - Flow(producer).buffer(100, overflowStrategy = OverflowStrategy.dropTail).produceTo(materializer, consumer) + Flow(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropTail).produceTo(materializer, subscriber) - val autoProducer = new StreamTestKit.AutoProducer(producer) - val sub = consumer.expectSubscription() + val autoPublisher = new StreamTestKit.AutoPublisher(publisher) + val sub = subscriber.expectSubscription() // Fill up buffer - for (i ← 1 to 200) autoProducer.sendNext(i) + for (i ← 1 to 200) autoPublisher.sendNext(i) // drain for (i ← 1 to 99) { - sub.requestMore(1) - consumer.expectNext(i) + sub.request(1) + subscriber.expectNext(i) } - sub.requestMore(1) - consumer.expectNext(200) + sub.request(1) + subscriber.expectNext(200) - sub.requestMore(1) - consumer.expectNoMsg(1.seconds) + sub.request(1) + subscriber.expectNoMsg(1.seconds) - autoProducer.sendNext(-1) - sub.requestMore(1) - consumer.expectNext(-1) + autoPublisher.sendNext(-1) + sub.request(1) + subscriber.expectNext(-1) sub.cancel() } "drop all elements if buffer is full and configured so" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[Int] + val publisher = StreamTestKit.PublisherProbe[Int] + val subscriber = StreamTestKit.SubscriberProbe[Int]() - Flow(producer).buffer(100, overflowStrategy = OverflowStrategy.dropBuffer).produceTo(materializer, consumer) + Flow(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropBuffer).produceTo(materializer, subscriber) - val autoProducer = new StreamTestKit.AutoProducer(producer) - val sub = consumer.expectSubscription() + val autoPublisher = new StreamTestKit.AutoPublisher(publisher) + val sub = subscriber.expectSubscription() // Fill up buffer - for (i ← 1 to 150) autoProducer.sendNext(i) + for (i ← 1 to 150) autoPublisher.sendNext(i) // drain for (i ← 101 to 150) { - sub.requestMore(1) - consumer.expectNext(i) + sub.request(1) + subscriber.expectNext(i) } - sub.requestMore(1) - consumer.expectNoMsg(1.seconds) + sub.request(1) + subscriber.expectNoMsg(1.seconds) - autoProducer.sendNext(-1) - sub.requestMore(1) - consumer.expectNext(-1) + autoPublisher.sendNext(-1) + sub.request(1) + subscriber.expectNext(-1) sub.cancel() } @@ -154,26 +154,26 @@ class FlowBufferSpec extends AkkaSpec { s"work with $strategy if buffer size of one" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[Int] + val publisher = StreamTestKit.PublisherProbe[Int] + val subscriber = StreamTestKit.SubscriberProbe[Int]() - Flow(producer).buffer(1, overflowStrategy = strategy).produceTo(materializer, consumer) + Flow(publisher).buffer(1, overflowStrategy = strategy).produceTo(materializer, subscriber) - val autoProducer = new StreamTestKit.AutoProducer(producer) - val sub = consumer.expectSubscription() + val autoPublisher = new StreamTestKit.AutoPublisher(publisher) + val sub = subscriber.expectSubscription() // Fill up buffer - for (i ← 1 to 200) autoProducer.sendNext(i) + for (i ← 1 to 200) autoPublisher.sendNext(i) - sub.requestMore(1) - consumer.expectNext(200) + sub.request(1) + subscriber.expectNext(200) - sub.requestMore(1) - consumer.expectNoMsg(1.seconds) + sub.request(1) + subscriber.expectNoMsg(1.seconds) - autoProducer.sendNext(-1) - sub.requestMore(1) - consumer.expectNext(-1) + autoPublisher.sendNext(-1) + sub.request(1) + subscriber.expectNext(-1) sub.cancel() } diff --git a/akka-stream/src/test/scala/akka/stream/FlowConcatAllSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowConcatAllSpec.scala index 1f5a8a9e6a..0314c31f8d 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowConcatAllSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowConcatAllSpec.scala @@ -7,7 +7,7 @@ import akka.stream.testkit.{ StreamTestKit, AkkaSpec } import akka.stream.scaladsl.Flow import scala.concurrent.duration._ import scala.concurrent.Await -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import scala.util.control.NoStackTrace class FlowConcatAllSpec extends AkkaSpec { @@ -24,13 +24,13 @@ class FlowConcatAllSpec extends AkkaSpec { val testException = new Exception("test") with NoStackTrace "work in the happy case" in { - val s1 = Flow((1 to 2).iterator).toProducer(m) - val s2 = Flow(List.empty[Int]).toProducer(m) - val s3 = Flow(List(3)).toProducer(m) - val s4 = Flow((4 to 6).iterator).toProducer(m) - val s5 = Flow((7 to 10).iterator).toProducer(m) + val s1 = Flow((1 to 2).iterator).toPublisher(m) + val s2 = Flow(List.empty[Int]).toPublisher(m) + val s3 = Flow(List(3)).toPublisher(m) + val s4 = Flow((4 to 6).iterator).toPublisher(m) + val s5 = Flow((7 to 10).iterator).toPublisher(m) - val main: Flow[Producer[Int]] = Flow(List(s1, s2, s3, s4, s5)) + val main: Flow[Publisher[Int]] = Flow(List(s1, s2, s3, s4, s5)) Await.result(main.flatten(FlattenStrategy.concat).grouped(10).toFuture(m), 3.seconds) should be(1 to 10) } @@ -42,56 +42,56 @@ class FlowConcatAllSpec extends AkkaSpec { } "on onError on master stream cancel the current open substream and signal error" in { - val producer = StreamTestKit.producerProbe[Producer[Int]] - val consumer = StreamTestKit.consumerProbe[Int] - Flow(producer).flatten(FlattenStrategy.concat).produceTo(m, consumer) + val publisher = StreamTestKit.PublisherProbe[Publisher[Int]]() + val subscriber = StreamTestKit.SubscriberProbe[Int]() + Flow(publisher).flatten(FlattenStrategy.concat).produceTo(m, subscriber) - val upstream = producer.expectSubscription() - val downstream = consumer.expectSubscription() - downstream.requestMore(1000) + val upstream = publisher.expectSubscription() + val downstream = subscriber.expectSubscription() + downstream.request(1000) - val substreamProducer = StreamTestKit.producerProbe[Int] - upstream.expectRequestMore() - upstream.sendNext(substreamProducer) - val subUpstream = substreamProducer.expectSubscription() + val substreamPublisher = StreamTestKit.PublisherProbe[Int]() + upstream.expectRequest() + upstream.sendNext(substreamPublisher) + val subUpstream = substreamPublisher.expectSubscription() upstream.sendError(testException) - consumer.expectError(testException) + subscriber.expectError(testException) subUpstream.expectCancellation() } "on onError on open substream, cancel the master stream and signal error " in { - val producer = StreamTestKit.producerProbe[Producer[Int]] - val consumer = StreamTestKit.consumerProbe[Int] - Flow(producer).flatten(FlattenStrategy.concat).produceTo(m, consumer) + val publisher = StreamTestKit.PublisherProbe[Publisher[Int]]() + val subscriber = StreamTestKit.SubscriberProbe[Int]() + Flow(publisher).flatten(FlattenStrategy.concat).produceTo(m, subscriber) - val upstream = producer.expectSubscription() - val downstream = consumer.expectSubscription() - downstream.requestMore(1000) + val upstream = publisher.expectSubscription() + val downstream = subscriber.expectSubscription() + downstream.request(1000) - val substreamProducer = StreamTestKit.producerProbe[Int] - upstream.expectRequestMore() - upstream.sendNext(substreamProducer) - val subUpstream = substreamProducer.expectSubscription() + val substreamPublisher = StreamTestKit.PublisherProbe[Int]() + upstream.expectRequest() + upstream.sendNext(substreamPublisher) + val subUpstream = substreamPublisher.expectSubscription() subUpstream.sendError(testException) - consumer.expectError(testException) + subscriber.expectError(testException) upstream.expectCancellation() } "on cancellation cancel the current open substream and the master stream" in { - val producer = StreamTestKit.producerProbe[Producer[Int]] - val consumer = StreamTestKit.consumerProbe[Int] - Flow(producer).flatten(FlattenStrategy.concat).produceTo(m, consumer) + val publisher = StreamTestKit.PublisherProbe[Publisher[Int]]() + val subscriber = StreamTestKit.SubscriberProbe[Int]() + Flow(publisher).flatten(FlattenStrategy.concat).produceTo(m, subscriber) - val upstream = producer.expectSubscription() - val downstream = consumer.expectSubscription() - downstream.requestMore(1000) + val upstream = publisher.expectSubscription() + val downstream = subscriber.expectSubscription() + downstream.request(1000) - val substreamProducer = StreamTestKit.producerProbe[Int] - upstream.expectRequestMore() - upstream.sendNext(substreamProducer) - val subUpstream = substreamProducer.expectSubscription() + val substreamPublisher = StreamTestKit.PublisherProbe[Int]() + upstream.expectRequest() + upstream.sendNext(substreamPublisher) + val subUpstream = substreamPublisher.expectSubscription() downstream.cancel() diff --git a/akka-stream/src/test/scala/akka/stream/FlowConcatSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowConcatSpec.scala index c377a14d46..e36a743778 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowConcatSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowConcatSpec.scala @@ -5,28 +5,28 @@ package akka.stream import akka.stream.testkit.StreamTestKit import akka.stream.scaladsl.Flow -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import scala.concurrent.Promise class FlowConcatSpec extends TwoStreamsSetup { type Outputs = Int - override def operationUnderTest(in1: Flow[Int], in2: Producer[Int]) = in1.concat(in2) + override def operationUnderTest(in1: Flow[Int], in2: Publisher[Int]) = in1.concat(in2) "Concat" must { "work in the happy case" in { - val source0 = Flow(List.empty[Int].iterator).toProducer(materializer) - val source1 = Flow((1 to 4).iterator).toProducer(materializer) - val source2 = Flow((5 to 10).iterator).toProducer(materializer) - val p = Flow(source0).concat(source1).concat(source2).toProducer(materializer) + val source0 = Flow(List.empty[Int].iterator).toPublisher(materializer) + val source1 = Flow((1 to 4).iterator).toPublisher(materializer) + val source2 = Flow((5 to 10).iterator).toPublisher(materializer) + val p = Flow(source0).concat(source1).concat(source2).toPublisher(materializer) - val probe = StreamTestKit.consumerProbe[Int] - p.produceTo(probe) + val probe = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(probe) val subscription = probe.expectSubscription() for (i ← 1 to 10) { - subscription.requestMore(1) + subscription.request(1) probe.expectNext(i) } @@ -35,74 +35,74 @@ class FlowConcatSpec extends TwoStreamsSetup { commonTests() - "work with one immediately completed and one nonempty producer" in { - val consumer1 = setup(completedPublisher, nonemptyPublisher((1 to 4).iterator)) - val subscription1 = consumer1.expectSubscription() - subscription1.requestMore(5) - consumer1.expectNext(1) - consumer1.expectNext(2) - consumer1.expectNext(3) - consumer1.expectNext(4) - consumer1.expectComplete() + "work with one immediately completed and one nonempty publisher" in { + val subscriber1 = setup(completedPublisher, nonemptyPublisher((1 to 4).iterator)) + val subscription1 = subscriber1.expectSubscription() + subscription1.request(5) + subscriber1.expectNext(1) + subscriber1.expectNext(2) + subscriber1.expectNext(3) + subscriber1.expectNext(4) + subscriber1.expectComplete() - val consumer2 = setup(nonemptyPublisher((1 to 4).iterator), completedPublisher) - val subscription2 = consumer2.expectSubscription() - subscription2.requestMore(5) - consumer2.expectNext(1) - consumer2.expectNext(2) - consumer2.expectNext(3) - consumer2.expectNext(4) - consumer2.expectComplete() + val subscriber2 = setup(nonemptyPublisher((1 to 4).iterator), completedPublisher) + val subscription2 = subscriber2.expectSubscription() + subscription2.request(5) + subscriber2.expectNext(1) + subscriber2.expectNext(2) + subscriber2.expectNext(3) + subscriber2.expectNext(4) + subscriber2.expectComplete() } - "work with one delayed completed and one nonempty producer" in { - val consumer1 = setup(soonToCompletePublisher, nonemptyPublisher((1 to 4).iterator)) - val subscription1 = consumer1.expectSubscription() - subscription1.requestMore(5) - consumer1.expectNext(1) - consumer1.expectNext(2) - consumer1.expectNext(3) - consumer1.expectNext(4) - consumer1.expectComplete() + "work with one delayed completed and one nonempty publisher" in { + val subscriber1 = setup(soonToCompletePublisher, nonemptyPublisher((1 to 4).iterator)) + val subscription1 = subscriber1.expectSubscription() + subscription1.request(5) + subscriber1.expectNext(1) + subscriber1.expectNext(2) + subscriber1.expectNext(3) + subscriber1.expectNext(4) + subscriber1.expectComplete() - val consumer2 = setup(nonemptyPublisher((1 to 4).iterator), soonToCompletePublisher) - val subscription2 = consumer2.expectSubscription() - subscription2.requestMore(5) - consumer2.expectNext(1) - consumer2.expectNext(2) - consumer2.expectNext(3) - consumer2.expectNext(4) - consumer2.expectComplete() + val subscriber2 = setup(nonemptyPublisher((1 to 4).iterator), soonToCompletePublisher) + val subscription2 = subscriber2.expectSubscription() + subscription2.request(5) + subscriber2.expectNext(1) + subscriber2.expectNext(2) + subscriber2.expectNext(3) + subscriber2.expectNext(4) + subscriber2.expectComplete() } - "work with one immediately failed and one nonempty producer" in { - val consumer1 = setup(failedPublisher, nonemptyPublisher((1 to 4).iterator)) - consumer1.expectErrorOrSubscriptionFollowedByError(TestException) + "work with one immediately failed and one nonempty publisher" in { + val subscriber1 = setup(failedPublisher, nonemptyPublisher((1 to 4).iterator)) + subscriber1.expectErrorOrSubscriptionFollowedByError(TestException) - val consumer2 = setup(nonemptyPublisher((1 to 4).iterator), failedPublisher) - consumer2.expectErrorOrSubscriptionFollowedByError(TestException) + val subscriber2 = setup(nonemptyPublisher((1 to 4).iterator), failedPublisher) + subscriber2.expectErrorOrSubscriptionFollowedByError(TestException) } - "work with one delayed failed and one nonempty producer" in { - val consumer1 = setup(soonToFailPublisher, nonemptyPublisher((1 to 4).iterator)) - consumer1.expectErrorOrSubscriptionFollowedByError(TestException) + "work with one delayed failed and one nonempty publisher" in { + val subscriber1 = setup(soonToFailPublisher, nonemptyPublisher((1 to 4).iterator)) + subscriber1.expectErrorOrSubscriptionFollowedByError(TestException) - val consumer2 = setup(nonemptyPublisher((1 to 4).iterator), soonToFailPublisher) - consumer2.expectErrorOrSubscriptionFollowedByError(TestException) + val subscriber2 = setup(nonemptyPublisher((1 to 4).iterator), soonToFailPublisher) + subscriber2.expectErrorOrSubscriptionFollowedByError(TestException) } "correctly handle async errors in secondary upstream" in { val promise = Promise[Int]() - val flow = Flow(List(1, 2, 3)).concat(Flow(promise.future).toProducer(materializer)) - val consumer = StreamTestKit.consumerProbe[Int] - flow.produceTo(materializer, consumer) - val subscription = consumer.expectSubscription() - subscription.requestMore(4) - consumer.expectNext(1) - consumer.expectNext(2) - consumer.expectNext(3) + val flow = Flow(List(1, 2, 3)).concat(Flow(promise.future).toPublisher(materializer)) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + flow.produceTo(materializer, subscriber) + val subscription = subscriber.expectSubscription() + subscription.request(4) + subscriber.expectNext(1) + subscriber.expectNext(2) + subscriber.expectNext(3) promise.failure(TestException) - consumer.expectError(TestException) + subscriber.expectError(TestException) } } } diff --git a/akka-stream/src/test/scala/akka/stream/FlowConflateSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowConflateSpec.scala index de04ad5c92..07ec36037b 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowConflateSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowConflateSpec.scala @@ -21,38 +21,38 @@ class FlowConflateSpec extends AkkaSpec { "Conflate" must { "pass-through elements unchanged when there is no rate difference" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[Int] + val publisher = StreamTestKit.PublisherProbe[Int]() + val subscriber = StreamTestKit.SubscriberProbe[Int]() - Flow(producer).conflate[Int](seed = i ⇒ i, aggregate = (sum, i) ⇒ sum + i).produceTo(materializer, consumer) + Flow(publisher).conflate[Int](seed = i ⇒ i, aggregate = (sum, i) ⇒ sum + i).produceTo(materializer, subscriber) - val autoProducer = new StreamTestKit.AutoProducer(producer) - val sub = consumer.expectSubscription() + val autoPublisher = new StreamTestKit.AutoPublisher(publisher) + val sub = subscriber.expectSubscription() for (i ← 1 to 100) { - sub.requestMore(1) - autoProducer.sendNext(i) - consumer.expectNext(i) + sub.request(1) + autoPublisher.sendNext(i) + subscriber.expectNext(i) } sub.cancel() } "conflate elements while downstream is silent" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[Int] + val publisher = StreamTestKit.PublisherProbe[Int]() + val subscriber = StreamTestKit.SubscriberProbe[Int]() - Flow(producer).conflate[Int](seed = i ⇒ i, aggregate = (sum, i) ⇒ sum + i).produceTo(materializer, consumer) + Flow(publisher).conflate[Int](seed = i ⇒ i, aggregate = (sum, i) ⇒ sum + i).produceTo(materializer, subscriber) - val autoProducer = new StreamTestKit.AutoProducer(producer) - val sub = consumer.expectSubscription() + val autoPublisher = new StreamTestKit.AutoPublisher(publisher) + val sub = subscriber.expectSubscription() for (i ← 1 to 100) { - autoProducer.sendNext(i) + autoPublisher.sendNext(i) } - consumer.expectNoMsg(1.second) - sub.requestMore(1) - consumer.expectNext(5050) + subscriber.expectNoMsg(1.second) + sub.request(1) + subscriber.expectNext(5050) sub.cancel() } @@ -66,31 +66,31 @@ class FlowConflateSpec extends AkkaSpec { Await.result(future, 10.seconds) should be(500500) } - "backpressure consumer when upstream is slower" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[Int] + "backpressure subscriber when upstream is slower" in { + val oublisher = StreamTestKit.PublisherProbe[Int]() + val subscriber = StreamTestKit.SubscriberProbe[Int]() - Flow(producer).conflate[Int](seed = i ⇒ i, aggregate = (sum, i) ⇒ sum + i).produceTo(materializer, consumer) + Flow(oublisher).conflate[Int](seed = i ⇒ i, aggregate = (sum, i) ⇒ sum + i).produceTo(materializer, subscriber) - val autoProducer = new StreamTestKit.AutoProducer(producer) - val sub = consumer.expectSubscription() + val autoPublisher = new StreamTestKit.AutoPublisher(oublisher) + val sub = subscriber.expectSubscription() - sub.requestMore(1) - autoProducer.sendNext(1) - consumer.expectNext(1) + sub.request(1) + autoPublisher.sendNext(1) + subscriber.expectNext(1) - sub.requestMore(1) - consumer.expectNoMsg(1.second) - autoProducer.sendNext(2) - consumer.expectNext(2) + sub.request(1) + subscriber.expectNoMsg(1.second) + autoPublisher.sendNext(2) + subscriber.expectNext(2) - autoProducer.sendNext(3) - autoProducer.sendNext(4) - sub.requestMore(1) - consumer.expectNext(7) + autoPublisher.sendNext(3) + autoPublisher.sendNext(4) + sub.request(1) + subscriber.expectNext(7) - sub.requestMore(1) - consumer.expectNoMsg(1.second) + sub.request(1) + subscriber.expectNoMsg(1.second) sub.cancel() } diff --git a/akka-stream/src/test/scala/akka/stream/FlowDropWithinSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowDropWithinSpec.scala index 4fb9a8a038..b859a905a6 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowDropWithinSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowDropWithinSpec.scala @@ -18,17 +18,17 @@ class FlowDropWithinSpec extends AkkaSpec { "deliver elements after the duration, but not before" in { val input = Iterator.from(1) - val p = StreamTestKit.producerProbe[Int] - val c = StreamTestKit.consumerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() + val c = StreamTestKit.SubscriberProbe[Int]() Flow(p).dropWithin(1.second).produceTo(materializer, c) val pSub = p.expectSubscription val cSub = c.expectSubscription - cSub.requestMore(100) - val demand1 = pSub.expectRequestMore + cSub.request(100) + val demand1 = pSub.expectRequest (1 to demand1) foreach { _ ⇒ pSub.sendNext(input.next()) } - val demand2 = pSub.expectRequestMore + val demand2 = pSub.expectRequest (1 to demand2) foreach { _ ⇒ pSub.sendNext(input.next()) } - val demand3 = pSub.expectRequestMore + val demand3 = pSub.expectRequest c.expectNoMsg(1500.millis) (1 to demand3) foreach { _ ⇒ pSub.sendNext(input.next()) } ((demand1 + demand2 + 1) to (demand1 + demand2 + demand3)) foreach { n ⇒ c.expectNext(n) } diff --git a/akka-stream/src/test/scala/akka/stream/FlowExpandSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowExpandSpec.scala index 7fb7235bcc..7ff525bcfa 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowExpandSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowExpandSpec.scala @@ -21,45 +21,45 @@ class FlowExpandSpec extends AkkaSpec { "Expand" must { "pass-through elements unchanged when there is no rate difference" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[Int] + val publisher = StreamTestKit.PublisherProbe[Int]() + val subscriber = StreamTestKit.SubscriberProbe[Int]() // Simply repeat the last element as an extrapolation step - Flow(producer).expand[Int, Int](seed = i ⇒ i, extrapolate = i ⇒ (i, i)).produceTo(materializer, consumer) + Flow(publisher).expand[Int, Int](seed = i ⇒ i, extrapolate = i ⇒ (i, i)).produceTo(materializer, subscriber) - val autoProducer = new StreamTestKit.AutoProducer(producer) - val sub = consumer.expectSubscription() + val autoPublisher = new StreamTestKit.AutoPublisher(publisher) + val sub = subscriber.expectSubscription() for (i ← 1 to 100) { // Order is important here: If the request comes first it will be extrapolated! - autoProducer.sendNext(i) - sub.requestMore(1) - consumer.expectNext(i) + autoPublisher.sendNext(i) + sub.request(1) + subscriber.expectNext(i) } sub.cancel() } "expand elements while upstream is silent" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[Int] + val publisher = StreamTestKit.PublisherProbe[Int]() + val subscriber = StreamTestKit.SubscriberProbe[Int]() // Simply repeat the last element as an extrapolation step - Flow(producer).expand[Int, Int](seed = i ⇒ i, extrapolate = i ⇒ (i, i)).produceTo(materializer, consumer) + Flow(publisher).expand[Int, Int](seed = i ⇒ i, extrapolate = i ⇒ (i, i)).produceTo(materializer, subscriber) - val autoProducer = new StreamTestKit.AutoProducer(producer) - val sub = consumer.expectSubscription() + val autoPublisher = new StreamTestKit.AutoPublisher(publisher) + val sub = subscriber.expectSubscription() - autoProducer.sendNext(42) + autoPublisher.sendNext(42) for (i ← 1 to 100) { - sub.requestMore(1) - consumer.expectNext(42) + sub.request(1) + subscriber.expectNext(42) } - autoProducer.sendNext(-42) - sub.requestMore(1) - consumer.expectNext(-42) + autoPublisher.sendNext(-42) + sub.request(1) + subscriber.expectNext(-42) sub.cancel() } @@ -74,44 +74,44 @@ class FlowExpandSpec extends AkkaSpec { Await.result(future, 10.seconds) should be(Set.empty[Int] ++ (1 to 100)) } - "backpressure producer when consumer is slower" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[Int] + "backpressure publisher when subscriber is slower" in { + val publisher = StreamTestKit.PublisherProbe[Int]() + val subscriber = StreamTestKit.SubscriberProbe[Int]() - Flow(producer).expand[Int, Int](seed = i ⇒ i, extrapolate = i ⇒ (i, i)).produceTo(materializer, consumer) + Flow(publisher).expand[Int, Int](seed = i ⇒ i, extrapolate = i ⇒ (i, i)).produceTo(materializer, subscriber) - val autoProducer = new StreamTestKit.AutoProducer(producer) - val sub = consumer.expectSubscription() + val autoPublisher = new StreamTestKit.AutoPublisher(publisher) + val sub = subscriber.expectSubscription() - autoProducer.sendNext(1) - sub.requestMore(1) - consumer.expectNext(1) - sub.requestMore(1) - consumer.expectNext(1) + autoPublisher.sendNext(1) + sub.request(1) + subscriber.expectNext(1) + sub.request(1) + subscriber.expectNext(1) - var pending = autoProducer.pendingRequests + var pending = autoPublisher.pendingRequests // Deplete pending requests coming from input buffer while (pending > 0) { - autoProducer.subscription.sendNext(2) + autoPublisher.subscription.sendNext(2) pending -= 1 } // The above sends are absorbed in the input buffer, and will result in two one-sized batch requests - pending += autoProducer.subscription.expectRequestMore() - pending += autoProducer.subscription.expectRequestMore() + pending += autoPublisher.subscription.expectRequest() + pending += autoPublisher.subscription.expectRequest() while (pending > 0) { - autoProducer.subscription.sendNext(2) + autoPublisher.subscription.sendNext(2) pending -= 1 } - producer.expectNoMsg(1.second) + publisher.expectNoMsg(1.second) - sub.requestMore(2) - consumer.expectNext(2) - consumer.expectNext(2) + sub.request(2) + subscriber.expectNext(2) + subscriber.expectNext(2) // Now production is resumed - autoProducer.subscription.expectRequestMore() + autoPublisher.subscription.expectRequest() } } diff --git a/akka-stream/src/test/scala/akka/stream/FlowFilterSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowFilterSpec.scala index 5f4ba6cf2e..e94173c470 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowFilterSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowFilterSpec.scala @@ -33,13 +33,13 @@ class FlowFilterSpec extends AkkaSpec with ScriptedTest { maxFanOutBufferSize = 1, dispatcher = "akka.test.stream-dispatcher")) - val probe = StreamTestKit.consumerProbe[Int] + val probe = StreamTestKit.SubscriberProbe[Int]() Flow(Iterator.fill(1000)(0) ++ List(1)).filter(_ != 0). - toProducer(materializer).produceTo(probe) + toPublisher(materializer).subscribe(probe) val subscription = probe.expectSubscription() for (_ ← 1 to 10000) { - subscription.requestMore(Int.MaxValue) + subscription.request(Int.MaxValue) } probe.expectNext(1) diff --git a/akka-stream/src/test/scala/akka/stream/FlowFromFutureSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowFromFutureSpec.scala index ca1302aa89..6bd81abcd6 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowFromFutureSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowFromFutureSpec.scala @@ -3,17 +3,11 @@ */ package akka.stream -import scala.concurrent.duration._ -import akka.stream.testkit.StreamTestKit -import akka.stream.testkit.AkkaSpec -import akka.stream.testkit.OnNext -import akka.dispatch.OnComplete -import akka.stream.testkit.OnComplete -import akka.stream.testkit.OnError -import akka.stream.testkit.OnSubscribe import akka.stream.scaladsl.Flow -import scala.concurrent.Future -import scala.concurrent.Promise +import akka.stream.testkit.{ AkkaSpec, StreamTestKit } + +import scala.concurrent.{ Future, Promise } +import scala.concurrent.duration._ @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class FlowFromFutureSpec extends AkkaSpec { @@ -22,31 +16,31 @@ class FlowFromFutureSpec extends AkkaSpec { "A Flow based on a Future" must { "produce one element from already successful Future" in { - val p = Flow(Future.successful(1)).toProducer(materializer) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = Flow(Future.successful(1)).toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) val sub = c.expectSubscription() c.expectNoMsg(100.millis) - sub.requestMore(1) + sub.request(1) c.expectNext(1) c.expectComplete() } "produce error from already failed Future" in { val ex = new RuntimeException("test") - val p = Flow(Future.failed[Int](ex)).toProducer(materializer) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = Flow(Future.failed[Int](ex)).toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) c.expectError(ex) } "produce one element when Future is completed" in { val promise = Promise[Int]() - val p = Flow(promise.future).toProducer(materializer) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = Flow(promise.future).toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) val sub = c.expectSubscription() - sub.requestMore(1) + sub.request(1) c.expectNoMsg(100.millis) promise.success(1) c.expectNext(1) @@ -56,29 +50,29 @@ class FlowFromFutureSpec extends AkkaSpec { "produce one element when Future is completed but not before request" in { val promise = Promise[Int]() - val p = Flow(promise.future).toProducer(materializer) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = Flow(promise.future).toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) val sub = c.expectSubscription() promise.success(1) c.expectNoMsg(200.millis) - sub.requestMore(1) + sub.request(1) c.expectNext(1) c.expectComplete() } "produce elements with multiple subscribers" in { val promise = Promise[Int]() - val p = Flow(promise.future).toProducer(materializer) - val c1 = StreamTestKit.consumerProbe[Int] - val c2 = StreamTestKit.consumerProbe[Int] - p.produceTo(c1) - p.produceTo(c2) + val p = Flow(promise.future).toPublisher(materializer) + val c1 = StreamTestKit.SubscriberProbe[Int]() + val c2 = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c1) + p.subscribe(c2) val sub1 = c1.expectSubscription() val sub2 = c2.expectSubscription() - sub1.requestMore(1) + sub1.request(1) promise.success(1) - sub2.requestMore(2) + sub2.request(2) c1.expectNext(1) c2.expectNext(1) c1.expectComplete() @@ -87,34 +81,34 @@ class FlowFromFutureSpec extends AkkaSpec { "produce elements to later subscriber" in { val promise = Promise[Int]() - val p = Flow(promise.future).toProducer(materializer) - val keepAlive = StreamTestKit.consumerProbe[Int] - val c1 = StreamTestKit.consumerProbe[Int] - val c2 = StreamTestKit.consumerProbe[Int] - p.produceTo(keepAlive) - p.produceTo(c1) + val p = Flow(promise.future).toPublisher(materializer) + val keepAlive = StreamTestKit.SubscriberProbe[Int]() + val c1 = StreamTestKit.SubscriberProbe[Int]() + val c2 = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(keepAlive) + p.subscribe(c1) val sub1 = c1.expectSubscription() - sub1.requestMore(1) + sub1.request(1) promise.success(1) c1.expectNext(1) c1.expectComplete() - p.produceTo(c2) + p.subscribe(c2) val sub2 = c2.expectSubscription() - sub2.requestMore(1) + sub2.request(1) c2.expectNext(1) c2.expectComplete() } "allow cancel before receiving element" in { val promise = Promise[Int]() - val p = Flow(promise.future).toProducer(materializer) - val keepAlive = StreamTestKit.consumerProbe[Int] - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(keepAlive) - p.produceTo(c) + val p = Flow(promise.future).toPublisher(materializer) + val keepAlive = StreamTestKit.SubscriberProbe[Int]() + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(keepAlive) + p.subscribe(c) val sub = c.expectSubscription() - sub.requestMore(1) + sub.request(1) sub.cancel() c.expectNoMsg(500.millis) promise.success(1) diff --git a/akka-stream/src/test/scala/akka/stream/FlowGroupBySpec.scala b/akka-stream/src/test/scala/akka/stream/FlowGroupBySpec.scala index 61530b9fed..7104a80bbe 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowGroupBySpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowGroupBySpec.scala @@ -5,7 +5,7 @@ package akka.stream import scala.concurrent.duration._ import akka.stream.testkit._ -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import akka.stream.scaladsl.Flow import scala.util.control.NoStackTrace @@ -19,12 +19,12 @@ class FlowGroupBySpec extends AkkaSpec { maxFanOutBufferSize = 2, dispatcher = "akka.test.stream-dispatcher")) - case class StreamPuppet(p: Producer[Int]) { - val probe = StreamTestKit.consumerProbe[Int] - p.produceTo(probe) + case class StreamPuppet(p: Publisher[Int]) { + val probe = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(probe) val subscription = probe.expectSubscription() - def requestMore(demand: Int): Unit = subscription.requestMore(demand) + def request(demand: Int): Unit = subscription.request(demand) def expectNext(elem: Int): Unit = probe.expectNext(elem) def expectNoMsg(max: FiniteDuration): Unit = probe.expectNoMsg(max) def expectComplete(): Unit = probe.expectComplete() @@ -33,20 +33,20 @@ class FlowGroupBySpec extends AkkaSpec { } class SubstreamsSupport(groupCount: Int = 2, elementCount: Int = 6) { - val source = Flow((1 to elementCount).iterator).toProducer(materializer) - val groupStream = Flow(source).groupBy(_ % groupCount).toProducer(materializer) - val masterConsumer = StreamTestKit.consumerProbe[(Int, Producer[Int])] + val source = Flow((1 to elementCount).iterator).toPublisher(materializer) + val groupStream = Flow(source).groupBy(_ % groupCount).toPublisher(materializer) + val masterSubscriber = StreamTestKit.SubscriberProbe[(Int, Publisher[Int])]() - groupStream.produceTo(masterConsumer) - val masterSubscription = masterConsumer.expectSubscription() + groupStream.subscribe(masterSubscriber) + val masterSubscription = masterSubscriber.expectSubscription() - def getSubproducer(expectedKey: Int): Producer[Int] = { - masterSubscription.requestMore(1) - expectSubproducer(expectedKey: Int) + def getSubPublisher(expectedKey: Int): Publisher[Int] = { + masterSubscription.request(1) + expectSubPublisher(expectedKey: Int) } - def expectSubproducer(expectedKey: Int): Producer[Int] = { - val (key, substream) = masterConsumer.expectNext() + def expectSubPublisher(expectedKey: Int): Publisher[Int] = { + val (key, substream) = masterSubscriber.expectNext() key should be(expectedKey) substream } @@ -57,63 +57,63 @@ class FlowGroupBySpec extends AkkaSpec { "groupBy" must { "work in the happy case" in new SubstreamsSupport(groupCount = 2) { - val s1 = StreamPuppet(getSubproducer(1)) - masterConsumer.expectNoMsg(100.millis) + val s1 = StreamPuppet(getSubPublisher(1)) + masterSubscriber.expectNoMsg(100.millis) s1.expectNoMsg(100.millis) - s1.requestMore(1) + s1.request(1) s1.expectNext(1) s1.expectNoMsg(100.millis) - val s2 = StreamPuppet(getSubproducer(0)) + val s2 = StreamPuppet(getSubPublisher(0)) s2.expectNoMsg(100.millis) - s2.requestMore(2) + s2.request(2) s2.expectNext(2) s2.expectNext(4) s2.expectNoMsg(100.millis) - s1.requestMore(1) + s1.request(1) s1.expectNext(3) - s2.requestMore(1) + s2.request(1) s2.expectNext(6) s2.expectComplete() - s1.requestMore(1) + s1.request(1) s1.expectNext(5) s1.expectComplete() - masterConsumer.expectComplete() + masterSubscriber.expectComplete() } "accept cancellation of substreams" in new SubstreamsSupport(groupCount = 2) { - StreamPuppet(getSubproducer(1)).cancel() + StreamPuppet(getSubPublisher(1)).cancel() - val substream = StreamPuppet(getSubproducer(0)) - substream.requestMore(2) + val substream = StreamPuppet(getSubPublisher(0)) + substream.request(2) substream.expectNext(2) substream.expectNext(4) substream.expectNoMsg(100.millis) - substream.requestMore(2) + substream.request(2) substream.expectNext(6) substream.expectComplete() - masterConsumer.expectComplete() + masterSubscriber.expectComplete() } "accept cancellation of master stream when not consumed anything" in { - val producerProbe = StreamTestKit.producerProbe[Int] - val producer = Flow(producerProbe).groupBy(_ % 2).toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[(Int, Producer[Int])] - producer.produceTo(consumer) + val publisherProbeProbe = StreamTestKit.PublisherProbe[Int]() + val publisher = Flow(publisherProbeProbe).groupBy(_ % 2).toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[(Int, Publisher[Int])]() + publisher.subscribe(subscriber) - val upstreamSubscription = producerProbe.expectSubscription() - val downstreamSubscription = consumer.expectSubscription() + val upstreamSubscription = publisherProbeProbe.expectSubscription() + val downstreamSubscription = subscriber.expectSubscription() downstreamSubscription.cancel() upstreamSubscription.expectCancellation() } @@ -123,14 +123,14 @@ class FlowGroupBySpec extends AkkaSpec { // FIXME: Needs handling of loose substreams that no one refers to anymore. // val substream = StreamPuppet(getSubproducer(1)) // - // substream.requestMore(1) + // substream.request(1) // substream.expectNext(1) // // masterSubscription.cancel() - // masterConsumer.expectNoMsg(100.millis) + // masterSubscriber.expectNoMsg(100.millis) // // // Open substreams still work, others are discarded - // substream.requestMore(4) + // substream.request(4) // substream.expectNext(4) // substream.expectNext(7) // substream.expectNext(10) @@ -139,115 +139,115 @@ class FlowGroupBySpec extends AkkaSpec { } "work with fanout on substreams" in new SubstreamsSupport(groupCount = 2) { - val substreamProducer = getSubproducer(1) - getSubproducer(0) + val substreamPublisher = getSubPublisher(1) + getSubPublisher(0) - val substreamConsumer1 = StreamPuppet(substreamProducer) - val substreamConsumer2 = StreamPuppet(substreamProducer) + val substreamSubscriber1 = StreamPuppet(substreamPublisher) + val substreamSubscriber2 = StreamPuppet(substreamPublisher) - substreamConsumer1.requestMore(1) - substreamConsumer1.expectNext(1) - substreamConsumer2.requestMore(1) - substreamConsumer2.expectNext(1) + substreamSubscriber1.request(1) + substreamSubscriber1.expectNext(1) + substreamSubscriber2.request(1) + substreamSubscriber2.expectNext(1) - substreamConsumer1.requestMore(1) - substreamConsumer1.expectNext(3) - substreamConsumer2.requestMore(1) - substreamConsumer2.expectNext(3) + substreamSubscriber1.request(1) + substreamSubscriber1.expectNext(3) + substreamSubscriber2.request(1) + substreamSubscriber2.expectNext(3) } "work with fanout on master stream" in { - val source = Flow((1 to 4).iterator).toProducer(materializer) - val groupStream = Flow(source).groupBy(_ % 2).toProducer(materializer) - val masterConsumer1 = StreamTestKit.consumerProbe[(Int, Producer[Int])] - val masterConsumer2 = StreamTestKit.consumerProbe[(Int, Producer[Int])] + val source = Flow((1 to 4).iterator).toPublisher(materializer) + val groupStream = Flow(source).groupBy(_ % 2).toPublisher(materializer) + val masterSubscriber1 = StreamTestKit.SubscriberProbe[(Int, Publisher[Int])]() + val masterSubscriber2 = StreamTestKit.SubscriberProbe[(Int, Publisher[Int])]() - groupStream.produceTo(masterConsumer1) - groupStream.produceTo(masterConsumer2) + groupStream.subscribe(masterSubscriber1) + groupStream.subscribe(masterSubscriber2) - val masterSubscription1 = masterConsumer1.expectSubscription() - val masterSubscription2 = masterConsumer2.expectSubscription() + val masterSubscription1 = masterSubscriber1.expectSubscription() + val masterSubscription2 = masterSubscriber2.expectSubscription() - masterSubscription1.requestMore(2) - masterSubscription2.requestMore(1) + masterSubscription1.request(2) + masterSubscription2.request(1) - val (key11, substream11) = masterConsumer1.expectNext() + val (key11, substream11) = masterSubscriber1.expectNext() key11 should be(1) - val (key21, substream21) = masterConsumer2.expectNext() + val (key21, substream21) = masterSubscriber2.expectNext() key21 should be(1) val puppet11 = StreamPuppet(substream11) val puppet21 = StreamPuppet(substream21) - puppet11.requestMore(2) + puppet11.request(2) puppet11.expectNext(1) puppet11.expectNext(3) - puppet21.requestMore(1) + puppet21.request(1) puppet21.expectNext(1) puppet21.cancel() masterSubscription2.cancel() - val (key12, substream12) = masterConsumer1.expectNext() + val (key12, substream12) = masterSubscriber1.expectNext() key12 should be(0) val puppet12 = StreamPuppet(substream12) - puppet12.requestMore(1) + puppet12.request(1) puppet12.expectNext(2) puppet12.cancel() masterSubscription1.cancel() } "work with empty input stream" in { - val producer = Flow(List.empty[Int]).groupBy(_ % 2).toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[(Int, Producer[Int])] - producer.produceTo(consumer) + val publisher = Flow(List.empty[Int]).groupBy(_ % 2).toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[(Int, Publisher[Int])]() + publisher.subscribe(subscriber) - consumer.expectCompletedOrSubscriptionFollowedByComplete() + subscriber.expectCompletedOrSubscriptionFollowedByComplete() } "abort on onError from upstream" in { - val producerProbe = StreamTestKit.producerProbe[Int] - val producer = Flow(producerProbe).groupBy(_ % 2).toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[(Int, Producer[Int])] - producer.produceTo(consumer) + val publisherProbeProbe = StreamTestKit.PublisherProbe[Int]() + val publisher = Flow(publisherProbeProbe).groupBy(_ % 2).toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[(Int, Publisher[Int])]() + publisher.subscribe(subscriber) - val upstreamSubscription = producerProbe.expectSubscription() + val upstreamSubscription = publisherProbeProbe.expectSubscription() - val downstreamSubscription = consumer.expectSubscription() - downstreamSubscription.requestMore(100) + val downstreamSubscription = subscriber.expectSubscription() + downstreamSubscription.request(100) val e = TE("test") upstreamSubscription.sendError(e) - consumer.expectError(e) + subscriber.expectError(e) } "abort on onError from upstream when substreams are running" in { - val producerProbe = StreamTestKit.producerProbe[Int] - val producer = Flow(producerProbe).groupBy(_ % 2).toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[(Int, Producer[Int])] - producer.produceTo(consumer) + val publisherProbeProbe = StreamTestKit.PublisherProbe[Int]() + val publisher = Flow(publisherProbeProbe).groupBy(_ % 2).toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[(Int, Publisher[Int])]() + publisher.subscribe(subscriber) - val upstreamSubscription = producerProbe.expectSubscription() + val upstreamSubscription = publisherProbeProbe.expectSubscription() - val downstreamSubscription = consumer.expectSubscription() - downstreamSubscription.requestMore(100) + val downstreamSubscription = subscriber.expectSubscription() + downstreamSubscription.request(100) upstreamSubscription.sendNext(1) - val (_, substream) = consumer.expectNext() + val (_, substream) = subscriber.expectNext() val substreamPuppet = StreamPuppet(substream) - substreamPuppet.requestMore(1) + substreamPuppet.request(1) substreamPuppet.expectNext(1) val e = TE("test") upstreamSubscription.sendError(e) substreamPuppet.expectError(e) - consumer.expectError(e) + subscriber.expectError(e) } } diff --git a/akka-stream/src/test/scala/akka/stream/FlowGroupedWithinSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowGroupedWithinSpec.scala index 3540f46c2d..22ec2ad136 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowGroupedWithinSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowGroupedWithinSpec.scala @@ -21,23 +21,23 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest { "group elements within the duration" in { val input = Iterator.from(1) - val p = StreamTestKit.producerProbe[Int] - val c = StreamTestKit.consumerProbe[immutable.Seq[Int]] + val p = StreamTestKit.PublisherProbe[Int]() + val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]() Flow(p).groupedWithin(1000, 1.second).produceTo(materializer, c) val pSub = p.expectSubscription val cSub = c.expectSubscription - cSub.requestMore(100) - val demand1 = pSub.expectRequestMore + cSub.request(100) + val demand1 = pSub.expectRequest (1 to demand1) foreach { _ ⇒ pSub.sendNext(input.next()) } - val demand2 = pSub.expectRequestMore + val demand2 = pSub.expectRequest (1 to demand2) foreach { _ ⇒ pSub.sendNext(input.next()) } - val demand3 = pSub.expectRequestMore + val demand3 = pSub.expectRequest c.expectNext((1 to (demand1 + demand2)).toVector) (1 to demand3) foreach { _ ⇒ pSub.sendNext(input.next()) } c.expectNoMsg(300.millis) c.expectNext(((demand1 + demand2 + 1) to (demand1 + demand2 + demand3)).toVector) c.expectNoMsg(300.millis) - pSub.expectRequestMore + pSub.expectRequest val last = input.next() pSub.sendNext(last) pSub.sendComplete() @@ -47,10 +47,10 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest { } "deliver bufferd elements onComplete before the timeout" in { - val c = StreamTestKit.consumerProbe[immutable.Seq[Int]] + val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]() Flow(1 to 3).groupedWithin(1000, 10.second).produceTo(materializer, c) val cSub = c.expectSubscription - cSub.requestMore(100) + cSub.request(100) c.expectNext((1 to 3).toList) c.expectComplete c.expectNoMsg(200.millis) @@ -58,19 +58,19 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest { "buffer groups until requested from downstream" in { val input = Iterator.from(1) - val p = StreamTestKit.producerProbe[Int] - val c = StreamTestKit.consumerProbe[immutable.Seq[Int]] + val p = StreamTestKit.PublisherProbe[Int]() + val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]() Flow(p).groupedWithin(1000, 1.second).produceTo(materializer, c) val pSub = p.expectSubscription val cSub = c.expectSubscription - cSub.requestMore(1) - val demand1 = pSub.expectRequestMore + cSub.request(1) + val demand1 = pSub.expectRequest (1 to demand1) foreach { _ ⇒ pSub.sendNext(input.next()) } c.expectNext((1 to demand1).toVector) - val demand2 = pSub.expectRequestMore + val demand2 = pSub.expectRequest (1 to demand2) foreach { _ ⇒ pSub.sendNext(input.next()) } c.expectNoMsg(300.millis) - cSub.requestMore(1) + cSub.request(1) c.expectNext(((demand1 + 1) to (demand1 + demand2)).toVector) pSub.sendComplete() c.expectComplete @@ -78,21 +78,20 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest { } "drop empty groups" in { - val input = Iterator.from(1) - val p = StreamTestKit.producerProbe[Int] - val c = StreamTestKit.consumerProbe[immutable.Seq[Int]] + val p = StreamTestKit.PublisherProbe[Int]() + val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]() Flow(p).groupedWithin(1000, 500.millis).produceTo(materializer, c) val pSub = p.expectSubscription val cSub = c.expectSubscription - cSub.requestMore(2) - pSub.expectRequestMore + cSub.request(2) + pSub.expectRequest c.expectNoMsg(600.millis) pSub.sendNext(1) pSub.sendNext(2) c.expectNext(List(1, 2)) // nothing more requested c.expectNoMsg(1100.millis) - cSub.requestMore(3) + cSub.request(3) c.expectNoMsg(600.millis) pSub.sendComplete() c.expectComplete @@ -101,13 +100,13 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest { "reset time window when max elements reached" in { val input = Iterator.from(1) - val p = StreamTestKit.producerProbe[Int] - val c = StreamTestKit.consumerProbe[immutable.Seq[Int]] + val p = StreamTestKit.PublisherProbe[Int]() + val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]() Flow(p).groupedWithin(3, 2.second).produceTo(materializer, c) val pSub = p.expectSubscription val cSub = c.expectSubscription - cSub.requestMore(4) - val demand1 = pSub.expectRequestMore + cSub.request(4) + val demand1 = pSub.expectRequest demand1 should be(4) c.expectNoMsg(1000.millis) (1 to demand1) foreach { _ ⇒ pSub.sendNext(input.next()) } diff --git a/akka-stream/src/test/scala/akka/stream/FlowIterableSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowIterableSpec.scala index 8987e21e14..93639889aa 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowIterableSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowIterableSpec.scala @@ -3,15 +3,11 @@ */ package akka.stream -import scala.concurrent.duration._ -import akka.stream.testkit.StreamTestKit -import akka.stream.testkit.AkkaSpec -import akka.stream.testkit.OnNext -import akka.dispatch.OnComplete -import akka.stream.testkit.OnComplete -import akka.stream.testkit.OnError -import akka.stream.testkit.OnSubscribe import akka.stream.scaladsl.Flow +import akka.stream.testkit.{ AkkaSpec, StreamTestKit } +import akka.stream.testkit.StreamTestKit.{ OnComplete, OnError, OnNext } + +import scala.concurrent.duration._ @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class FlowIterableSpec extends AkkaSpec { @@ -22,48 +18,48 @@ class FlowIterableSpec extends AkkaSpec { "A Flow based on an iterable" must { "produce elements" in { - val p = Flow(List(1, 2, 3)).toProducer(materializer) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = Flow(List(1, 2, 3)).toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) val sub = c.expectSubscription() - sub.requestMore(1) + sub.request(1) c.expectNext(1) c.expectNoMsg(100.millis) - sub.requestMore(2) + sub.request(2) c.expectNext(2) c.expectNext(3) c.expectComplete() } "complete empty" in { - val p = Flow(List.empty[Int]).toProducer(materializer) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = Flow(List.empty[Int]).toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) c.expectComplete() c.expectNoMsg(100.millis) - val c2 = StreamTestKit.consumerProbe[Int] - p.produceTo(c2) + val c2 = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c2) c2.expectComplete() } "produce elements with multiple subscribers" in { - val p = Flow(List(1, 2, 3)).toProducer(materializer) - val c1 = StreamTestKit.consumerProbe[Int] - val c2 = StreamTestKit.consumerProbe[Int] - p.produceTo(c1) - p.produceTo(c2) + val p = Flow(List(1, 2, 3)).toPublisher(materializer) + val c1 = StreamTestKit.SubscriberProbe[Int]() + val c2 = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c1) + p.subscribe(c2) val sub1 = c1.expectSubscription() val sub2 = c2.expectSubscription() - sub1.requestMore(1) - sub2.requestMore(2) + sub1.request(1) + sub2.request(2) c1.expectNext(1) c2.expectNext(1) c2.expectNext(2) c1.expectNoMsg(100.millis) c2.expectNoMsg(100.millis) - sub1.requestMore(2) - sub2.requestMore(2) + sub1.request(2) + sub2.request(2) c1.expectNext(2) c1.expectNext(3) c2.expectNext(3) @@ -72,37 +68,37 @@ class FlowIterableSpec extends AkkaSpec { } "produce elements to later subscriber" in { - val p = Flow(List(1, 2, 3)).toProducer(materializer) - val c1 = StreamTestKit.consumerProbe[Int] - val c2 = StreamTestKit.consumerProbe[Int] - p.produceTo(c1) + val p = Flow(List(1, 2, 3)).toPublisher(materializer) + val c1 = StreamTestKit.SubscriberProbe[Int]() + val c2 = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c1) val sub1 = c1.expectSubscription() - sub1.requestMore(1) + sub1.request(1) c1.expectNext(1) c1.expectNoMsg(100.millis) - p.produceTo(c2) + p.subscribe(c2) val sub2 = c2.expectSubscription() - sub2.requestMore(2) + sub2.request(2) // starting from first element, new iterator per subscriber c2.expectNext(1) c2.expectNext(2) c2.expectNoMsg(100.millis) - sub2.requestMore(1) + sub2.request(1) c2.expectNext(3) c2.expectComplete() - sub1.requestMore(2) + sub1.request(2) c1.expectNext(2) c1.expectNext(3) c1.expectComplete() } "produce elements with one transformation step" in { - val p = Flow(List(1, 2, 3)).map(_ * 2).toProducer(materializer) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = Flow(List(1, 2, 3)).map(_ * 2).toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) val sub = c.expectSubscription() - sub.requestMore(10) + sub.request(10) c.expectNext(2) c.expectNext(4) c.expectNext(6) @@ -110,11 +106,11 @@ class FlowIterableSpec extends AkkaSpec { } "produce elements with two transformation steps" in { - val p = Flow(List(1, 2, 3, 4)).filter(_ % 2 == 0).map(_ * 2).toProducer(materializer) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = Flow(List(1, 2, 3, 4)).filter(_ % 2 == 0).map(_ * 2).toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) val sub = c.expectSubscription() - sub.requestMore(10) + sub.request(10) c.expectNext(4) c.expectNext(8) c.expectComplete() @@ -122,11 +118,11 @@ class FlowIterableSpec extends AkkaSpec { "allow cancel before receiving all elements" in { val count = 100000 - val p = Flow(1 to count).toProducer(materializer) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = Flow(1 to count).toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) val sub = c.expectSubscription() - sub.requestMore(count) + sub.request(count) c.expectNext(1) sub.cancel() val got = c.probe.receiveWhile(3.seconds) { @@ -137,20 +133,20 @@ class FlowIterableSpec extends AkkaSpec { got.size should be < (count - 1) } - "have value equality of producer" in { - val p1 = Flow(List(1, 2, 3)).toProducer(materializer) - val p2 = Flow(List(1, 2, 3)).toProducer(materializer) + "have value equality of publisher" in { + val p1 = Flow(List(1, 2, 3)).toPublisher(materializer) + val p2 = Flow(List(1, 2, 3)).toPublisher(materializer) p1 should be(p2) p2 should be(p1) - val p3 = Flow(List(1, 2, 3, 4)).toProducer(materializer) + val p3 = Flow(List(1, 2, 3, 4)).toPublisher(materializer) p1 should not be (p3) p3 should not be (p1) - val p4 = Flow(Vector.empty[String]).toProducer(materializer) - val p5 = Flow(Set.empty[String]).toProducer(materializer) + val p4 = Flow(Vector.empty[String]).toPublisher(materializer) + val p5 = Flow(Set.empty[String]).toPublisher(materializer) p1 should not be (p4) p4 should be(p5) p5 should be(p4) - val p6 = Flow(List(1, 2, 3).iterator).toProducer(materializer) + val p6 = Flow(List(1, 2, 3).iterator).toPublisher(materializer) p1 should not be (p6) p6 should not be (p1) } diff --git a/akka-stream/src/test/scala/akka/stream/FlowIteratorSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowIteratorSpec.scala index 286b95d5f6..4792dea6d6 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowIteratorSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowIteratorSpec.scala @@ -6,9 +6,9 @@ package akka.stream import scala.concurrent.duration._ import akka.stream.testkit.StreamTestKit import akka.stream.testkit.AkkaSpec -import akka.stream.testkit.OnNext -import akka.stream.testkit.OnComplete -import akka.stream.testkit.OnError +import akka.stream.testkit.StreamTestKit.OnNext +import akka.stream.testkit.StreamTestKit.OnComplete +import akka.stream.testkit.StreamTestKit.OnError import akka.stream.scaladsl.Flow @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @@ -23,48 +23,48 @@ class FlowIteratorSpec extends AkkaSpec { "A Flow based on an iterator" must { "produce elements" in { - val p = Flow(List(1, 2, 3).iterator).toProducer(materializer) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = Flow(List(1, 2, 3).iterator).toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) val sub = c.expectSubscription() - sub.requestMore(1) + sub.request(1) c.expectNext(1) c.expectNoMsg(100.millis) - sub.requestMore(3) + sub.request(3) c.expectNext(2) c.expectNext(3) c.expectComplete() } "complete empty" in { - val p = Flow(List.empty[Int].iterator).toProducer(materializer) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = Flow(List.empty[Int].iterator).toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) c.expectComplete() c.expectNoMsg(100.millis) - val c2 = StreamTestKit.consumerProbe[Int] - p.produceTo(c2) + val c2 = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c2) c2.expectComplete() } "produce elements with multiple subscribers" in { - val p = Flow(List(1, 2, 3).iterator).toProducer(materializer) - val c1 = StreamTestKit.consumerProbe[Int] - val c2 = StreamTestKit.consumerProbe[Int] - p.produceTo(c1) - p.produceTo(c2) + val p = Flow(List(1, 2, 3).iterator).toPublisher(materializer) + val c1 = StreamTestKit.SubscriberProbe[Int]() + val c2 = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c1) + p.subscribe(c2) val sub1 = c1.expectSubscription() val sub2 = c2.expectSubscription() - sub1.requestMore(1) - sub2.requestMore(2) + sub1.request(1) + sub2.request(2) c1.expectNext(1) c2.expectNext(1) c2.expectNext(2) c1.expectNoMsg(100.millis) c2.expectNoMsg(100.millis) - sub1.requestMore(2) - sub2.requestMore(2) + sub1.request(2) + sub2.request(2) c1.expectNext(2) c1.expectNext(3) c2.expectNext(3) @@ -73,34 +73,34 @@ class FlowIteratorSpec extends AkkaSpec { } "produce elements to later subscriber" in { - val p = Flow(List(1, 2, 3).iterator).toProducer(materializer) - val c1 = StreamTestKit.consumerProbe[Int] - val c2 = StreamTestKit.consumerProbe[Int] - p.produceTo(c1) + val p = Flow(List(1, 2, 3).iterator).toPublisher(materializer) + val c1 = StreamTestKit.SubscriberProbe[Int]() + val c2 = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c1) val sub1 = c1.expectSubscription() - sub1.requestMore(1) + sub1.request(1) c1.expectNext(1) c1.expectNoMsg(100.millis) - p.produceTo(c2) + p.subscribe(c2) val sub2 = c2.expectSubscription() - sub2.requestMore(3) + sub2.request(3) // element 1 is already gone c2.expectNext(2) c2.expectNext(3) c2.expectComplete() - sub1.requestMore(3) + sub1.request(3) c1.expectNext(2) c1.expectNext(3) c1.expectComplete() } "produce elements with one transformation step" in { - val p = Flow(List(1, 2, 3).iterator).map(_ * 2).toProducer(materializer) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = Flow(List(1, 2, 3).iterator).map(_ * 2).toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) val sub = c.expectSubscription() - sub.requestMore(10) + sub.request(10) c.expectNext(2) c.expectNext(4) c.expectNext(6) @@ -108,11 +108,11 @@ class FlowIteratorSpec extends AkkaSpec { } "produce elements with two transformation steps" in { - val p = Flow(List(1, 2, 3, 4).iterator).filter(_ % 2 == 0).map(_ * 2).toProducer(materializer) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = Flow(List(1, 2, 3, 4).iterator).filter(_ % 2 == 0).map(_ * 2).toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) val sub = c.expectSubscription() - sub.requestMore(10) + sub.request(10) c.expectNext(4) c.expectNext(8) c.expectComplete() @@ -120,11 +120,11 @@ class FlowIteratorSpec extends AkkaSpec { "allow cancel before receiving all elements" in { val count = 100000 - val p = Flow((1 to count).iterator).toProducer(materializer) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = Flow((1 to count).iterator).toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) val sub = c.expectSubscription() - sub.requestMore(count) + sub.request(count) c.expectNext(1) sub.cancel() val got = c.probe.receiveWhile(3.seconds) { diff --git a/akka-stream/src/test/scala/akka/stream/FlowMapFutureSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowMapFutureSpec.scala index c50d6a7e2d..bcf4736da7 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowMapFutureSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowMapFutureSpec.scala @@ -23,35 +23,35 @@ class FlowMapFutureSpec extends AkkaSpec { "A Flow with mapFuture" must { "produce future elements" in { - val c = StreamTestKit.consumerProbe[Int] + val c = StreamTestKit.SubscriberProbe[Int]() implicit val ec = system.dispatcher val p = Flow(1 to 3).mapFuture(n ⇒ Future(n)).produceTo(materializer, c) val sub = c.expectSubscription() - sub.requestMore(2) + sub.request(2) c.expectNext(1) c.expectNext(2) c.expectNoMsg(200.millis) - sub.requestMore(2) + sub.request(2) c.expectNext(3) c.expectComplete() } "produce future elements in order" in { - val c = StreamTestKit.consumerProbe[Int] + val c = StreamTestKit.SubscriberProbe[Int]() implicit val ec = system.dispatcher val p = Flow(1 to 50).mapFuture(n ⇒ Future { Thread.sleep(ThreadLocalRandom.current().nextInt(1, 10)) n }).produceTo(materializer, c) val sub = c.expectSubscription() - sub.requestMore(1000) + sub.request(1000) for (n ← 1 to 50) c.expectNext(n) c.expectComplete() } "not run more futures than requested elements" in { val probe = TestProbe() - val c = StreamTestKit.consumerProbe[Int] + val c = StreamTestKit.SubscriberProbe[Int]() implicit val ec = system.dispatcher val p = Flow(1 to 20).mapFuture(n ⇒ Future { probe.ref ! n @@ -60,13 +60,13 @@ class FlowMapFutureSpec extends AkkaSpec { val sub = c.expectSubscription() // nothing before requested probe.expectNoMsg(500.millis) - sub.requestMore(1) + sub.request(1) probe.expectMsg(1) probe.expectNoMsg(500.millis) - sub.requestMore(2) + sub.request(2) probe.receiveN(2).toSet should be(Set(2, 3)) probe.expectNoMsg(500.millis) - sub.requestMore(10) + sub.request(10) probe.receiveN(10).toSet should be((4 to 13).toSet) probe.expectNoMsg(200.millis) @@ -76,7 +76,7 @@ class FlowMapFutureSpec extends AkkaSpec { "signal future failure" in { val latch = TestLatch(1) - val c = StreamTestKit.consumerProbe[Int] + val c = StreamTestKit.SubscriberProbe[Int]() implicit val ec = system.dispatcher val p = Flow(1 to 5).mapFuture(n ⇒ Future { if (n == 3) throw new RuntimeException("err1") with NoStackTrace @@ -86,14 +86,14 @@ class FlowMapFutureSpec extends AkkaSpec { } }).produceTo(materializer, c) val sub = c.expectSubscription() - sub.requestMore(10) + sub.request(10) c.expectError.getMessage should be("err1") latch.countDown() } "signal error from mapFuture" in { val latch = TestLatch(1) - val c = StreamTestKit.consumerProbe[Int] + val c = StreamTestKit.SubscriberProbe[Int]() implicit val ec = system.dispatcher val p = Flow(1 to 5).mapFuture(n ⇒ if (n == 3) throw new RuntimeException("err2") with NoStackTrace @@ -105,7 +105,7 @@ class FlowMapFutureSpec extends AkkaSpec { }). produceTo(materializer, c) val sub = c.expectSubscription() - sub.requestMore(10) + sub.request(10) c.expectError.getMessage should be("err2") latch.countDown() } diff --git a/akka-stream/src/test/scala/akka/stream/FlowMapSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowMapSpec.scala index 3f21a72626..3e21d88459 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowMapSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowMapSpec.scala @@ -28,14 +28,14 @@ class FlowMapSpec extends AkkaSpec with ScriptedTest { } "not blow up with high request counts" in { - val probe = StreamTestKit.consumerProbe[Int] + val probe = StreamTestKit.SubscriberProbe[Int]() Flow(List(1).iterator). map(_ + 1).map(_ + 1).map(_ + 1).map(_ + 1).map(_ + 1). - toProducer(gen).produceTo(probe) + toPublisher(gen).subscribe(probe) val subscription = probe.expectSubscription() for (_ ← 1 to 10000) { - subscription.requestMore(Int.MaxValue) + subscription.request(Int.MaxValue) } probe.expectNext(6) diff --git a/akka-stream/src/test/scala/akka/stream/FlowMergeSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowMergeSpec.scala index 9e3041b2e7..6afac6d0cb 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowMergeSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowMergeSpec.scala @@ -3,33 +3,34 @@ */ package akka.stream +import org.reactivestreams.Publisher + import scala.concurrent.duration._ import akka.stream.testkit.StreamTestKit import akka.stream.testkit.AkkaSpec -import org.reactivestreams.api.Producer import akka.stream.scaladsl.Flow class FlowMergeSpec extends TwoStreamsSetup { type Outputs = Int - override def operationUnderTest(in1: Flow[Int], in2: Producer[Int]) = in1.merge(in2) + override def operationUnderTest(in1: Flow[Int], in2: Publisher[Int]) = in1.merge(in2) "merge" must { "work in the happy case" in { // Different input sizes (4 and 6) - val source1 = Flow((1 to 4).iterator).toProducer(materializer) - val source2 = Flow((5 to 10).iterator).toProducer(materializer) - val source3 = Flow(List.empty[Int].iterator).toProducer(materializer) - val p = Flow(source1).merge(source2).merge(source3).toProducer(materializer) + val source1 = Flow((1 to 4).iterator).toPublisher(materializer) + val source2 = Flow((5 to 10).iterator).toPublisher(materializer) + val source3 = Flow(List.empty[Int].iterator).toPublisher(materializer) + val p = Flow(source1).merge(source2).merge(source3).toPublisher(materializer) - val probe = StreamTestKit.consumerProbe[Int] - p.produceTo(probe) + val probe = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(probe) val subscription = probe.expectSubscription() var collected = Set.empty[Int] for (_ ← 1 to 10) { - subscription.requestMore(1) + subscription.request(1) collected += probe.expectNext() } @@ -39,52 +40,52 @@ class FlowMergeSpec extends TwoStreamsSetup { commonTests() - "work with one immediately completed and one nonempty producer" in { - val consumer1 = setup(completedPublisher, nonemptyPublisher((1 to 4).iterator)) - val subscription1 = consumer1.expectSubscription() - subscription1.requestMore(4) - consumer1.expectNext(1) - consumer1.expectNext(2) - consumer1.expectNext(3) - consumer1.expectNext(4) - consumer1.expectComplete() + "work with one immediately completed and one nonempty publisher" in { + val subscriber1 = setup(completedPublisher, nonemptyPublisher((1 to 4).iterator)) + val subscription1 = subscriber1.expectSubscription() + subscription1.request(4) + subscriber1.expectNext(1) + subscriber1.expectNext(2) + subscriber1.expectNext(3) + subscriber1.expectNext(4) + subscriber1.expectComplete() - val consumer2 = setup(nonemptyPublisher((1 to 4).iterator), completedPublisher) - val subscription2 = consumer2.expectSubscription() - subscription2.requestMore(4) - consumer2.expectNext(1) - consumer2.expectNext(2) - consumer2.expectNext(3) - consumer2.expectNext(4) - consumer2.expectComplete() + val subscriber2 = setup(nonemptyPublisher((1 to 4).iterator), completedPublisher) + val subscription2 = subscriber2.expectSubscription() + subscription2.request(4) + subscriber2.expectNext(1) + subscriber2.expectNext(2) + subscriber2.expectNext(3) + subscriber2.expectNext(4) + subscriber2.expectComplete() } - "work with one delayed completed and one nonempty producer" in { - val consumer1 = setup(soonToCompletePublisher, nonemptyPublisher((1 to 4).iterator)) - val subscription1 = consumer1.expectSubscription() - subscription1.requestMore(4) - consumer1.expectNext(1) - consumer1.expectNext(2) - consumer1.expectNext(3) - consumer1.expectNext(4) - consumer1.expectComplete() + "work with one delayed completed and one nonempty publisher" in { + val subscriber1 = setup(soonToCompletePublisher, nonemptyPublisher((1 to 4).iterator)) + val subscription1 = subscriber1.expectSubscription() + subscription1.request(4) + subscriber1.expectNext(1) + subscriber1.expectNext(2) + subscriber1.expectNext(3) + subscriber1.expectNext(4) + subscriber1.expectComplete() - val consumer2 = setup(nonemptyPublisher((1 to 4).iterator), soonToCompletePublisher) - val subscription2 = consumer2.expectSubscription() - subscription2.requestMore(4) - consumer2.expectNext(1) - consumer2.expectNext(2) - consumer2.expectNext(3) - consumer2.expectNext(4) - consumer2.expectComplete() + val subscriber2 = setup(nonemptyPublisher((1 to 4).iterator), soonToCompletePublisher) + val subscription2 = subscriber2.expectSubscription() + subscription2.request(4) + subscriber2.expectNext(1) + subscriber2.expectNext(2) + subscriber2.expectNext(3) + subscriber2.expectNext(4) + subscriber2.expectComplete() } - "work with one immediately failed and one nonempty producer" in { + "work with one immediately failed and one nonempty publisher" in { // This is nondeterministic, multiple scenarios can happen pending } - "work with one delayed failed and one nonempty producer" in { + "work with one delayed failed and one nonempty publisher" in { // This is nondeterministic, multiple scenarios can happen pending } diff --git a/akka-stream/src/test/scala/akka/stream/FlowOnCompleteSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowOnCompleteSpec.scala index 6276f8bebc..6e7d17a8a6 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowOnCompleteSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowOnCompleteSpec.scala @@ -30,10 +30,10 @@ class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest { "invoke callback on normal completion" in { val onCompleteProbe = TestProbe() - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() Flow(p).onComplete(materializer) { onCompleteProbe.ref ! _ } val proc = p.expectSubscription - proc.expectRequestMore() + proc.expectRequest() proc.sendNext(42) onCompleteProbe.expectNoMsg(100.millis) proc.sendComplete() @@ -42,10 +42,10 @@ class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest { "yield the first error" in { val onCompleteProbe = TestProbe() - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() Flow(p).onComplete(materializer) { onCompleteProbe.ref ! _ } val proc = p.expectSubscription - proc.expectRequestMore() + proc.expectRequest() val ex = new RuntimeException("ex") with NoStackTrace proc.sendError(ex) onCompleteProbe.expectMsg(Failure(ex)) @@ -54,10 +54,10 @@ class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest { "invoke callback for an empty stream" in { val onCompleteProbe = TestProbe() - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() Flow(p).onComplete(materializer) { onCompleteProbe.ref ! _ } val proc = p.expectSubscription - proc.expectRequestMore() + proc.expectRequest() proc.sendComplete() onCompleteProbe.expectMsg(Success(())) onCompleteProbe.expectNoMsg(100.millis) @@ -65,7 +65,7 @@ class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest { "invoke callback after transform and foreach steps " in { val onCompleteProbe = TestProbe() - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() Flow(p).map { x ⇒ onCompleteProbe.ref ! ("map-" + x) x @@ -73,7 +73,7 @@ class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest { x ⇒ onCompleteProbe.ref ! ("foreach-" + x) }.onComplete(materializer) { onCompleteProbe.ref ! _ } val proc = p.expectSubscription - proc.expectRequestMore() + proc.expectRequest() proc.sendNext(42) proc.sendComplete() onCompleteProbe.expectMsg("map-42") diff --git a/akka-stream/src/test/scala/akka/stream/FlowPrefixAndTailSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowPrefixAndTailSpec.scala index 6a1d756e3b..8afd5ae4c8 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowPrefixAndTailSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowPrefixAndTailSpec.scala @@ -3,12 +3,13 @@ */ package akka.stream -import akka.stream.testkit.{ StreamTestKit, AkkaSpec } +import akka.stream.impl.EmptyPublisher import akka.stream.scaladsl.Flow +import akka.stream.testkit.{ AkkaSpec, StreamTestKit } +import org.reactivestreams.Publisher + import scala.concurrent.Await import scala.concurrent.duration._ -import akka.stream.impl.EmptyProducer -import org.reactivestreams.api.Producer import scala.util.control.NoStackTrace class FlowPrefixAndTailSpec extends AkkaSpec { @@ -25,11 +26,11 @@ class FlowPrefixAndTailSpec extends AkkaSpec { val testException = new Exception("test") with NoStackTrace "work on empty input" in { - Await.result(Flow(Nil).prefixAndTail(10).toFuture(m), 3.seconds) should be((Nil, EmptyProducer)) + Await.result(Flow(Nil).prefixAndTail(10).toFuture(m), 3.seconds) should be((Nil, EmptyPublisher)) } "work on short input" in { - Await.result(Flow(List(1, 2, 3)).prefixAndTail(10).toFuture(m), 3.seconds) should be((List(1, 2, 3), EmptyProducer)) + Await.result(Flow(List(1, 2, 3)).prefixAndTail(10).toFuture(m), 3.seconds) should be((List(1, 2, 3), EmptyPublisher)) } "work on longer inputs" in { @@ -47,68 +48,68 @@ class FlowPrefixAndTailSpec extends AkkaSpec { "work if size of take is equals to stream size" in { val (takes, tail) = Await.result(Flow((1 to 10).iterator).prefixAndTail(10).toFuture(m), 3.seconds) takes should be(1 to 10) - val consumer = StreamTestKit.consumerProbe[Int] - Flow(tail).produceTo(m, consumer) - consumer.expectCompletedOrSubscriptionFollowedByComplete() + val subscriber = StreamTestKit.SubscriberProbe[Int]() + Flow(tail).produceTo(m, subscriber) + subscriber.expectCompletedOrSubscriptionFollowedByComplete() } "handle onError when no substream open" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[(Seq[Int], Producer[Int])] + val publisher = StreamTestKit.PublisherProbe[Int]() + val subscriber = StreamTestKit.SubscriberProbe[(Seq[Int], Publisher[Int])]() - Flow(producer).prefixAndTail(3).produceTo(m, consumer) + Flow(publisher).prefixAndTail(3).produceTo(m, subscriber) - val upstream = producer.expectSubscription() - val downstream = consumer.expectSubscription() + val upstream = publisher.expectSubscription() + val downstream = subscriber.expectSubscription() - downstream.requestMore(1) + downstream.request(1) - upstream.expectRequestMore() + upstream.expectRequest() upstream.sendNext(1) upstream.sendError(testException) - consumer.expectError(testException) + subscriber.expectError(testException) } "handle onError when substream is open" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[(Seq[Int], Producer[Int])] + val publisher = StreamTestKit.PublisherProbe[Int]() + val subscriber = StreamTestKit.SubscriberProbe[(Seq[Int], Publisher[Int])]() - Flow(producer).prefixAndTail(1).produceTo(m, consumer) + Flow(publisher).prefixAndTail(1).produceTo(m, subscriber) - val upstream = producer.expectSubscription() - val downstream = consumer.expectSubscription() + val upstream = publisher.expectSubscription() + val downstream = subscriber.expectSubscription() - downstream.requestMore(1000) + downstream.request(1000) - upstream.expectRequestMore() + upstream.expectRequest() upstream.sendNext(1) - val (head, tail) = consumer.expectNext() + val (head, tail) = subscriber.expectNext() head should be(List(1)) - consumer.expectComplete() + subscriber.expectComplete() - val substreamConsumer = StreamTestKit.consumerProbe[Int] - Flow(tail).produceTo(m, substreamConsumer) - val subUpstream = substreamConsumer.expectSubscription() + val substreamSubscriber = StreamTestKit.SubscriberProbe[Int]() + Flow(tail).produceTo(m, substreamSubscriber) + substreamSubscriber.expectSubscription() upstream.sendError(testException) - substreamConsumer.expectError(testException) + substreamSubscriber.expectError(testException) } "handle master stream cancellation" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[(Seq[Int], Producer[Int])] + val publisher = StreamTestKit.PublisherProbe[Int]() + val subscriber = StreamTestKit.SubscriberProbe[(Seq[Int], Publisher[Int])]() - Flow(producer).prefixAndTail(3).produceTo(m, consumer) + Flow(publisher).prefixAndTail(3).produceTo(m, subscriber) - val upstream = producer.expectSubscription() - val downstream = consumer.expectSubscription() + val upstream = publisher.expectSubscription() + val downstream = subscriber.expectSubscription() - downstream.requestMore(1) + downstream.request(1) - upstream.expectRequestMore() + upstream.expectRequest() upstream.sendNext(1) downstream.cancel() @@ -116,26 +117,26 @@ class FlowPrefixAndTailSpec extends AkkaSpec { } "handle substream cancellation" in { - val producer = StreamTestKit.producerProbe[Int] - val consumer = StreamTestKit.consumerProbe[(Seq[Int], Producer[Int])] + val publisher = StreamTestKit.PublisherProbe[Int]() + val subscriber = StreamTestKit.SubscriberProbe[(Seq[Int], Publisher[Int])]() - Flow(producer).prefixAndTail(1).produceTo(m, consumer) + Flow(publisher).prefixAndTail(1).produceTo(m, subscriber) - val upstream = producer.expectSubscription() - val downstream = consumer.expectSubscription() + val upstream = publisher.expectSubscription() + val downstream = subscriber.expectSubscription() - downstream.requestMore(1000) + downstream.request(1000) - upstream.expectRequestMore() + upstream.expectRequest() upstream.sendNext(1) - val (head, tail) = consumer.expectNext() + val (head, tail) = subscriber.expectNext() head should be(List(1)) - consumer.expectComplete() + subscriber.expectComplete() - val substreamConsumer = StreamTestKit.consumerProbe[Int] - Flow(tail).produceTo(m, substreamConsumer) - substreamConsumer.expectSubscription().cancel() + val substreamSubscriber = StreamTestKit.SubscriberProbe[Int]() + Flow(tail).produceTo(m, substreamSubscriber) + substreamSubscriber.expectSubscription().cancel() upstream.expectCancellation() diff --git a/akka-stream/src/test/scala/akka/stream/FlowProduceToConsumerSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowProduceToSubscriberSpec.scala similarity index 71% rename from akka-stream/src/test/scala/akka/stream/FlowProduceToConsumerSpec.scala rename to akka-stream/src/test/scala/akka/stream/FlowProduceToSubscriberSpec.scala index 1154070819..c4b92316f0 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowProduceToConsumerSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowProduceToSubscriberSpec.scala @@ -7,17 +7,17 @@ import akka.stream.scaladsl.Flow import akka.stream.testkit.AkkaSpec import akka.stream.testkit.StreamTestKit -class FlowProduceToConsumerSpec extends AkkaSpec { +class FlowProduceToSubscriberSpec extends AkkaSpec { val materializer = FlowMaterializer(MaterializerSettings(dispatcher = "akka.test.stream-dispatcher")) - "A Flow with toProducer" must { + "A Flow with toPublisher" must { - "produce elements to the consumer" in { - val c = StreamTestKit.consumerProbe[Int] + "produce elements to the subscriber" in { + val c = StreamTestKit.SubscriberProbe[Int]() Flow(List(1, 2, 3)).produceTo(materializer, c) val s = c.expectSubscription() - s.requestMore(3) + s.request(3) c.expectNext(1) c.expectNext(2) c.expectNext(3) diff --git a/akka-stream/src/test/scala/akka/stream/FlowSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowSpec.scala index dbf6952aba..00e403655c 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowSpec.scala @@ -6,8 +6,6 @@ package akka.stream import scala.concurrent.duration._ import akka.stream.testkit.{ AkkaSpec, ChainSetup, StreamTestKit } import akka.testkit._ -import org.reactivestreams.api.Producer -import org.scalatest.FreeSpecLike import com.typesafe.config.ConfigFactory import akka.stream.scaladsl.Flow @@ -31,21 +29,21 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece for ((name, op) ← List("identity" -> identity, "identity2" -> identity2); n ← List(1, 2, 4)) { s"requests initial elements from upstream ($name, $n)" in { new ChainSetup(op, settings.copy(initialInputBufferSize = n)) { - upstream.expectRequestMore(upstreamSubscription, settings.initialInputBufferSize) + upstream.expectRequest(upstreamSubscription, settings.initialInputBufferSize) } } } "requests more elements from upstream when downstream requests more elements" in { new ChainSetup(identity, settings) { - upstream.expectRequestMore(upstreamSubscription, settings.initialInputBufferSize) - downstreamSubscription.requestMore(1) + upstream.expectRequest(upstreamSubscription, settings.initialInputBufferSize) + downstreamSubscription.request(1) upstream.expectNoMsg(100.millis) - downstreamSubscription.requestMore(2) + downstreamSubscription.request(2) upstream.expectNoMsg(100.millis) upstreamSubscription.sendNext("a") downstream.expectNext("a") - upstream.expectRequestMore(upstreamSubscription, 1) + upstream.expectRequest(upstreamSubscription, 1) upstream.expectNoMsg(100.millis) upstreamSubscription.sendNext("b") upstreamSubscription.sendNext("c") @@ -57,7 +55,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece "deliver events when publisher sends elements and then completes" in { new ChainSetup(identity, settings) { - downstreamSubscription.requestMore(1) + downstreamSubscription.request(1) upstreamSubscription.sendNext("test") upstreamSubscription.sendComplete() downstream.expectNext("test") @@ -84,12 +82,12 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece "single subscriber cancels subscription while receiving data" in { new ChainSetup(identity, settings.copy(initialInputBufferSize = 1)) { - downstreamSubscription.requestMore(5) - upstreamSubscription.expectRequestMore(1) + downstreamSubscription.request(5) + upstreamSubscription.expectRequest(1) upstreamSubscription.sendNext("test") - upstreamSubscription.expectRequestMore(1) + upstreamSubscription.expectRequest(1) upstreamSubscription.sendNext("test2") - upstreamSubscription.expectRequestMore(1) + upstreamSubscription.expectRequest(1) downstream.expectNext("test") downstream.expectNext("test2") downstreamSubscription.cancel() @@ -102,62 +100,62 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece } "A Flow with multiple subscribers (FanOutBox)" must { - "adapt speed to the currently slowest consumer" in { + "adapt speed to the currently slowest subscriber" in { new ChainSetup(identity, settings.copy(initialInputBufferSize = 1, maxFanOutBufferSize = 1)) { - val downstream2 = StreamTestKit.consumerProbe[Any]() - producer.produceTo(downstream2) + val downstream2 = StreamTestKit.SubscriberProbe[Any]() + publisher.subscribe(downstream2) val downstream2Subscription = downstream2.expectSubscription() - downstreamSubscription.requestMore(5) - upstream.expectRequestMore(upstreamSubscription, 1) // because initialInputBufferSize=1 + downstreamSubscription.request(5) + upstream.expectRequest(upstreamSubscription, 1) // because initialInputBufferSize=1 upstreamSubscription.sendNext("firstElement") downstream.expectNext("firstElement") - upstream.expectRequestMore(upstreamSubscription, 1) + upstream.expectRequest(upstreamSubscription, 1) upstreamSubscription.sendNext("element2") downstream.expectNoMsg(1.second) - downstream2Subscription.requestMore(1) + downstream2Subscription.request(1) downstream2.expectNext("firstElement") downstream.expectNext("element2") - downstream2Subscription.requestMore(1) + downstream2Subscription.request(1) downstream2.expectNext("element2") } } - "support slow consumer with fan-out 2" in { + "support slow subscriber with fan-out 2" in { new ChainSetup(identity, settings.copy(initialInputBufferSize = 1, initialFanOutBufferSize = 2, maxFanOutBufferSize = 2)) { - val downstream2 = StreamTestKit.consumerProbe[Any]() - producer.produceTo(downstream2) + val downstream2 = StreamTestKit.SubscriberProbe[Any]() + publisher.subscribe(downstream2) val downstream2Subscription = downstream2.expectSubscription() - downstreamSubscription.requestMore(5) + downstreamSubscription.request(5) - upstream.expectRequestMore(upstreamSubscription, 1) // because initialInputBufferSize=1 + upstream.expectRequest(upstreamSubscription, 1) // because initialInputBufferSize=1 upstreamSubscription.sendNext("element1") downstream.expectNext("element1") - upstreamSubscription.expectRequestMore(1) + upstreamSubscription.expectRequest(1) upstreamSubscription.sendNext("element2") downstream.expectNext("element2") - upstreamSubscription.expectRequestMore(1) + upstreamSubscription.expectRequest(1) upstreamSubscription.sendNext("element3") // downstream2 has not requested anything, fan-out buffer 2 downstream.expectNoMsg(100.millis.dilated) - downstream2Subscription.requestMore(2) + downstream2Subscription.request(2) downstream.expectNext("element3") downstream2.expectNext("element1") downstream2.expectNext("element2") downstream2.expectNoMsg(100.millis.dilated) - upstreamSubscription.expectRequestMore(1) + upstreamSubscription.expectRequest(1) upstreamSubscription.sendNext("element4") downstream.expectNext("element4") - downstream2Subscription.requestMore(2) + downstream2Subscription.request(2) downstream2.expectNext("element3") downstream2.expectNext("element4") @@ -169,20 +167,20 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece "incoming subscriber while elements were requested before" in { new ChainSetup(identity, settings.copy(initialInputBufferSize = 1, maxFanOutBufferSize = 1)) { - downstreamSubscription.requestMore(5) - upstream.expectRequestMore(upstreamSubscription, 1) + downstreamSubscription.request(5) + upstream.expectRequest(upstreamSubscription, 1) upstreamSubscription.sendNext("a1") downstream.expectNext("a1") - upstream.expectRequestMore(upstreamSubscription, 1) + upstream.expectRequest(upstreamSubscription, 1) upstreamSubscription.sendNext("a2") downstream.expectNext("a2") - upstream.expectRequestMore(upstreamSubscription, 1) + upstream.expectRequest(upstreamSubscription, 1) // link now while an upstream element is already requested - val downstream2 = StreamTestKit.consumerProbe[Any]() - producer.produceTo(downstream2) + val downstream2 = StreamTestKit.SubscriberProbe[Any]() + publisher.subscribe(downstream2) val downstream2Subscription = downstream2.expectSubscription() // situation here: @@ -193,35 +191,35 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece downstream.expectNext("a3") downstream2.expectNoMsg(100.millis.dilated) // as nothing was requested yet, fanOutBox needs to cache element in this case - downstream2Subscription.requestMore(1) + downstream2Subscription.request(1) downstream2.expectNext("a3") // d1 now has 2 outstanding // d2 now has 0 outstanding // buffer should be empty so we should be requesting one new element - upstream.expectRequestMore(upstreamSubscription, 1) // because of buffer size 1 + upstream.expectRequest(upstreamSubscription, 1) // because of buffer size 1 } } "blocking subscriber cancels subscription" in { new ChainSetup(identity, settings.copy(initialInputBufferSize = 1, maxFanOutBufferSize = 1)) { - val downstream2 = StreamTestKit.consumerProbe[Any]() - producer.produceTo(downstream2) + val downstream2 = StreamTestKit.SubscriberProbe[Any]() + publisher.subscribe(downstream2) val downstream2Subscription = downstream2.expectSubscription() - downstreamSubscription.requestMore(5) - upstreamSubscription.expectRequestMore(1) + downstreamSubscription.request(5) + upstreamSubscription.expectRequest(1) upstreamSubscription.sendNext("firstElement") downstream.expectNext("firstElement") - downstream2Subscription.requestMore(1) + downstream2Subscription.request(1) downstream2.expectNext("firstElement") - upstreamSubscription.expectRequestMore(1) + upstreamSubscription.expectRequest(1) upstreamSubscription.sendNext("element2") downstream.expectNext("element2") - upstreamSubscription.expectRequestMore(1) + upstreamSubscription.expectRequest(1) upstreamSubscription.sendNext("element3") downstream2.expectNoMsg(100.millis.dilated) @@ -229,7 +227,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece upstream.expectNoMsg(100.millis.dilated) // should unblock fanoutbox downstream2Subscription.cancel() - upstreamSubscription.expectRequestMore(1) + upstreamSubscription.expectRequest(1) downstream.expectNext("element3") upstreamSubscription.sendNext("element4") downstream.expectNext("element4") @@ -241,22 +239,22 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece "after initial upstream was completed future subscribers' onComplete should be called instead of onSubscribed" in { new ChainSetup(identity, settings.copy(initialInputBufferSize = 1, maxFanOutBufferSize = 1)) { - val downstream2 = StreamTestKit.consumerProbe[Any]() + val downstream2 = StreamTestKit.SubscriberProbe[Any]() // don't link it just yet - downstreamSubscription.requestMore(5) - upstream.expectRequestMore(upstreamSubscription, 1) + downstreamSubscription.request(5) + upstream.expectRequest(upstreamSubscription, 1) upstreamSubscription.sendNext("a1") downstream.expectNext("a1") - upstream.expectRequestMore(upstreamSubscription, 1) + upstream.expectRequest(upstreamSubscription, 1) upstreamSubscription.sendNext("a2") downstream.expectNext("a2") - upstream.expectRequestMore(upstreamSubscription, 1) + upstream.expectRequest(upstreamSubscription, 1) // link now while an upstream element is already requested - producer.produceTo(downstream2) + publisher.subscribe(downstream2) val downstream2Subscription = downstream2.expectSubscription() upstreamSubscription.sendNext("a3") @@ -266,44 +264,44 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece downstream2.expectNoMsg(100.millis.dilated) // as nothing was requested yet, fanOutBox needs to cache element in this case - downstream2Subscription.requestMore(1) + downstream2Subscription.request(1) downstream2.expectNext("a3") downstream2.expectComplete() // FIXME when adding a sleep before the following link this will fail with IllegalStateExc shut-down // what is the expected shutdown behavior? Is the title of this test wrong? - // val downstream3 = StreamTestKit.consumerProbe[Any]() - // producer.produceTo(downstream3) + // val downstream3 = StreamTestKit.SubscriberProbe[Any]() + // publisher.subscribe(downstream3) // downstream3.expectComplete() } } "after initial upstream reported an error future subscribers' onError should be called instead of onSubscribed" in { new ChainSetup[Int, String](_.map(_ ⇒ throw TestException), settings.copy(initialInputBufferSize = 1, maxFanOutBufferSize = 1)) { - downstreamSubscription.requestMore(1) - upstreamSubscription.expectRequestMore(1) + downstreamSubscription.request(1) + upstreamSubscription.expectRequest(1) EventFilter[TestException.type](occurrences = 1) intercept { upstreamSubscription.sendNext(5) - upstreamSubscription.expectRequestMore(1) + upstreamSubscription.expectRequest(1) upstreamSubscription.expectCancellation() downstream.expectError(TestException) } - val downstream2 = StreamTestKit.consumerProbe[String]() - producer.produceTo(downstream2) + val downstream2 = StreamTestKit.SubscriberProbe[String]() + publisher.subscribe(downstream2) downstream2.expectError() should be(TestException) } } "when all subscriptions were cancelled future subscribers' onError should be called" in { new ChainSetup(identity, settings.copy(initialInputBufferSize = 1)) { - upstreamSubscription.expectRequestMore(1) + upstreamSubscription.expectRequest(1) downstreamSubscription.cancel() upstreamSubscription.expectCancellation() - val downstream2 = StreamTestKit.consumerProbe[Any]() - producer.produceTo(downstream2) + val downstream2 = StreamTestKit.SubscriberProbe[Any]() + publisher.subscribe(downstream2) // IllegalStateException shut down downstream2.expectError().isInstanceOf[IllegalStateException] should be(true) } diff --git a/akka-stream/src/test/scala/akka/stream/FlowSplitWhenSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowSplitWhenSpec.scala index e39a3c2bf7..962af313ad 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowSplitWhenSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowSplitWhenSpec.scala @@ -6,7 +6,7 @@ package akka.stream import scala.concurrent.duration._ import akka.stream.testkit.StreamTestKit import akka.stream.testkit.AkkaSpec -import org.reactivestreams.api.Producer +import org.reactivestreams.Publisher import akka.stream.scaladsl.Flow @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @@ -19,12 +19,12 @@ class FlowSplitWhenSpec extends AkkaSpec { maxFanOutBufferSize = 2, dispatcher = "akka.test.stream-dispatcher")) - case class StreamPuppet(p: Producer[Int]) { - val probe = StreamTestKit.consumerProbe[Int] - p.produceTo(probe) + case class StreamPuppet(p: Publisher[Int]) { + val probe = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(probe) val subscription = probe.expectSubscription() - def requestMore(demand: Int): Unit = subscription.requestMore(demand) + def request(demand: Int): Unit = subscription.request(demand) def expectNext(elem: Int): Unit = probe.expectNext(elem) def expectNoMsg(max: FiniteDuration): Unit = probe.expectNoMsg(max) def expectComplete(): Unit = probe.expectComplete() @@ -32,20 +32,20 @@ class FlowSplitWhenSpec extends AkkaSpec { } class SubstreamsSupport(splitWhen: Int = 3, elementCount: Int = 6) { - val source = Flow((1 to elementCount).iterator).toProducer(materializer) - val groupStream = Flow(source).splitWhen(_ == splitWhen).toProducer(materializer) - val masterConsumer = StreamTestKit.consumerProbe[Producer[Int]] + val source = Flow((1 to elementCount).iterator).toPublisher(materializer) + val groupStream = Flow(source).splitWhen(_ == splitWhen).toPublisher(materializer) + val masterSubscriber = StreamTestKit.SubscriberProbe[Publisher[Int]]() - groupStream.produceTo(masterConsumer) - val masterSubscription = masterConsumer.expectSubscription() + groupStream.subscribe(masterSubscriber) + val masterSubscription = masterSubscriber.expectSubscription() - def getSubproducer(): Producer[Int] = { - masterSubscription.requestMore(1) - expectSubproducer() + def getSubPublisher(): Publisher[Int] = { + masterSubscription.request(1) + expectSubPublisher() } - def expectSubproducer(): Producer[Int] = { - val substream = masterConsumer.expectNext() + def expectSubPublisher(): Publisher[Int] = { + val substream = masterSubscriber.expectNext() substream } @@ -54,46 +54,46 @@ class FlowSplitWhenSpec extends AkkaSpec { "splitWhen" must { "work in the happy case" in new SubstreamsSupport(elementCount = 4) { - val s1 = StreamPuppet(getSubproducer()) - masterConsumer.expectNoMsg(100.millis) + val s1 = StreamPuppet(getSubPublisher()) + masterSubscriber.expectNoMsg(100.millis) - s1.requestMore(2) + s1.request(2) s1.expectNext(1) s1.expectNext(2) s1.expectComplete() - val s2 = StreamPuppet(getSubproducer()) - masterConsumer.expectComplete() + val s2 = StreamPuppet(getSubPublisher()) + masterSubscriber.expectComplete() - s2.requestMore(1) + s2.request(1) s2.expectNext(3) s2.expectNoMsg(100.millis) - s2.requestMore(1) + s2.request(1) s2.expectNext(4) s2.expectComplete() } "support cancelling substreams" in new SubstreamsSupport(splitWhen = 5, elementCount = 8) { - val s1 = StreamPuppet(getSubproducer()) + val s1 = StreamPuppet(getSubPublisher()) s1.cancel() - val s2 = StreamPuppet(getSubproducer()) + val s2 = StreamPuppet(getSubPublisher()) - s2.requestMore(4) + s2.request(4) s2.expectNext(5) s2.expectNext(6) s2.expectNext(7) s2.expectNext(8) s2.expectComplete() - masterConsumer.expectComplete() + masterSubscriber.expectComplete() } "support cancelling the master stream" in new SubstreamsSupport(splitWhen = 5, elementCount = 8) { - val s1 = StreamPuppet(getSubproducer()) + val s1 = StreamPuppet(getSubPublisher()) masterSubscription.cancel() - s1.requestMore(4) + s1.request(4) s1.expectNext(1) s1.expectNext(2) s1.expectNext(3) diff --git a/akka-stream/src/test/scala/akka/stream/FlowTakeSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowTakeSpec.scala index 493adac17b..75732e67a0 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowTakeSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowTakeSpec.scala @@ -6,7 +6,7 @@ package akka.stream import akka.stream.testkit.AkkaSpec import akka.stream.testkit.ScriptedTest import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random } -import akka.stream.actor.ActorConsumer.{ OnNext, OnComplete } +import akka.stream.actor.ActorSubscriber.{ OnNext, OnComplete } import akka.stream.impl.RequestMore class FlowTakeSpec extends AkkaSpec with ScriptedTest { diff --git a/akka-stream/src/test/scala/akka/stream/FlowTakeWithinSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowTakeWithinSpec.scala index 3cf34bb975..c1a8d80090 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowTakeWithinSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowTakeWithinSpec.scala @@ -18,17 +18,17 @@ class FlowTakeWithinSpec extends AkkaSpec { "deliver elements within the duration, but not afterwards" in { val input = Iterator.from(1) - val p = StreamTestKit.producerProbe[Int] - val c = StreamTestKit.consumerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() + val c = StreamTestKit.SubscriberProbe[Int]() Flow(p).takeWithin(1.second).produceTo(materializer, c) val pSub = p.expectSubscription val cSub = c.expectSubscription - cSub.requestMore(100) - val demand1 = pSub.expectRequestMore + cSub.request(100) + val demand1 = pSub.expectRequest (1 to demand1) foreach { _ ⇒ pSub.sendNext(input.next()) } - val demand2 = pSub.expectRequestMore + val demand2 = pSub.expectRequest (1 to demand2) foreach { _ ⇒ pSub.sendNext(input.next()) } - val demand3 = pSub.expectRequestMore + val demand3 = pSub.expectRequest val sentN = demand1 + demand2 (1 to sentN) foreach { n ⇒ c.expectNext(n) } within(2.seconds) { @@ -39,12 +39,11 @@ class FlowTakeWithinSpec extends AkkaSpec { } "deliver bufferd elements onComplete before the timeout" in { - val input = Iterator.from(1) - val c = StreamTestKit.consumerProbe[Int] + val c = StreamTestKit.SubscriberProbe[Int]() Flow(1 to 3).takeWithin(1.second).produceTo(materializer, c) val cSub = c.expectSubscription c.expectNoMsg(200.millis) - cSub.requestMore(100) + cSub.request(100) (1 to 3) foreach { n ⇒ c.expectNext(n) } c.expectComplete c.expectNoMsg(200.millis) diff --git a/akka-stream/src/test/scala/akka/stream/FlowTeeSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowTeeSpec.scala index c768205a1e..d8e42434bc 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowTeeSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowTeeSpec.scala @@ -20,42 +20,42 @@ class FlowTeeSpec extends AkkaSpec { "A Tee" must { - "tee to other consumer" in { - val c1 = StreamTestKit.consumerProbe[Int] - val c2 = StreamTestKit.consumerProbe[Int] + "tee to other subscriber" in { + val c1 = StreamTestKit.SubscriberProbe[Int]() + val c2 = StreamTestKit.SubscriberProbe[Int]() val p = Flow(List(1, 2, 3)). tee(c2). - toProducer(materializer) - p.produceTo(c1) + toPublisher(materializer) + p.subscribe(c1) val sub1 = c1.expectSubscription() val sub2 = c2.expectSubscription() - sub1.requestMore(1) - sub2.requestMore(2) + sub1.request(1) + sub2.request(2) c1.expectNext(1) c1.expectNoMsg(100.millis) c2.expectNext(1) c2.expectNext(2) c2.expectNoMsg(100.millis) - sub1.requestMore(3) + sub1.request(3) c1.expectNext(2) c1.expectNext(3) c1.expectComplete() - sub2.requestMore(3) + sub2.request(3) c2.expectNext(3) c2.expectComplete() } "produce to other even though downstream cancels" in { - val c1 = StreamTestKit.consumerProbe[Int] - val c2 = StreamTestKit.consumerProbe[Int] + val c1 = StreamTestKit.SubscriberProbe[Int]() + val c2 = StreamTestKit.SubscriberProbe[Int]() val p = Flow(List(1, 2, 3)). tee(c2). - toProducer(materializer) - p.produceTo(c1) + toPublisher(materializer) + p.subscribe(c1) val sub1 = c1.expectSubscription() sub1.cancel() val sub2 = c2.expectSubscription() - sub2.requestMore(3) + sub2.request(3) c2.expectNext(1) c2.expectNext(2) c2.expectNext(3) @@ -63,16 +63,16 @@ class FlowTeeSpec extends AkkaSpec { } "produce to downstream even though other cancels" in { - val c1 = StreamTestKit.consumerProbe[Int] - val c2 = StreamTestKit.consumerProbe[Int] + val c1 = StreamTestKit.SubscriberProbe[Int]() + val c2 = StreamTestKit.SubscriberProbe[Int]() val p = Flow(List(1, 2, 3)). tee(c1). - toProducer(materializer) - p.produceTo(c2) + toPublisher(materializer) + p.subscribe(c2) val sub1 = c1.expectSubscription() sub1.cancel() val sub2 = c2.expectSubscription() - sub2.requestMore(3) + sub2.request(3) c2.expectNext(1) c2.expectNext(2) c2.expectNext(3) diff --git a/akka-stream/src/test/scala/akka/stream/FlowTimerTransformerSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowTimerTransformerSpec.scala index 772a185282..39bdf78729 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowTimerTransformerSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowTimerTransformerSpec.scala @@ -22,7 +22,7 @@ class FlowTimerTransformerSpec extends AkkaSpec { "A Flow with TimerTransformer operations" must { "produce scheduled ticks as expected" in { - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() val p2 = Flow(p). transform(new TimerTransformer[Int, Int] { schedulePeriodically("tick", 100.millis) @@ -35,19 +35,19 @@ class FlowTimerTransformerSpec extends AkkaSpec { } override def isComplete: Boolean = !isTimerActive("tick") }). - toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[Int] - p2.produceTo(consumer) - val subscription = consumer.expectSubscription() - subscription.requestMore(5) - consumer.expectNext(1) - consumer.expectNext(2) - consumer.expectNext(3) - consumer.expectComplete() + toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(subscriber) + val subscription = subscriber.expectSubscription() + subscription.request(5) + subscriber.expectNext(1) + subscriber.expectNext(2) + subscriber.expectNext(3) + subscriber.expectComplete() } "schedule ticks when last transformation step (consume)" in { - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() val p2 = Flow(p). transform(new TimerTransformer[Int, Int] { schedulePeriodically("tick", 100.millis) @@ -71,7 +71,7 @@ class FlowTimerTransformerSpec extends AkkaSpec { "propagate error if onTimer throws an exception" in { val exception = new Exception("Expected exception to the rule") with NoStackTrace - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() val p2 = Flow(p). transform(new TimerTransformer[Int, Int] { scheduleOnce("tick", 100.millis) @@ -79,13 +79,13 @@ class FlowTimerTransformerSpec extends AkkaSpec { def onNext(element: Int) = Nil override def onTimer(timerKey: Any) = throw exception - }).toProducer(materializer) + }).toPublisher(materializer) - val consumer = StreamTestKit.consumerProbe[Int] - p2.produceTo(consumer) - val subscription = consumer.expectSubscription() - subscription.requestMore(5) - consumer.expectError(exception) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(subscriber) + val subscription = subscriber.expectSubscription() + subscription.request(5) + subscriber.expectError(exception) } } } diff --git a/akka-stream/src/test/scala/akka/stream/FlowToFutureSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowToFutureSpec.scala index 6db9bde8b8..7bbec76b01 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowToFutureSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowToFutureSpec.scala @@ -24,20 +24,20 @@ class FlowToFutureSpec extends AkkaSpec with ScriptedTest { "A Flow with toFuture" must { "yield the first value" in { - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() val f = Flow(p).toFuture(materializer) val proc = p.expectSubscription - proc.expectRequestMore() + proc.expectRequest() proc.sendNext(42) Await.result(f, 100.millis) should be(42) proc.expectCancellation() } "yield the first error" in { - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() val f = Flow(p).toFuture(materializer) val proc = p.expectSubscription - proc.expectRequestMore() + proc.expectRequest() val ex = new RuntimeException("ex") proc.sendError(ex) Await.ready(f, 100.millis) @@ -45,10 +45,10 @@ class FlowToFutureSpec extends AkkaSpec with ScriptedTest { } "yield NoSuchElementExcption for empty stream" in { - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() val f = Flow(p).toFuture(materializer) val proc = p.expectSubscription - proc.expectRequestMore() + proc.expectRequest() proc.sendComplete() Await.ready(f, 100.millis) f.value.get match { diff --git a/akka-stream/src/test/scala/akka/stream/FlowTransformRecoverSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowTransformRecoverSpec.scala index ae8117da84..5d761aea51 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowTransformRecoverSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowTransformRecoverSpec.scala @@ -40,7 +40,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { "A Flow with transformRecover operations" must { "produce one-to-one transformation as expected" in { - val p = Flow(List(1, 2, 3).iterator).toProducer(materializer) + val p = Flow(List(1, 2, 3).iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[Int, Int] { var tot = 0 @@ -54,21 +54,21 @@ class FlowTransformRecoverSpec extends AkkaSpec { case Some(_) ⇒ List(-1) } }). - toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[Int] - p2.produceTo(consumer) - val subscription = consumer.expectSubscription() - subscription.requestMore(1) - consumer.expectNext(1) - consumer.expectNoMsg(200.millis) - subscription.requestMore(2) - consumer.expectNext(3) - consumer.expectNext(6) - consumer.expectComplete() + toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(subscriber) + val subscription = subscriber.expectSubscription() + subscription.request(1) + subscriber.expectNext(1) + subscriber.expectNoMsg(200.millis) + subscription.request(2) + subscriber.expectNext(3) + subscriber.expectNext(6) + subscriber.expectComplete() } "produce one-to-several transformation as expected" in { - val p = Flow(List(1, 2, 3).iterator).toProducer(materializer) + val p = Flow(List(1, 2, 3).iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[Int, Int] { var tot = 0 @@ -82,24 +82,24 @@ class FlowTransformRecoverSpec extends AkkaSpec { case Some(_) ⇒ List(-1) } }). - toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[Int] - p2.produceTo(consumer) - val subscription = consumer.expectSubscription() - subscription.requestMore(4) - consumer.expectNext(1) - consumer.expectNext(3) - consumer.expectNext(3) - consumer.expectNext(6) - consumer.expectNoMsg(200.millis) - subscription.requestMore(100) - consumer.expectNext(6) - consumer.expectNext(6) - consumer.expectComplete() + toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(subscriber) + val subscription = subscriber.expectSubscription() + subscription.request(4) + subscriber.expectNext(1) + subscriber.expectNext(3) + subscriber.expectNext(3) + subscriber.expectNext(6) + subscriber.expectNoMsg(200.millis) + subscription.request(100) + subscriber.expectNext(6) + subscriber.expectNext(6) + subscriber.expectComplete() } "produce dropping transformation as expected" in { - val p = Flow(List(1, 2, 3, 4).iterator).toProducer(materializer) + val p = Flow(List(1, 2, 3, 4).iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[Int, Int] { var tot = 0 @@ -113,21 +113,21 @@ class FlowTransformRecoverSpec extends AkkaSpec { case Some(_) ⇒ List(-1) } }). - toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[Int] - p2.produceTo(consumer) - val subscription = consumer.expectSubscription() - subscription.requestMore(1) - consumer.expectNext(1) - consumer.expectNoMsg(200.millis) - subscription.requestMore(1) - consumer.expectNext(6) - subscription.requestMore(1) - consumer.expectComplete() + toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(subscriber) + val subscription = subscriber.expectSubscription() + subscription.request(1) + subscriber.expectNext(1) + subscriber.expectNoMsg(200.millis) + subscription.request(1) + subscriber.expectNext(6) + subscription.request(1) + subscriber.expectComplete() } "produce multi-step transformation as expected" in { - val p = Flow(List("a", "bc", "def").iterator).toProducer(materializer) + val p = Flow(List("a", "bc", "def").iterator).toPublisher(materializer) val p2 = Flow(p). transform(new TryRecoveryTransformer[String, Int] { var concat = "" @@ -148,21 +148,21 @@ class FlowTransformRecoverSpec extends AkkaSpec { case Some(_) ⇒ List(-1) } }). - toProducer(materializer) - val c1 = StreamTestKit.consumerProbe[Int] - p2.produceTo(c1) + toPublisher(materializer) + val c1 = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(c1) val sub1 = c1.expectSubscription() - val c2 = StreamTestKit.consumerProbe[Int] - p2.produceTo(c2) + val c2 = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(c2) val sub2 = c2.expectSubscription() - sub1.requestMore(1) - sub2.requestMore(2) + sub1.request(1) + sub2.request(2) c1.expectNext(10) c2.expectNext(10) c2.expectNext(31) c1.expectNoMsg(200.millis) - sub1.requestMore(2) - sub2.requestMore(2) + sub1.request(2) + sub2.request(2) c1.expectNext(31) c1.expectNext(64) c2.expectNext(64) @@ -171,7 +171,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { } "invoke onComplete when done" in { - val p = Flow(List("a").iterator).toProducer(materializer) + val p = Flow(List("a").iterator).toPublisher(materializer) val p2 = Flow(p). transform(new TryRecoveryTransformer[String, String] { var s = "" @@ -181,17 +181,17 @@ class FlowTransformRecoverSpec extends AkkaSpec { } override def onTermination(e: Option[Throwable]) = List(s + "B") }). - toProducer(materializer) - val c = StreamTestKit.consumerProbe[String] - p2.produceTo(c) + toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[String]() + p2.subscribe(c) val s = c.expectSubscription() - s.requestMore(1) + s.request(1) c.expectNext("Success(a)B") c.expectComplete() } "allow cancellation using isComplete" in { - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() val p2 = Flow(p). transform(new TryRecoveryTransformer[Int, Int] { var s = "" @@ -201,12 +201,12 @@ class FlowTransformRecoverSpec extends AkkaSpec { } override def isComplete = s == "Success(1)" }). - toProducer(materializer) + toPublisher(materializer) val proc = p.expectSubscription - val c = StreamTestKit.consumerProbe[Int] - p2.produceTo(c) + val c = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(c) val s = c.expectSubscription() - s.requestMore(10) + s.request(10) proc.sendNext(1) proc.sendNext(2) c.expectNext(1) @@ -215,7 +215,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { } "call onComplete after isComplete signaled completion" in { - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() val p2 = Flow(p). transform(new TryRecoveryTransformer[Int, Int] { var s = "" @@ -226,12 +226,12 @@ class FlowTransformRecoverSpec extends AkkaSpec { override def isComplete = s == "Success(1)" override def onTermination(e: Option[Throwable]) = List(s.length + 10) }). - toProducer(materializer) + toPublisher(materializer) val proc = p.expectSubscription - val c = StreamTestKit.consumerProbe[Int] - p2.produceTo(c) + val c = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(c) val s = c.expectSubscription() - s.requestMore(10) + s.request(10) proc.sendNext(1) proc.sendNext(2) c.expectNext(1) @@ -241,7 +241,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { } "report error when exception is thrown" in { - val p = Flow(List(1, 2, 3).iterator).toProducer(materializer) + val p = Flow(List(1, 2, 3).iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[Int, Int] { override def onNext(elem: Int) = { @@ -250,18 +250,18 @@ class FlowTransformRecoverSpec extends AkkaSpec { } override def onError(e: Throwable) = List(-1) }). - toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[Int] - p2.produceTo(consumer) - val subscription = consumer.expectSubscription() + toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(subscriber) + val subscription = subscriber.expectSubscription() EventFilter[IllegalArgumentException]("two not allowed") intercept { - subscription.requestMore(1) - consumer.expectNext(1) - consumer.expectNoMsg(200.millis) - subscription.requestMore(100) - consumer.expectNext(1) - consumer.expectError().getMessage should be("two not allowed") - consumer.expectNoMsg(200.millis) + subscription.request(1) + subscriber.expectNext(1) + subscriber.expectNoMsg(200.millis) + subscription.request(100) + subscriber.expectNext(1) + subscriber.expectError().getMessage should be("two not allowed") + subscriber.expectNoMsg(200.millis) } } @@ -280,42 +280,42 @@ class FlowTransformRecoverSpec extends AkkaSpec { case Some(_) ⇒ List(-1, -2, -3) } }). - toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[Int] - p2.produceTo(consumer) - val subscription = consumer.expectSubscription() + toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(subscriber) + val subscription = subscriber.expectSubscription() - subscription.requestMore(1) - consumer.expectNext(101) - consumer.expectNoMsg(100.millis) - subscription.requestMore(1) - consumer.expectNext(102) - consumer.expectNoMsg(100.millis) - subscription.requestMore(1) - consumer.expectNext(103) - consumer.expectNoMsg(100.millis) - subscription.requestMore(1) - consumer.expectNext(104) - consumer.expectNoMsg(100.millis) - subscription.requestMore(1) - consumer.expectNext(105) - consumer.expectNoMsg(100.millis) + subscription.request(1) + subscriber.expectNext(101) + subscriber.expectNoMsg(100.millis) + subscription.request(1) + subscriber.expectNext(102) + subscriber.expectNoMsg(100.millis) + subscription.request(1) + subscriber.expectNext(103) + subscriber.expectNoMsg(100.millis) + subscription.request(1) + subscriber.expectNext(104) + subscriber.expectNoMsg(100.millis) + subscription.request(1) + subscriber.expectNext(105) + subscriber.expectNoMsg(100.millis) - subscription.requestMore(1) - consumer.expectNext(-1) - consumer.expectNoMsg(100.millis) - subscription.requestMore(10) - consumer.expectNext(-2) - consumer.expectNext(-3) - consumer.expectComplete() - consumer.expectNoMsg(200.millis) + subscription.request(1) + subscriber.expectNext(-1) + subscriber.expectNoMsg(100.millis) + subscription.request(10) + subscriber.expectNext(-2) + subscriber.expectNext(-3) + subscriber.expectComplete() + subscriber.expectNoMsg(200.millis) } } case class TE(message: String) extends RuntimeException(message) with NoStackTrace "transform errors in sequence with normal messages" in { - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() val p2 = Flow(p). transform(new Transformer[Int, String] { var s = "" @@ -333,33 +333,33 @@ class FlowTransformRecoverSpec extends AkkaSpec { } } }). - toProducer(materializer) + toPublisher(materializer) val proc = p.expectSubscription() - val c = StreamTestKit.consumerProbe[String] - p2.produceTo(c) + val c = StreamTestKit.SubscriberProbe[String]() + p2.subscribe(c) val s = c.expectSubscription() proc.sendNext(0) proc.sendError(TE("1")) // Request late to prove the in-sequence nature - s.requestMore(10) + s.request(10) c.expectNext("0") c.expectNext("01") c.expectComplete() } "forward errors when received and thrown" in { - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() val p2 = Flow(p). transform(new Transformer[Int, Int] { override def onNext(in: Int) = List(in) override def onError(e: Throwable) = throw e }). - toProducer(materializer) + toPublisher(materializer) val proc = p.expectSubscription() - val c = StreamTestKit.consumerProbe[Int] - p2.produceTo(c) + val c = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(c) val s = c.expectSubscription() - s.requestMore(10) + s.request(10) EventFilter[TE](occurrences = 1) intercept { proc.sendError(TE("1")) c.expectError(TE("1")) @@ -367,23 +367,23 @@ class FlowTransformRecoverSpec extends AkkaSpec { } "support cancel as expected" in { - val p = Flow(List(1, 2, 3).iterator).toProducer(materializer) + val p = Flow(List(1, 2, 3).iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[Int, Int] { override def onNext(elem: Int) = List(elem, elem) override def onError(e: Throwable) = List(-1) }). - toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[Int] - p2.produceTo(consumer) - val subscription = consumer.expectSubscription() - subscription.requestMore(2) - consumer.expectNext(1) + toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(subscriber) + val subscription = subscriber.expectSubscription() + subscription.request(2) + subscriber.expectNext(1) subscription.cancel() - consumer.expectNext(1) - consumer.expectNoMsg(500.millis) - subscription.requestMore(2) - consumer.expectNoMsg(200.millis) + subscriber.expectNext(1) + subscriber.expectNoMsg(500.millis) + subscription.request(2) + subscriber.expectNoMsg(200.millis) } } diff --git a/akka-stream/src/test/scala/akka/stream/FlowTransformSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowTransformSpec.scala index 52670021ce..aa864632ef 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowTransformSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowTransformSpec.scala @@ -27,7 +27,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d "A Flow with transform operations" must { "produce one-to-one transformation as expected" in { - val p = Flow(List(1, 2, 3).iterator).toProducer(materializer) + val p = Flow(List(1, 2, 3).iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[Int, Int] { var tot = 0 @@ -36,21 +36,21 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d List(tot) } }). - toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[Int] - p2.produceTo(consumer) - val subscription = consumer.expectSubscription() - subscription.requestMore(1) - consumer.expectNext(1) - consumer.expectNoMsg(200.millis) - subscription.requestMore(2) - consumer.expectNext(3) - consumer.expectNext(6) - consumer.expectComplete() + toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(subscriber) + val subscription = subscriber.expectSubscription() + subscription.request(1) + subscriber.expectNext(1) + subscriber.expectNoMsg(200.millis) + subscription.request(2) + subscriber.expectNext(3) + subscriber.expectNext(6) + subscriber.expectComplete() } "produce one-to-several transformation as expected" in { - val p = Flow(List(1, 2, 3).iterator).toProducer(materializer) + val p = Flow(List(1, 2, 3).iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[Int, Int] { var tot = 0 @@ -59,24 +59,24 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d Vector.fill(elem)(tot) } }). - toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[Int] - p2.produceTo(consumer) - val subscription = consumer.expectSubscription() - subscription.requestMore(4) - consumer.expectNext(1) - consumer.expectNext(3) - consumer.expectNext(3) - consumer.expectNext(6) - consumer.expectNoMsg(200.millis) - subscription.requestMore(100) - consumer.expectNext(6) - consumer.expectNext(6) - consumer.expectComplete() + toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(subscriber) + val subscription = subscriber.expectSubscription() + subscription.request(4) + subscriber.expectNext(1) + subscriber.expectNext(3) + subscriber.expectNext(3) + subscriber.expectNext(6) + subscriber.expectNoMsg(200.millis) + subscription.request(100) + subscriber.expectNext(6) + subscriber.expectNext(6) + subscriber.expectComplete() } "produce dropping transformation as expected" in { - val p = Flow(List(1, 2, 3, 4).iterator).toProducer(materializer) + val p = Flow(List(1, 2, 3, 4).iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[Int, Int] { var tot = 0 @@ -85,21 +85,21 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d if (elem % 2 == 0) Nil else List(tot) } }). - toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[Int] - p2.produceTo(consumer) - val subscription = consumer.expectSubscription() - subscription.requestMore(1) - consumer.expectNext(1) - consumer.expectNoMsg(200.millis) - subscription.requestMore(1) - consumer.expectNext(6) - subscription.requestMore(1) - consumer.expectComplete() + toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(subscriber) + val subscription = subscriber.expectSubscription() + subscription.request(1) + subscriber.expectNext(1) + subscriber.expectNoMsg(200.millis) + subscription.request(1) + subscriber.expectNext(6) + subscription.request(1) + subscriber.expectComplete() } "produce multi-step transformation as expected" in { - val p = Flow(List("a", "bc", "def").iterator).toProducer(materializer) + val p = Flow(List("a", "bc", "def").iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[String, Int] { var concat = "" @@ -115,21 +115,21 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d List(tot) } }). - toProducer(materializer) - val c1 = StreamTestKit.consumerProbe[Int] - p2.produceTo(c1) + toPublisher(materializer) + val c1 = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(c1) val sub1 = c1.expectSubscription() - val c2 = StreamTestKit.consumerProbe[Int] - p2.produceTo(c2) + val c2 = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(c2) val sub2 = c2.expectSubscription() - sub1.requestMore(1) - sub2.requestMore(2) + sub1.request(1) + sub2.request(2) c1.expectNext(1) c2.expectNext(1) c2.expectNext(4) c1.expectNoMsg(200.millis) - sub1.requestMore(2) - sub2.requestMore(2) + sub1.request(2) + sub2.request(2) c1.expectNext(4) c1.expectNext(10) c2.expectNext(10) @@ -138,7 +138,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } "invoke onComplete when done" in { - val p = Flow(List("a").iterator).toProducer(materializer) + val p = Flow(List("a").iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[String, String] { var s = "" @@ -148,18 +148,18 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } override def onTermination(e: Option[Throwable]) = List(s + "B") }). - toProducer(materializer) - val c = StreamTestKit.consumerProbe[String] - p2.produceTo(c) + toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[String]() + p2.subscribe(c) val s = c.expectSubscription() - s.requestMore(1) + s.request(1) c.expectNext("aB") c.expectComplete() } "invoke cleanup when done" in { val cleanupProbe = TestProbe() - val p = Flow(List("a").iterator).toProducer(materializer) + val p = Flow(List("a").iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[String, String] { var s = "" @@ -170,11 +170,11 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d override def onTermination(e: Option[Throwable]) = List(s + "B") override def cleanup() = cleanupProbe.ref ! s }). - toProducer(materializer) - val c = StreamTestKit.consumerProbe[String] - p2.produceTo(c) + toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[String]() + p2.subscribe(c) val s = c.expectSubscription() - s.requestMore(1) + s.request(1) c.expectNext("aB") c.expectComplete() cleanupProbe.expectMsg("a") @@ -182,8 +182,8 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d "invoke cleanup when done consume" in { val cleanupProbe = TestProbe() - val p = Flow(List("a").iterator).toProducer(materializer) - val p2 = Flow(p). + val p = Flow(List("a").iterator).toPublisher(materializer) + Flow(p). transform(new Transformer[String, String] { var s = "x" override def onNext(element: String) = { @@ -198,7 +198,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d "invoke cleanup when done after error" in { val cleanupProbe = TestProbe() - val p = Flow(List("a", "b", "c").iterator).toProducer(materializer) + val p = Flow(List("a", "b", "c").iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[String, String] { var s = "" @@ -213,19 +213,19 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d override def onTermination(e: Option[Throwable]) = List(s + "B") override def cleanup() = cleanupProbe.ref ! s }). - toProducer(materializer) - val c = StreamTestKit.consumerProbe[String] - p2.produceTo(c) + toPublisher(materializer) + val c = StreamTestKit.SubscriberProbe[String]() + p2.subscribe(c) val s = c.expectSubscription() - s.requestMore(1) + s.request(1) c.expectNext("a") - s.requestMore(1) + s.request(1) c.expectError() cleanupProbe.expectMsg("A") } "allow cancellation using isComplete" in { - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() val p2 = Flow(p). transform(new Transformer[Int, Int] { var s = "" @@ -235,12 +235,12 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } override def isComplete = s == "1" }). - toProducer(materializer) + toPublisher(materializer) val proc = p.expectSubscription - val c = StreamTestKit.consumerProbe[Int] - p2.produceTo(c) + val c = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(c) val s = c.expectSubscription() - s.requestMore(10) + s.request(10) proc.sendNext(1) proc.sendNext(2) c.expectNext(1) @@ -250,7 +250,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d "call onComplete after isComplete signaled completion" in { val cleanupProbe = TestProbe() - val p = StreamTestKit.producerProbe[Int] + val p = StreamTestKit.PublisherProbe[Int]() val p2 = Flow(p). transform(new Transformer[Int, Int] { var s = "" @@ -262,12 +262,12 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d override def onTermination(e: Option[Throwable]) = List(s.length + 10) override def cleanup() = cleanupProbe.ref ! s }). - toProducer(materializer) + toPublisher(materializer) val proc = p.expectSubscription - val c = StreamTestKit.consumerProbe[Int] - p2.produceTo(c) + val c = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(c) val s = c.expectSubscription() - s.requestMore(10) + s.request(10) proc.sendNext(1) proc.sendNext(2) c.expectNext(1) @@ -278,8 +278,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } "report error when exception is thrown" in { - val errProbe = TestProbe() - val p = Flow(List(1, 2, 3).iterator).toProducer(materializer) + val p = Flow(List(1, 2, 3).iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[Int, Int] { override def onNext(elem: Int) = { @@ -287,59 +286,59 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d else List(elem, elem) } }). - toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[Int] - p2.produceTo(consumer) - val subscription = consumer.expectSubscription() + toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(subscriber) + val subscription = subscriber.expectSubscription() EventFilter[IllegalArgumentException]("two not allowed") intercept { - subscription.requestMore(100) - consumer.expectNext(1) - consumer.expectNext(1) - consumer.expectError().getMessage should be("two not allowed") - consumer.expectNoMsg(200.millis) + subscription.request(100) + subscriber.expectNext(1) + subscriber.expectNext(1) + subscriber.expectError().getMessage should be("two not allowed") + subscriber.expectNoMsg(200.millis) } } "support cancel as expected" in { - val p = Flow(List(1, 2, 3).iterator).toProducer(materializer) + val p = Flow(List(1, 2, 3).iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[Int, Int] { override def onNext(elem: Int) = List(elem, elem) }). - toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[Int] - p2.produceTo(consumer) - val subscription = consumer.expectSubscription() - subscription.requestMore(2) - consumer.expectNext(1) + toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(subscriber) + val subscription = subscriber.expectSubscription() + subscription.request(2) + subscriber.expectNext(1) subscription.cancel() - consumer.expectNext(1) - consumer.expectNoMsg(500.millis) - subscription.requestMore(2) - consumer.expectNoMsg(200.millis) + subscriber.expectNext(1) + subscriber.expectNoMsg(500.millis) + subscription.request(2) + subscriber.expectNoMsg(200.millis) } "support producing elements from empty inputs" in { - val p = Flow(List.empty[Int].iterator).toProducer(materializer) + val p = Flow(List.empty[Int].iterator).toPublisher(materializer) val p2 = Flow(p). transform(new Transformer[Int, Int] { override def onNext(elem: Int) = Nil override def onTermination(e: Option[Throwable]) = List(1, 2, 3) }). - toProducer(materializer) - val consumer = StreamTestKit.consumerProbe[Int] - p2.produceTo(consumer) - val subscription = consumer.expectSubscription() - subscription.requestMore(4) - consumer.expectNext(1) - consumer.expectNext(2) - consumer.expectNext(3) - consumer.expectComplete() + toPublisher(materializer) + val subscriber = StreamTestKit.SubscriberProbe[Int]() + p2.subscribe(subscriber) + val subscription = subscriber.expectSubscription() + subscription.request(4) + subscriber.expectNext(1) + subscriber.expectNext(2) + subscriber.expectNext(3) + subscriber.expectComplete() } "support converting onComplete into onError" in { - val consumer = StreamTestKit.consumerProbe[Int] + val subscriber = StreamTestKit.SubscriberProbe[Int]() Flow(List(5, 1, 2, 3)).transform(new Transformer[Int, Int] { var expectedNumberOfElements: Option[Int] = None var count = 0 @@ -360,15 +359,15 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d case _ ⇒ Nil } } - }).produceTo(materializer, consumer) + }).produceTo(materializer, subscriber) - val subscription = consumer.expectSubscription() - subscription.requestMore(10) + val subscription = subscriber.expectSubscription() + subscription.request(10) - consumer.expectNext(1) - consumer.expectNext(2) - consumer.expectNext(3) - consumer.expectError.getMessage should be("Expected 5, got 3") + subscriber.expectNext(1) + subscriber.expectNext(2) + subscriber.expectNext(3) + subscriber.expectError.getMessage should be("Expected 5, got 3") } } diff --git a/akka-stream/src/test/scala/akka/stream/FlowZipSpec.scala b/akka-stream/src/test/scala/akka/stream/FlowZipSpec.scala index b6dcbde3ae..0a0d478f31 100644 --- a/akka-stream/src/test/scala/akka/stream/FlowZipSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/FlowZipSpec.scala @@ -3,37 +3,34 @@ */ package akka.stream -import akka.stream.testkit.StreamTestKit -import akka.stream.testkit.AkkaSpec import akka.stream.scaladsl.Flow -import org.reactivestreams.api.Producer -import akka.stream.testkit.OnSubscribe -import akka.stream.testkit.OnError +import akka.stream.testkit.StreamTestKit +import org.reactivestreams.Publisher class FlowZipSpec extends TwoStreamsSetup { type Outputs = (Int, Int) - override def operationUnderTest(in1: Flow[Int], in2: Producer[Int]) = in1.zip(in2) + override def operationUnderTest(in1: Flow[Int], in2: Publisher[Int]) = in1.zip(in2) "Zip" must { "work in the happy case" in { // Different input sizes (4 and 6) - val source1 = Flow((1 to 4).iterator).toProducer(materializer) - val source2 = Flow(List("A", "B", "C", "D", "E", "F").iterator).toProducer(materializer) - val p = Flow(source1).zip(source2).toProducer(materializer) + val source1 = Flow((1 to 4).iterator).toPublisher(materializer) + val source2 = Flow(List("A", "B", "C", "D", "E", "F").iterator).toPublisher(materializer) + val p = Flow(source1).zip(source2).toPublisher(materializer) - val probe = StreamTestKit.consumerProbe[(Int, String)] - p.produceTo(probe) + val probe = StreamTestKit.SubscriberProbe[(Int, String)]() + p.subscribe(probe) val subscription = probe.expectSubscription() - subscription.requestMore(2) + subscription.request(2) probe.expectNext((1, "A")) probe.expectNext((2, "B")) - subscription.requestMore(1) + subscription.request(1) probe.expectNext((3, "C")) - subscription.requestMore(1) + subscription.request(1) probe.expectNext((4, "D")) probe.expectComplete() @@ -41,36 +38,36 @@ class FlowZipSpec extends TwoStreamsSetup { commonTests() - "work with one immediately completed and one nonempty producer" in { - val consumer1 = setup(completedPublisher, nonemptyPublisher((1 to 4).iterator)) - consumer1.expectCompletedOrSubscriptionFollowedByComplete() + "work with one immediately completed and one nonempty publisher" in { + val subscriber1 = setup(completedPublisher, nonemptyPublisher((1 to 4).iterator)) + subscriber1.expectCompletedOrSubscriptionFollowedByComplete() - val consumer2 = setup(nonemptyPublisher((1 to 4).iterator), completedPublisher) - consumer2.expectCompletedOrSubscriptionFollowedByComplete() + val subscriber2 = setup(nonemptyPublisher((1 to 4).iterator), completedPublisher) + subscriber2.expectCompletedOrSubscriptionFollowedByComplete() } - "work with one delayed completed and one nonempty producer" in { - val consumer1 = setup(soonToCompletePublisher, nonemptyPublisher((1 to 4).iterator)) - consumer1.expectCompletedOrSubscriptionFollowedByComplete() + "work with one delayed completed and one nonempty publisher" in { + val subscriber1 = setup(soonToCompletePublisher, nonemptyPublisher((1 to 4).iterator)) + subscriber1.expectCompletedOrSubscriptionFollowedByComplete() - val consumer2 = setup(nonemptyPublisher((1 to 4).iterator), soonToCompletePublisher) - consumer2.expectCompletedOrSubscriptionFollowedByComplete() + val subscriber2 = setup(nonemptyPublisher((1 to 4).iterator), soonToCompletePublisher) + subscriber2.expectCompletedOrSubscriptionFollowedByComplete() } - "work with one immediately failed and one nonempty producer" in { - val consumer1 = setup(failedPublisher, nonemptyPublisher((1 to 4).iterator)) - consumer1.expectErrorOrSubscriptionFollowedByError(TestException) + "work with one immediately failed and one nonempty publisher" in { + val subscriber1 = setup(failedPublisher, nonemptyPublisher((1 to 4).iterator)) + subscriber1.expectErrorOrSubscriptionFollowedByError(TestException) - val consumer2 = setup(nonemptyPublisher((1 to 4).iterator), failedPublisher) - consumer2.expectErrorOrSubscriptionFollowedByError(TestException) + val subscriber2 = setup(nonemptyPublisher((1 to 4).iterator), failedPublisher) + subscriber2.expectErrorOrSubscriptionFollowedByError(TestException) } - "work with one delayed failed and one nonempty producer" in { - val consumer1 = setup(soonToFailPublisher, nonemptyPublisher((1 to 4).iterator)) - consumer1.expectErrorOrSubscriptionFollowedByError(TestException) + "work with one delayed failed and one nonempty publisher" in { + val subscriber1 = setup(soonToFailPublisher, nonemptyPublisher((1 to 4).iterator)) + subscriber1.expectErrorOrSubscriptionFollowedByError(TestException) - val consumer2 = setup(nonemptyPublisher((1 to 4).iterator), soonToFailPublisher) - val subscription2 = consumer2.expectErrorOrSubscriptionFollowedByError(TestException) + val subscriber2 = setup(nonemptyPublisher((1 to 4).iterator), soonToFailPublisher) + val subscription2 = subscriber2.expectErrorOrSubscriptionFollowedByError(TestException) } } diff --git a/akka-stream/src/test/scala/akka/stream/IdentityProcessorTest.scala b/akka-stream/src/test/scala/akka/stream/IdentityProcessorTest.scala index b4847c0932..a4daef3514 100644 --- a/akka-stream/src/test/scala/akka/stream/IdentityProcessorTest.scala +++ b/akka-stream/src/test/scala/akka/stream/IdentityProcessorTest.scala @@ -4,9 +4,8 @@ package akka.stream import org.scalatest.testng.TestNGSuiteLike -import org.reactivestreams.spi.Publisher -import org.reactivestreams.api.Processor -import org.reactivestreams.tck.{ PublisherVerification, TestEnvironment, IdentityProcessorVerification } +// FIXME: new TCK needed +//import org.reactivestreams.tck.{ PublisherVerification, TestEnvironment, IdentityProcessorVerification } import akka.actor.{ ActorSystem, Props } import akka.stream.impl.ActorProcessor import akka.stream.impl.TransformProcessorImpl @@ -17,53 +16,8 @@ import akka.stream.scaladsl.Flow import akka.stream.testkit.AkkaSpec import java.util.concurrent.atomic.AtomicInteger -class IdentityProcessorTest(_system: ActorSystem, env: TestEnvironment, publisherShutdownTimeout: Long) - extends IdentityProcessorVerification[Int](env, publisherShutdownTimeout) - with WithActorSystem with TestNGSuiteLike { +class IdentityProcessorTest(_system: ActorSystem, /*env: TestEnvironment,*/ publisherShutdownTimeout: Long) { - implicit val system = _system - import system.dispatcher - - def this(system: ActorSystem) { - this(system, new TestEnvironment(Timeouts.defaultTimeoutMillis(system)), Timeouts.publisherShutdownTimeoutMillis) - } - - def this() { - this(ActorSystem(classOf[IdentityProcessorTest].getSimpleName, AkkaSpec.testConf)) - } - - system.eventStream.publish(TestEvent.Mute(EventFilter[RuntimeException]("Test exception"))) - val processorCounter = new AtomicInteger - - def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = { - val fanoutSize = maxBufferSize / 2 - val inputSize = maxBufferSize - fanoutSize - - val materializer = new ActorBasedFlowMaterializer( - MaterializerSettings( - initialInputBufferSize = inputSize, - maximumInputBufferSize = inputSize, - initialFanOutBufferSize = fanoutSize, - maxFanOutBufferSize = fanoutSize, - dispatcher = "akka.test.stream-dispatcher"), - system, system.name) - - val processor = materializer.processorForNode(Ast.Transform( - new Transformer[Any, Any] { - override def onNext(in: Any) = List(in) - }), "IdentityProcessorTest-" + processorCounter.incrementAndGet(), 1) - - processor.asInstanceOf[Processor[Int, Int]] - } - - def createHelperPublisher(elements: Int): Publisher[Int] = { - val materializer = FlowMaterializer(MaterializerSettings( - maximumInputBufferSize = 512, dispatcher = "akka.test.stream-dispatcher"))(system) - val iter = Iterator from 1000 - Flow(if (elements > 0) iter take elements else iter).toProducer(materializer).getPublisher - } - - override def createErrorStatePublisher(): Publisher[Int] = null // ignore error-state tests - - override def createCompletedStatePublisher(): Publisher[Int] = null // ignore completed-state tests + // FIXME: new TCK needed + // Original code available in 82734877d080577cf538c2a47d60c117e078ac1c } diff --git a/akka-stream/src/test/scala/akka/stream/IterableProducerTest.scala b/akka-stream/src/test/scala/akka/stream/IterableProducerTest.scala index cbd9fad7c9..b0f777eb49 100644 --- a/akka-stream/src/test/scala/akka/stream/IterableProducerTest.scala +++ b/akka-stream/src/test/scala/akka/stream/IterableProducerTest.scala @@ -4,41 +4,14 @@ package akka.stream import org.scalatest.testng.TestNGSuiteLike -import org.reactivestreams.spi.Publisher -import org.reactivestreams.tck.{ TestEnvironment, PublisherVerification } +// FIXME: Needs new TCK version +//import org.reactivestreams.tck.{ TestEnvironment, PublisherVerification } import scala.collection.immutable import akka.stream.scaladsl.Flow import akka.actor.ActorSystem import akka.stream.testkit.AkkaSpec -class IterableProducerTest(_system: ActorSystem, env: TestEnvironment, publisherShutdownTimeout: Long) - extends PublisherVerification[Int](env, publisherShutdownTimeout) - with WithActorSystem with TestNGSuiteLike { - - implicit val system = _system - - def this(system: ActorSystem) { - this(system, new TestEnvironment(Timeouts.defaultTimeoutMillis(system)), Timeouts.publisherShutdownTimeoutMillis) - } - - def this() { - this(ActorSystem(classOf[IterableProducerTest].getSimpleName, AkkaSpec.testConf)) - } - - val materializer = FlowMaterializer(MaterializerSettings( - maximumInputBufferSize = 512, dispatcher = "akka.test.stream-dispatcher"))(system) - - def createPublisher(elements: Int): Publisher[Int] = { - val iterable: immutable.Iterable[Int] = - if (elements == 0) - new immutable.Iterable[Int] { override def iterator = Iterator from 0 } - else - 0 until elements - Flow(iterable).toProducer(materializer).getPublisher - } - - override def createCompletedStatePublisher(): Publisher[Int] = - Flow[Int](Nil).toProducer(materializer).getPublisher - - override def createErrorStatePublisher(): Publisher[Int] = null // ignore error-state tests +class IterableProducerTest(_system: ActorSystem, /*env: TestEnvironment, */ publisherShutdownTimeout: Long) { + // FIXME: Needs new TCK version + // Original code available in 82734877d080577cf538c2a47d60c117e078ac1c } \ No newline at end of file diff --git a/akka-stream/src/test/scala/akka/stream/IteratorProducerTest.scala b/akka-stream/src/test/scala/akka/stream/IteratorProducerTest.scala index d5dba584f2..a4a50d4717 100644 --- a/akka-stream/src/test/scala/akka/stream/IteratorProducerTest.scala +++ b/akka-stream/src/test/scala/akka/stream/IteratorProducerTest.scala @@ -4,40 +4,13 @@ package akka.stream import org.scalatest.testng.TestNGSuiteLike -import org.reactivestreams.spi.Publisher -import org.reactivestreams.tck.{ TestEnvironment, PublisherVerification } +// FIXME: Needs new TCK version +// import org.reactivestreams.tck.{ TestEnvironment, PublisherVerification } import akka.stream.scaladsl.Flow import akka.actor.ActorSystem import akka.stream.testkit.AkkaSpec -class IteratorProducerTest(_system: ActorSystem, env: TestEnvironment, publisherShutdownTimeout: Long) - extends PublisherVerification[Int](env, publisherShutdownTimeout) - with WithActorSystem with TestNGSuiteLike { - - implicit val system = _system - - def this(system: ActorSystem) { - this(system, new TestEnvironment(Timeouts.defaultTimeoutMillis(system)), Timeouts.publisherShutdownTimeoutMillis) - } - - def this() { - this(ActorSystem(classOf[IteratorProducerTest].getSimpleName, AkkaSpec.testConf)) - } - - val materializer = FlowMaterializer(MaterializerSettings( - maximumInputBufferSize = 512, dispatcher = "akka.test.stream-dispatcher"))(system) - - def createPublisher(elements: Int): Publisher[Int] = { - val iter: Iterator[Int] = - if (elements == 0) - Iterator from 0 - else - (Iterator from 0).take(elements) - Flow(iter).toProducer(materializer).getPublisher - } - - override def createCompletedStatePublisher(): Publisher[Int] = - Flow(List.empty[Int].iterator).toProducer(materializer).getPublisher - - override def createErrorStatePublisher(): Publisher[Int] = null // ignore error-state tests +class IteratorProducerTest(_system: ActorSystem, /*env: TestEnvironment, */ publisherShutdownTimeout: Long) { + // FIXME: Needs new TCK version + // Original code available in 82734877d080577cf538c2a47d60c117e078ac1c } \ No newline at end of file diff --git a/akka-stream/src/test/scala/akka/stream/ProcessorHierarchySpec.scala b/akka-stream/src/test/scala/akka/stream/ProcessorHierarchySpec.scala index 0f8a9de9b2..3b6c9e74bf 100644 --- a/akka-stream/src/test/scala/akka/stream/ProcessorHierarchySpec.scala +++ b/akka-stream/src/test/scala/akka/stream/ProcessorHierarchySpec.scala @@ -24,7 +24,7 @@ class ProcessorHierarchySpec extends AkkaSpec("akka.actor.debug.lifecycle=off\na "generate the right level of descendants" in { val f = Flow(() ⇒ { testActor ! self - Flow(List(1)).map(x ⇒ { testActor ! self; x }).toProducer(materializer) + Flow(List(1)).map(x ⇒ { testActor ! self; x }).toPublisher(materializer) }).take(3).foreach(x ⇒ { testActor ! self Flow(x).foreach(_ ⇒ testActor ! self).consume(materializer) diff --git a/akka-stream/src/test/scala/akka/stream/TickProducerSpec.scala b/akka-stream/src/test/scala/akka/stream/TickPublisherSpec.scala similarity index 75% rename from akka-stream/src/test/scala/akka/stream/TickProducerSpec.scala rename to akka-stream/src/test/scala/akka/stream/TickPublisherSpec.scala index a61a3280e3..a9255f4006 100644 --- a/akka-stream/src/test/scala/akka/stream/TickProducerSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/TickPublisherSpec.scala @@ -10,18 +10,18 @@ import akka.stream.testkit.StreamTestKit import scala.util.control.NoStackTrace @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class TickProducerSpec extends AkkaSpec { +class TickPublisherSpec extends AkkaSpec { val materializer = FlowMaterializer(MaterializerSettings( dispatcher = "akka.test.stream-dispatcher")) - "A Flow based on tick producer" must { + "A Flow based on tick publisher" must { "produce ticks" in { val tickGen = Iterator from 1 - val c = StreamTestKit.consumerProbe[String] + val c = StreamTestKit.SubscriberProbe[String]() Flow(1.second, () ⇒ "tick-" + tickGen.next()).produceTo(materializer, c) val sub = c.expectSubscription() - sub.requestMore(3) + sub.request(3) c.expectNext("tick-1") c.expectNoMsg(200.millis) c.expectNext("tick-2") @@ -33,15 +33,15 @@ class TickProducerSpec extends AkkaSpec { "drop ticks when not requested" in { val tickGen = Iterator from 1 - val c = StreamTestKit.consumerProbe[String] + val c = StreamTestKit.SubscriberProbe[String]() Flow(1.second, () ⇒ "tick-" + tickGen.next()).produceTo(materializer, c) val sub = c.expectSubscription() - sub.requestMore(2) + sub.request(2) c.expectNext("tick-1") c.expectNoMsg(200.millis) c.expectNext("tick-2") c.expectNoMsg(1400.millis) - sub.requestMore(2) + sub.request(2) c.expectNext("tick-4") c.expectNoMsg(200.millis) c.expectNext("tick-5") @@ -51,22 +51,22 @@ class TickProducerSpec extends AkkaSpec { "produce ticks with multiple subscribers" in { val tickGen = Iterator from 1 - val p = Flow(1.second, () ⇒ "tick-" + tickGen.next()).toProducer(materializer) - val c1 = StreamTestKit.consumerProbe[String] - val c2 = StreamTestKit.consumerProbe[String] - p.produceTo(c1) - p.produceTo(c2) + val p = Flow(1.second, () ⇒ "tick-" + tickGen.next()).toPublisher(materializer) + val c1 = StreamTestKit.SubscriberProbe[String]() + val c2 = StreamTestKit.SubscriberProbe[String]() + p.subscribe(c1) + p.subscribe(c2) val sub1 = c1.expectSubscription() val sub2 = c2.expectSubscription() - sub1.requestMore(1) - sub2.requestMore(2) + sub1.request(1) + sub2.request(2) c1.expectNext("tick-1") c2.expectNext("tick-1") c2.expectNoMsg(200.millis) c2.expectNext("tick-2") c1.expectNoMsg(200.millis) - sub1.requestMore(2) - sub2.requestMore(2) + sub1.request(2) + sub2.request(2) c1.expectNext("tick-3") c2.expectNext("tick-3") sub1.cancel() @@ -74,20 +74,19 @@ class TickProducerSpec extends AkkaSpec { } "signal onError when tick closure throws" in { - val tickGen = Iterator from 1 - val c = StreamTestKit.consumerProbe[String] + val c = StreamTestKit.SubscriberProbe[String]() Flow(1.second, () ⇒ throw new RuntimeException("tick err") with NoStackTrace).produceTo(materializer, c) val sub = c.expectSubscription() - sub.requestMore(3) + sub.request(3) c.expectError.getMessage should be("tick err") } "be usable with zip for a simple form of rate limiting" in { - val c = StreamTestKit.consumerProbe[Int] - val rate = Flow(1.second, () ⇒ "tick").toProducer(materializer) + val c = StreamTestKit.SubscriberProbe[Int]() + val rate = Flow(1.second, () ⇒ "tick").toPublisher(materializer) Flow(1 to 100).zip(rate).map { case (n, _) ⇒ n }.produceTo(materializer, c) val sub = c.expectSubscription() - sub.requestMore(1000) + sub.request(1000) c.expectNext(1) c.expectNoMsg(200.millis) c.expectNext(2) diff --git a/akka-stream/src/test/scala/akka/stream/TwoStreamsSetup.scala b/akka-stream/src/test/scala/akka/stream/TwoStreamsSetup.scala index 941a7b2ad0..e454d4775c 100644 --- a/akka-stream/src/test/scala/akka/stream/TwoStreamsSetup.scala +++ b/akka-stream/src/test/scala/akka/stream/TwoStreamsSetup.scala @@ -3,13 +3,11 @@ */ package akka.stream -import scala.util.control.NoStackTrace -import org.reactivestreams.api.{ Consumer, Producer } -import org.reactivestreams.spi.{ Subscriber, Publisher, Subscription } -import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.scaladsl.Flow -import akka.stream.testkit.OnSubscribe -import akka.stream.testkit.OnError +import akka.stream.testkit.{ AkkaSpec, StreamTestKit } +import org.reactivestreams.Publisher + +import scala.util.control.NoStackTrace abstract class TwoStreamsSetup extends AkkaSpec { @@ -26,66 +24,60 @@ abstract class TwoStreamsSetup extends AkkaSpec { type Outputs - def operationUnderTest(in1: Flow[Int], in2: Producer[Int]): Flow[Outputs] + def operationUnderTest(in1: Flow[Int], in2: Publisher[Int]): Flow[Outputs] def setup(p1: Publisher[Int], p2: Publisher[Int]) = { - val consumer = StreamTestKit.consumerProbe[Outputs] - operationUnderTest(Flow(producerFromPublisher(p1)), producerFromPublisher(p2)).toProducer(materializer).produceTo(consumer) - consumer + val subscriber = StreamTestKit.SubscriberProbe[Outputs]() + operationUnderTest(Flow(p1), p2).toPublisher(materializer).subscribe(subscriber) + subscriber } - def producerFromPublisher[T](publisher: Publisher[T]): Producer[T] = new Producer[T] { - private val pub = publisher - override def produceTo(consumer: Consumer[T]): Unit = pub.subscribe(consumer.getSubscriber) - override def getPublisher: Publisher[T] = pub - } + def failedPublisher[T]: Publisher[T] = StreamTestKit.errorPublisher[T](TestException) - def failedPublisher[T]: Publisher[T] = StreamTestKit.errorProducer[T](TestException).getPublisher + def completedPublisher[T]: Publisher[T] = StreamTestKit.emptyPublisher[T] - def completedPublisher[T]: Publisher[T] = StreamTestKit.emptyProducer[T].getPublisher + def nonemptyPublisher[T](elems: Iterator[T]): Publisher[T] = Flow(elems).toPublisher(materializer) - def nonemptyPublisher[T](elems: Iterator[T]): Publisher[T] = Flow(elems).toProducer(materializer).getPublisher + def soonToFailPublisher[T]: Publisher[T] = StreamTestKit.lazyErrorPublisher[T](TestException) - def soonToFailPublisher[T]: Publisher[T] = StreamTestKit.lazyErrorProducer[T](TestException).getPublisher - - def soonToCompletePublisher[T]: Publisher[T] = StreamTestKit.lazyEmptyProducer[T].getPublisher + def soonToCompletePublisher[T]: Publisher[T] = StreamTestKit.lazyEmptyPublisher[T] def commonTests() = { - "work with two immediately completed producers" in { - val consumer = setup(completedPublisher, completedPublisher) - consumer.expectCompletedOrSubscriptionFollowedByComplete() + "work with two immediately completed publishers" in { + val subscriber = setup(completedPublisher, completedPublisher) + subscriber.expectCompletedOrSubscriptionFollowedByComplete() } - "work with two delayed completed producers" in { - val consumer = setup(soonToCompletePublisher, soonToCompletePublisher) - consumer.expectCompletedOrSubscriptionFollowedByComplete() + "work with two delayed completed publishers" in { + val subscriber = setup(soonToCompletePublisher, soonToCompletePublisher) + subscriber.expectCompletedOrSubscriptionFollowedByComplete() } - "work with one immediately completed and one delayed completed producer" in { - val consumer = setup(completedPublisher, soonToCompletePublisher) - consumer.expectCompletedOrSubscriptionFollowedByComplete() + "work with one immediately completed and one delayed completed publisher" in { + val subscriber = setup(completedPublisher, soonToCompletePublisher) + subscriber.expectCompletedOrSubscriptionFollowedByComplete() } - "work with two immediately failed producers" in { - val consumer = setup(failedPublisher, failedPublisher) - consumer.expectErrorOrSubscriptionFollowedByError(TestException) + "work with two immediately failed publishers" in { + val subscriber = setup(failedPublisher, failedPublisher) + subscriber.expectErrorOrSubscriptionFollowedByError(TestException) } - "work with two delayed failed producers" in { - val consumer = setup(soonToFailPublisher, soonToFailPublisher) - consumer.expectErrorOrSubscriptionFollowedByError(TestException) + "work with two delayed failed publishers" in { + val subscriber = setup(soonToFailPublisher, soonToFailPublisher) + subscriber.expectErrorOrSubscriptionFollowedByError(TestException) } // Warning: The two test cases below are somewhat implementation specific and might fail if the implementation // is changed. They are here to be an early warning though. - "work with one immediately failed and one delayed failed producer (case 1)" in { - val consumer = setup(soonToFailPublisher, failedPublisher) - consumer.expectErrorOrSubscriptionFollowedByError(TestException) + "work with one immediately failed and one delayed failed publisher (case 1)" in { + val subscriber = setup(soonToFailPublisher, failedPublisher) + subscriber.expectErrorOrSubscriptionFollowedByError(TestException) } - "work with one immediately failed and one delayed failed producer (case 2)" in { - val consumer = setup(failedPublisher, soonToFailPublisher) - consumer.expectErrorOrSubscriptionFollowedByError(TestException) + "work with one immediately failed and one delayed failed publisher (case 2)" in { + val subscriber = setup(failedPublisher, soonToFailPublisher) + subscriber.expectErrorOrSubscriptionFollowedByError(TestException) } } diff --git a/akka-stream/src/test/scala/akka/stream/WithActorSystem.scala b/akka-stream/src/test/scala/akka/stream/WithActorSystem.scala index f3282f295e..eb94bbfb86 100644 --- a/akka-stream/src/test/scala/akka/stream/WithActorSystem.scala +++ b/akka-stream/src/test/scala/akka/stream/WithActorSystem.scala @@ -4,11 +4,13 @@ package akka.stream import akka.actor.ActorSystem -import org.testng.annotations.AfterClass +// FIXME: TestNG dependency comes from TCK. Needs new TCK version +//import org.testng.annotations.AfterClass trait WithActorSystem { def system: ActorSystem - @AfterClass - def shutdownActorSystem(): Unit = system.shutdown() + // FIXME: TestNG dependency comes from TCK. Needs new TCK version + // @AfterClass + // def shutdownActorSystem(): Unit = system.shutdown() } \ No newline at end of file diff --git a/akka-stream/src/test/scala/akka/stream/actor/ActorProducerSpec.scala b/akka-stream/src/test/scala/akka/stream/actor/ActorPublisherSpec.scala similarity index 62% rename from akka-stream/src/test/scala/akka/stream/actor/ActorProducerSpec.scala rename to akka-stream/src/test/scala/akka/stream/actor/ActorPublisherSpec.scala index a7bda22b6a..da3bfac65c 100644 --- a/akka-stream/src/test/scala/akka/stream/actor/ActorProducerSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/actor/ActorPublisherSpec.scala @@ -10,7 +10,7 @@ import akka.actor.PoisonPill import akka.actor.Props import akka.stream.FlowMaterializer import akka.stream.MaterializerSettings -import akka.stream.actor.ActorConsumer.WatermarkRequestStrategy +import akka.stream.actor.ActorSubscriber.WatermarkRequestStrategy import akka.stream.scaladsl.Flow import akka.stream.testkit.AkkaSpec import akka.stream.testkit.StreamTestKit @@ -19,10 +19,10 @@ import akka.testkit.ImplicitSender import akka.testkit.TestEvent.Mute import akka.testkit.TestProbe -object ActorProducerSpec { +object ActorPublisherSpec { - def testProducerProps(probe: ActorRef): Props = - Props(new TestProducer(probe)).withDispatcher("akka.test.stream-dispatcher") + def testPublisherProps(probe: ActorRef): Props = + Props(new TestPublisher(probe)).withDispatcher("akka.test.stream-dispatcher") case class TotalDemand(elements: Long) case class Produce(elem: String) @@ -30,8 +30,8 @@ object ActorProducerSpec { case object Boom case object Complete - class TestProducer(probe: ActorRef) extends ActorProducer[String] { - import ActorProducer._ + class TestPublisher(probe: ActorRef) extends ActorPublisher[String] { + import ActorPublisher._ def receive = { case Request(element) ⇒ probe ! TotalDemand(totalDemand) @@ -44,9 +44,9 @@ object ActorProducerSpec { def senderProps: Props = Props[Sender].withDispatcher("akka.test.stream-dispatcher") - class Sender extends ActorProducer[Int] { - import ActorProducer.Cancel - import ActorProducer.Request + class Sender extends ActorPublisher[Int] { + import ActorPublisher.Cancel + import ActorPublisher.Request var buf = Vector.empty[Int] @@ -75,8 +75,8 @@ object ActorProducerSpec { def receiverProps(probe: ActorRef): Props = Props(new Receiver(probe)).withDispatcher("akka.test.stream-dispatcher") - class Receiver(probe: ActorRef) extends ActorConsumer { - import ActorConsumer._ + class Receiver(probe: ActorRef) extends ActorSubscriber { + import ActorSubscriber._ override val requestStrategy = WatermarkRequestStrategy(10) @@ -89,36 +89,36 @@ object ActorProducerSpec { } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class ActorProducerSpec extends AkkaSpec with ImplicitSender { - import ActorProducerSpec._ - import ActorProducer._ +class ActorPublisherSpec extends AkkaSpec with ImplicitSender { + import ActorPublisherSpec._ + import ActorPublisher._ system.eventStream.publish(Mute(EventFilter[IllegalStateException]())) - "An ActorProducer" must { + "An ActorPublisher" must { "accumulate demand" in { val probe = TestProbe() - val ref = system.actorOf(testProducerProps(probe.ref)) - val p = ActorProducer[String](ref) - val c = StreamTestKit.consumerProbe[String] - p.produceTo(c) + val ref = system.actorOf(testPublisherProps(probe.ref)) + val p = ActorPublisher[String](ref) + val c = StreamTestKit.SubscriberProbe[String]() + p.subscribe(c) val sub = c.expectSubscription - sub.requestMore(2) + sub.request(2) probe.expectMsg(TotalDemand(2)) - sub.requestMore(3) + sub.request(3) probe.expectMsg(TotalDemand(5)) sub.cancel() } "allow onNext up to requested elements, but not more" in { val probe = TestProbe() - val ref = system.actorOf(testProducerProps(probe.ref)) - val p = ActorProducer[String](ref) - val c = StreamTestKit.consumerProbe[String] - p.produceTo(c) + val ref = system.actorOf(testPublisherProps(probe.ref)) + val p = ActorPublisher[String](ref) + val c = StreamTestKit.SubscriberProbe[String]() + p.subscribe(c) val sub = c.expectSubscription - sub.requestMore(2) + sub.request(2) ref ! Produce("elem-1") ref ! Produce("elem-2") ref ! Produce("elem-3") @@ -130,9 +130,9 @@ class ActorProducerSpec extends AkkaSpec with ImplicitSender { "signal error" in { val probe = TestProbe() - val ref = system.actorOf(testProducerProps(probe.ref)) - val c = StreamTestKit.consumerProbe[String] - ActorProducer[String](ref).produceTo(c) + val ref = system.actorOf(testPublisherProps(probe.ref)) + val c = StreamTestKit.SubscriberProbe[String]() + ActorPublisher[String](ref).subscribe(c) ref ! Err("wrong") c.expectSubscription c.expectError.getMessage should be("wrong") @@ -140,21 +140,21 @@ class ActorProducerSpec extends AkkaSpec with ImplicitSender { "signal error before subscribe" in { val probe = TestProbe() - val ref = system.actorOf(testProducerProps(probe.ref)) + val ref = system.actorOf(testPublisherProps(probe.ref)) ref ! Err("early err") - val c = StreamTestKit.consumerProbe[String] - ActorProducer[String](ref).produceTo(c) + val c = StreamTestKit.SubscriberProbe[String]() + ActorPublisher[String](ref).subscribe(c) c.expectError.getMessage should be("early err") } "drop onNext elements after cancel" in { val probe = TestProbe() - val ref = system.actorOf(testProducerProps(probe.ref)) - val p = ActorProducer[String](ref) - val c = StreamTestKit.consumerProbe[String] - p.produceTo(c) + val ref = system.actorOf(testPublisherProps(probe.ref)) + val p = ActorPublisher[String](ref) + val c = StreamTestKit.SubscriberProbe[String]() + p.subscribe(c) val sub = c.expectSubscription - sub.requestMore(2) + sub.request(2) ref ! Produce("elem-1") sub.cancel() ref ! Produce("elem-2") @@ -165,30 +165,30 @@ class ActorProducerSpec extends AkkaSpec with ImplicitSender { "remember requested after restart" in { val probe = TestProbe() - val ref = system.actorOf(testProducerProps(probe.ref)) - val p = ActorProducer[String](ref) - val c = StreamTestKit.consumerProbe[String] - p.produceTo(c) + val ref = system.actorOf(testPublisherProps(probe.ref)) + val p = ActorPublisher[String](ref) + val c = StreamTestKit.SubscriberProbe[String]() + p.subscribe(c) val sub = c.expectSubscription - sub.requestMore(3) + sub.request(3) probe.expectMsg(TotalDemand(3)) ref ! Produce("elem-1") ref ! Boom ref ! Produce("elem-2") c.expectNext("elem-1") c.expectNext("elem-2") - sub.requestMore(5) + sub.request(5) probe.expectMsg(TotalDemand(6)) sub.cancel() } "signal onComplete" in { val probe = TestProbe() - val ref = system.actorOf(testProducerProps(probe.ref)) - val c = StreamTestKit.consumerProbe[String] - ActorProducer[String](ref).produceTo(c) + val ref = system.actorOf(testPublisherProps(probe.ref)) + val c = StreamTestKit.SubscriberProbe[String]() + ActorPublisher[String](ref).subscribe(c) val sub = c.expectSubscription - sub.requestMore(3) + sub.request(3) ref ! Produce("elem-1") ref ! Complete c.expectNext("elem-1") @@ -197,42 +197,42 @@ class ActorProducerSpec extends AkkaSpec with ImplicitSender { "signal immediate onComplete" in { val probe = TestProbe() - val ref = system.actorOf(testProducerProps(probe.ref)) + val ref = system.actorOf(testPublisherProps(probe.ref)) ref ! Complete - val c = StreamTestKit.consumerProbe[String] - ActorProducer[String](ref).produceTo(c) + val c = StreamTestKit.SubscriberProbe[String]() + ActorPublisher[String](ref).subscribe(c) c.expectComplete } "only allow one subscriber" in { val probe = TestProbe() - val ref = system.actorOf(testProducerProps(probe.ref)) - val c = StreamTestKit.consumerProbe[String] - ActorProducer[String](ref).produceTo(c) - val sub = c.expectSubscription - val c2 = StreamTestKit.consumerProbe[String] - ActorProducer[String](ref).produceTo(c2) + val ref = system.actorOf(testPublisherProps(probe.ref)) + val c = StreamTestKit.SubscriberProbe[String]() + ActorPublisher[String](ref).subscribe(c) + c.expectSubscription + val c2 = StreamTestKit.SubscriberProbe[String]() + ActorPublisher[String](ref).subscribe(c2) c2.expectError.getClass should be(classOf[IllegalStateException]) } "signal onCompete when actor is stopped" in { val probe = TestProbe() - val ref = system.actorOf(testProducerProps(probe.ref)) - val c = StreamTestKit.consumerProbe[String] - ActorProducer[String](ref).produceTo(c) - val sub = c.expectSubscription + val ref = system.actorOf(testPublisherProps(probe.ref)) + val c = StreamTestKit.SubscriberProbe[String]() + ActorPublisher[String](ref).subscribe(c) + c.expectSubscription ref ! PoisonPill c.expectComplete } - "work together with Flow and ActorConsumer" in { + "work together with Flow and ActorSubscriber" in { val materializer = FlowMaterializer(MaterializerSettings(dispatcher = "akka.test.stream-dispatcher")) val probe = TestProbe() val snd = system.actorOf(senderProps) val rcv = system.actorOf(receiverProps(probe.ref)) - Flow(ActorProducer[Int](snd)).collect { + Flow(ActorPublisher[Int](snd)).collect { case n if n % 2 == 0 ⇒ "elem-" + n - }.produceTo(materializer, ActorConsumer(rcv)) + }.produceTo(materializer, ActorSubscriber(rcv)) (1 to 3) foreach { snd ! _ } probe.expectMsg("elem-2") diff --git a/akka-stream/src/test/scala/akka/stream/actor/ActorConsumerSpec.scala b/akka-stream/src/test/scala/akka/stream/actor/ActorSubscriberSpec.scala similarity index 79% rename from akka-stream/src/test/scala/akka/stream/actor/ActorConsumerSpec.scala rename to akka-stream/src/test/scala/akka/stream/actor/ActorSubscriberSpec.scala index 50db444ec0..602f0e4914 100644 --- a/akka-stream/src/test/scala/akka/stream/actor/ActorConsumerSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/actor/ActorSubscriberSpec.scala @@ -10,7 +10,7 @@ import akka.stream.FlowMaterializer import akka.stream.MaterializerSettings import akka.stream.scaladsl.Flow import akka.stream.testkit.AkkaSpec -import akka.stream.actor.ActorConsumer.RequestStrategy +import akka.stream.actor.ActorSubscriber.RequestStrategy import akka.actor.Actor import akka.routing.ActorRefRoutee import akka.routing.Router @@ -18,13 +18,13 @@ import akka.routing.RoundRobinRoutingLogic import akka.testkit.ImplicitSender import scala.util.control.NoStackTrace -object ActorConsumerSpec { +object ActorSubscriberSpec { - def manualConsumerProps(probe: ActorRef): Props = - Props(new ManualConsumer(probe)).withDispatcher("akka.test.stream-dispatcher") + def manualSubscriberProps(probe: ActorRef): Props = + Props(new ManualSubscriber(probe)).withDispatcher("akka.test.stream-dispatcher") - class ManualConsumer(probe: ActorRef) extends ActorConsumer { - import ActorConsumer._ + class ManualSubscriber(probe: ActorRef) extends ActorSubscriber { + import ActorSubscriber._ override val requestStrategy = ZeroRequestStrategy @@ -38,11 +38,11 @@ object ActorConsumerSpec { } } - def requestStrategyConsumerProps(probe: ActorRef, strat: RequestStrategy): Props = - Props(new RequestStrategyConsumer(probe, strat)).withDispatcher("akka.test.stream-dispatcher") + def requestStrategySubscriberProps(probe: ActorRef, strat: RequestStrategy): Props = + Props(new RequestStrategySubscriber(probe, strat)).withDispatcher("akka.test.stream-dispatcher") - class RequestStrategyConsumer(probe: ActorRef, strat: RequestStrategy) extends ActorConsumer { - import ActorConsumer._ + class RequestStrategySubscriber(probe: ActorRef, strat: RequestStrategy) extends ActorSubscriber { + import ActorSubscriber._ override val requestStrategy = strat @@ -60,8 +60,8 @@ object ActorConsumerSpec { def streamerProps: Props = Props(new Streamer).withDispatcher("akka.test.stream-dispatcher") - class Streamer extends ActorConsumer { - import ActorConsumer._ + class Streamer extends ActorSubscriber { + import ActorSubscriber._ var queue = Map.empty[Int, ActorRef] val router = { @@ -96,17 +96,17 @@ object ActorConsumerSpec { } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class ActorConsumerSpec extends AkkaSpec with ImplicitSender { - import ActorConsumerSpec._ - import ActorConsumer._ +class ActorSubscriberSpec extends AkkaSpec with ImplicitSender { + import ActorSubscriberSpec._ + import ActorSubscriber._ val materializer = FlowMaterializer(MaterializerSettings(dispatcher = "akka.test.stream-dispatcher")) - "An ActorConsumer" must { + "An ActorSubscriber" must { "receive requested elements" in { - val ref = system.actorOf(manualConsumerProps(testActor)) - Flow(List(1, 2, 3)).produceTo(materializer, ActorConsumer(ref)) + val ref = system.actorOf(manualSubscriberProps(testActor)) + Flow(List(1, 2, 3)).produceTo(materializer, ActorSubscriber(ref)) expectNoMsg(200.millis) ref ! "ready" // requesting 2 expectMsg(OnNext(1)) @@ -118,16 +118,16 @@ class ActorConsumerSpec extends AkkaSpec with ImplicitSender { } "signal error" in { - val ref = system.actorOf(manualConsumerProps(testActor)) + val ref = system.actorOf(manualSubscriberProps(testActor)) val e = new RuntimeException("simulated") with NoStackTrace - Flow(() ⇒ throw e).produceTo(materializer, ActorConsumer(ref)) + Flow(() ⇒ throw e).produceTo(materializer, ActorSubscriber(ref)) ref ! "ready" expectMsg(OnError(e)) } "remember requested after restart" in { - val ref = system.actorOf(manualConsumerProps(testActor)) - Flow(1 to 7).produceTo(materializer, ActorConsumer(ref)) + val ref = system.actorOf(manualSubscriberProps(testActor)) + Flow(1 to 7).produceTo(materializer, ActorSubscriber(ref)) ref ! "ready" expectMsg(OnNext(1)) expectMsg(OnNext(2)) @@ -144,8 +144,8 @@ class ActorConsumerSpec extends AkkaSpec with ImplicitSender { } "not deliver more after cancel" in { - val ref = system.actorOf(manualConsumerProps(testActor)) - Flow(1 to 5).produceTo(materializer, ActorConsumer(ref)) + val ref = system.actorOf(manualSubscriberProps(testActor)) + Flow(1 to 5).produceTo(materializer, ActorSubscriber(ref)) ref ! "ready" expectMsg(OnNext(1)) expectMsg(OnNext(2)) @@ -154,15 +154,15 @@ class ActorConsumerSpec extends AkkaSpec with ImplicitSender { } "work with OneByOneRequestStrategy" in { - val ref = system.actorOf(requestStrategyConsumerProps(testActor, OneByOneRequestStrategy)) - Flow(1 to 17).produceTo(materializer, ActorConsumer(ref)) + val ref = system.actorOf(requestStrategySubscriberProps(testActor, OneByOneRequestStrategy)) + Flow(1 to 17).produceTo(materializer, ActorSubscriber(ref)) for (n ← 1 to 17) expectMsg(OnNext(n)) expectMsg(OnComplete) } "work with WatermarkRequestStrategy" in { - val ref = system.actorOf(requestStrategyConsumerProps(testActor, WatermarkRequestStrategy(highWatermark = 10))) - Flow(1 to 17).produceTo(materializer, ActorConsumer(ref)) + val ref = system.actorOf(requestStrategySubscriberProps(testActor, WatermarkRequestStrategy(highWatermark = 10))) + Flow(1 to 17).produceTo(materializer, ActorSubscriber(ref)) for (n ← 1 to 17) expectMsg(OnNext(n)) expectMsg(OnComplete) } @@ -170,7 +170,7 @@ class ActorConsumerSpec extends AkkaSpec with ImplicitSender { "suport custom max in flight request strategy with child workers" in { val ref = system.actorOf(streamerProps) val N = 117 - Flow(1 to N).map(Msg(_, testActor)).produceTo(materializer, ActorConsumer(ref)) + Flow(1 to N).map(Msg(_, testActor)).produceTo(materializer, ActorSubscriber(ref)) receiveN(N).toSet should be((1 to N).map(Done(_)).toSet) } diff --git a/akka-stream/src/test/scala/akka/stream/extra/FlowTimedSpec.scala b/akka-stream/src/test/scala/akka/stream/extra/FlowTimedSpec.scala index 3d6df4d730..1f1ce10216 100644 --- a/akka-stream/src/test/scala/akka/stream/extra/FlowTimedSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/extra/FlowTimedSpec.scala @@ -3,11 +3,11 @@ */ package akka.stream.extra -import akka.stream.testkit.{ AkkaConsumerProbe, StreamTestKit, ScriptedTest, AkkaSpec } +import akka.stream.testkit.{ StreamTestKit, ScriptedTest, AkkaSpec } import akka.stream.{ FlowMaterializer, MaterializerSettings } import akka.testkit.TestProbe import akka.stream.scaladsl.{ Flow, Duct } -import org.reactivestreams.api.{ Producer, Consumer } +import org.reactivestreams.{ Subscriber, Publisher } class FlowTimedSpec extends AkkaSpec with ScriptedTest { @@ -82,14 +82,14 @@ class FlowTimedSpec extends AkkaSpec with ScriptedTest { val duct: Duct[Int, Long] = Duct[Int].map(_.toLong).timedIntervalBetween(in ⇒ in % 2 == 1, d ⇒ probe.ref ! d) - val c1 = StreamTestKit.consumerProbe[Long]() - val c2: Consumer[Int] = duct.produceTo(materializer, c1) + val c1 = StreamTestKit.SubscriberProbe[Long]() + val c2: Subscriber[Int] = duct.produceTo(materializer, c1) - val p = Flow(List(1, 2, 3)).toProducer(materializer) - p.produceTo(c2) + val p = Flow(List(1, 2, 3)).toPublisher(materializer) + p.subscribe(c2) val s = c1.expectSubscription() - s.requestMore(100) + s.request(100) c1.expectNext(1L) c1.expectNext(2L) c1.expectNext(3L) @@ -111,16 +111,16 @@ class FlowTimedSpec extends AkkaSpec with ScriptedTest { map(_.toString), duration ⇒ probe.ref ! duration). map { s: String ⇒ s + "!" } - val (ductIn: Consumer[Int], ductOut: Producer[String]) = duct.build(materializer) + val (ductIn: Subscriber[Int], ductOut: Publisher[String]) = duct.build(materializer) - val c1 = StreamTestKit.consumerProbe[String]() - val c2 = ductOut.produceTo(c1) + val c1 = StreamTestKit.SubscriberProbe[String]() + val c2 = ductOut.subscribe(c1) - val p = Flow(0 to 100).toProducer(materializer) - p.produceTo(ductIn) + val p = Flow(0 to 100).toPublisher(materializer) + p.subscribe(ductIn) val s = c1.expectSubscription() - s.requestMore(200) + s.request(200) 0 to 100 foreach { i ⇒ c1.expectNext(i.toString + "!") } c1.expectComplete() diff --git a/akka-stream/src/test/scala/akka/stream/impl/SynchronousProducerFromIterableSpec.scala b/akka-stream/src/test/scala/akka/stream/impl/SynchronousPublisherFromIterableSpec.scala similarity index 59% rename from akka-stream/src/test/scala/akka/stream/impl/SynchronousProducerFromIterableSpec.scala rename to akka-stream/src/test/scala/akka/stream/impl/SynchronousPublisherFromIterableSpec.scala index 57a8a13b46..9232db7846 100644 --- a/akka-stream/src/test/scala/akka/stream/impl/SynchronousProducerFromIterableSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/impl/SynchronousPublisherFromIterableSpec.scala @@ -7,58 +7,56 @@ import scala.collection.immutable import scala.concurrent.duration._ import akka.stream.testkit.AkkaSpec import akka.stream.testkit.StreamTestKit -import org.reactivestreams.api.Consumer -import org.reactivestreams.spi.Subscriber import akka.testkit.TestProbe -import org.reactivestreams.spi.Subscription +import org.reactivestreams.{ Subscriber, Subscription } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class SynchronousProducerFromIterableSpec extends AkkaSpec { +class SynchronousPublisherFromIterableSpec extends AkkaSpec { - "A SynchronousProducerFromIterable" must { + "A SynchronousPublisherFromIterable" must { "produce elements" in { - val p = SynchronousProducerFromIterable(List(1, 2, 3)) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = SynchronousPublisherFromIterable(List(1, 2, 3)) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) val sub = c.expectSubscription() - sub.requestMore(1) + sub.request(1) c.expectNext(1) c.expectNoMsg(100.millis) - sub.requestMore(2) + sub.request(2) c.expectNext(2) c.expectNext(3) c.expectComplete() } "complete empty" in { - val p = SynchronousProducerFromIterable(List.empty[Int]) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = SynchronousPublisherFromIterable(List.empty[Int]) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) c.expectComplete() c.expectNoMsg(100.millis) - val c2 = StreamTestKit.consumerProbe[Int] - p.produceTo(c2) + val c2 = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c2) c2.expectComplete() } "produce elements with multiple subscribers" in { - val p = SynchronousProducerFromIterable(List(1, 2, 3)) - val c1 = StreamTestKit.consumerProbe[Int] - val c2 = StreamTestKit.consumerProbe[Int] - p.produceTo(c1) - p.produceTo(c2) + val p = SynchronousPublisherFromIterable(List(1, 2, 3)) + val c1 = StreamTestKit.SubscriberProbe[Int]() + val c2 = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c1) + p.subscribe(c2) val sub1 = c1.expectSubscription() val sub2 = c2.expectSubscription() - sub1.requestMore(1) - sub2.requestMore(2) + sub1.request(1) + sub2.request(2) c1.expectNext(1) c2.expectNext(1) c2.expectNext(2) c1.expectNoMsg(100.millis) c2.expectNoMsg(100.millis) - sub1.requestMore(2) - sub2.requestMore(2) + sub1.request(2) + sub2.request(2) c1.expectNext(2) c1.expectNext(3) c2.expectNext(3) @@ -67,49 +65,48 @@ class SynchronousProducerFromIterableSpec extends AkkaSpec { } "produce elements to later subscriber" in { - val p = SynchronousProducerFromIterable(List(1, 2, 3)) - val c1 = StreamTestKit.consumerProbe[Int] - val c2 = StreamTestKit.consumerProbe[Int] - p.produceTo(c1) + val p = SynchronousPublisherFromIterable(List(1, 2, 3)) + val c1 = StreamTestKit.SubscriberProbe[Int]() + val c2 = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c1) val sub1 = c1.expectSubscription() - sub1.requestMore(1) + sub1.request(1) c1.expectNext(1) c1.expectNoMsg(100.millis) - p.produceTo(c2) + p.subscribe(c2) val sub2 = c2.expectSubscription() - sub2.requestMore(2) + sub2.request(2) // starting from first element, new iterator per subscriber c2.expectNext(1) c2.expectNext(2) c2.expectNoMsg(100.millis) - sub2.requestMore(1) + sub2.request(1) c2.expectNext(3) c2.expectComplete() - sub1.requestMore(2) + sub1.request(2) c1.expectNext(2) c1.expectNext(3) c1.expectComplete() } "not produce after cancel" in { - val p = SynchronousProducerFromIterable(List(1, 2, 3)) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = SynchronousPublisherFromIterable(List(1, 2, 3)) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) val sub = c.expectSubscription() - sub.requestMore(1) + sub.request(1) c.expectNext(1) sub.cancel() - sub.requestMore(2) + sub.request(2) c.expectNoMsg(100.millis) } "not produce after cancel from onNext" in { - val p = SynchronousProducerFromIterable(List(1, 2, 3, 4, 5)) + val p = SynchronousPublisherFromIterable(List(1, 2, 3, 4, 5)) val probe = TestProbe() - p.produceTo(new Consumer[Int] with Subscriber[Int] { + p.subscribe(new Subscriber[Int] { var sub: Subscription = _ - override val getSubscriber: Subscriber[Int] = this override def onError(cause: Throwable): Unit = probe.ref ! cause override def onComplete(): Unit = probe.ref ! "complete" override def onNext(element: Int): Unit = { @@ -118,7 +115,7 @@ class SynchronousProducerFromIterableSpec extends AkkaSpec { } override def onSubscribe(subscription: Subscription): Unit = { sub = subscription - sub.requestMore(10) + sub.request(10) } }) @@ -140,59 +137,58 @@ class SynchronousProducerFromIterableSpec extends AkkaSpec { } } } - val p = SynchronousProducerFromIterable(iterable) - val c = StreamTestKit.consumerProbe[Int] - p.produceTo(c) + val p = SynchronousPublisherFromIterable(iterable) + val c = StreamTestKit.SubscriberProbe[Int]() + p.subscribe(c) val sub = c.expectSubscription() - sub.requestMore(1) + sub.request(1) c.expectNext(1) c.expectNoMsg(100.millis) - sub.requestMore(2) + sub.request(2) c.expectError.getMessage should be("not two") - sub.requestMore(2) + sub.request(2) c.expectNoMsg(100.millis) } "handle reentrant requests" in { val N = 50000 - val p = SynchronousProducerFromIterable(1 to N) + val p = SynchronousPublisherFromIterable(1 to N) val probe = TestProbe() - p.produceTo(new Consumer[Int] with Subscriber[Int] { + p.subscribe(new Subscriber[Int] { var sub: Subscription = _ - override val getSubscriber: Subscriber[Int] = this override def onError(cause: Throwable): Unit = probe.ref ! cause override def onComplete(): Unit = probe.ref ! "complete" override def onNext(element: Int): Unit = { probe.ref ! element - sub.requestMore(1) + sub.request(1) } override def onSubscribe(subscription: Subscription): Unit = { sub = subscription - sub.requestMore(1) + sub.request(1) } }) probe.receiveN(N) should be((1 to N).toVector) probe.expectMsg("complete") } - "have value equality of producer" in { - val p1 = SynchronousProducerFromIterable(List(1, 2, 3)) - val p2 = SynchronousProducerFromIterable(List(1, 2, 3)) + "have value equality of publisher" in { + val p1 = SynchronousPublisherFromIterable(List(1, 2, 3)) + val p2 = SynchronousPublisherFromIterable(List(1, 2, 3)) p1 should be(p2) p2 should be(p1) - val p3 = SynchronousProducerFromIterable(List(1, 2, 3, 4)) + val p3 = SynchronousPublisherFromIterable(List(1, 2, 3, 4)) p1 should not be (p3) p3 should not be (p1) - val p4 = SynchronousProducerFromIterable(Vector.empty[String]) - val p5 = SynchronousProducerFromIterable(Set.empty[String]) + val p4 = SynchronousPublisherFromIterable(Vector.empty[String]) + val p5 = SynchronousPublisherFromIterable(Set.empty[String]) p1 should not be (p4) p4 should be(p5) p5 should be(p4) } "have nice toString" in { - SynchronousProducerFromIterable(List(1, 2, 3)).toString should be("SynchronousProducerFromIterable(1, 2, 3)") + SynchronousPublisherFromIterable(List(1, 2, 3)).toString should be("SynchronousPublisherFromIterable(1, 2, 3)") } } } diff --git a/akka-stream/src/test/scala/akka/stream/io/TcpFlowSpec.scala b/akka-stream/src/test/scala/akka/stream/io/TcpFlowSpec.scala index 6dd4b3b1ca..613bc674bd 100644 --- a/akka-stream/src/test/scala/akka/stream/io/TcpFlowSpec.scala +++ b/akka-stream/src/test/scala/akka/stream/io/TcpFlowSpec.scala @@ -12,7 +12,7 @@ import akka.stream.impl.ActorProcessor import akka.stream.scaladsl.Flow import akka.util.ByteString import akka.stream.testkit.{ StreamTestKit, AkkaSpec } -import org.reactivestreams.api.Processor +import org.reactivestreams.Processor import akka.actor.{ Props, ActorRef, Actor } import scala.collection.immutable.Queue import scala.concurrent.{ Future, Await } @@ -149,15 +149,15 @@ class TcpFlowSpec extends AkkaSpec { } class TcpReadProbe(tcpProcessor: Processor[ByteString, ByteString]) { - val consumerProbe = StreamTestKit.consumerProbe[ByteString]() - tcpProcessor.produceTo(consumerProbe) - val tcpReadSubscription = consumerProbe.expectSubscription() + val subscriberProbe = StreamTestKit.SubscriberProbe[ByteString]() + tcpProcessor.subscribe(subscriberProbe) + val tcpReadSubscription = subscriberProbe.expectSubscription() def read(count: Int): ByteString = { var result = ByteString.empty while (result.size < count) { - tcpReadSubscription.requestMore(1) - result ++= consumerProbe.expectNext() + tcpReadSubscription.request(1) + result ++= subscriberProbe.expectNext() } result } @@ -166,13 +166,13 @@ class TcpFlowSpec extends AkkaSpec { } class TcpWriteProbe(tcpProcessor: Processor[ByteString, ByteString]) { - val producerProbe = StreamTestKit.producerProbe[ByteString]() - producerProbe.produceTo(tcpProcessor) - val tcpWriteSubscription = producerProbe.expectSubscription() + val publisherProbe = StreamTestKit.PublisherProbe[ByteString]() + publisherProbe.subscribe(tcpProcessor) + val tcpWriteSubscription = publisherProbe.expectSubscription() var demand = 0 def write(bytes: ByteString): Unit = { - if (demand == 0) demand += tcpWriteSubscription.expectRequestMore() + if (demand == 0) demand += tcpWriteSubscription.expectRequest() tcpWriteSubscription.sendNext(bytes) demand -= 1 } @@ -203,7 +203,7 @@ class TcpFlowSpec extends AkkaSpec { def echoServer(serverAddress: InetSocketAddress = temporaryServerAddress): Future[Unit] = Flow(bind(serverAddress).connectionStream).foreach { conn ⇒ - conn.inputStream.produceTo(conn.outputStream) + conn.inputStream.subscribe(conn.outputStream) }.toFuture(materializer) "Outgoing TCP stream" must { @@ -241,7 +241,7 @@ class TcpFlowSpec extends AkkaSpec { serverConnection.read(256) Flow(tcpProcessor).consume(materializer) - Flow(testInput).toProducer(materializer).produceTo(tcpProcessor) + Flow(testInput).toPublisher(materializer).subscribe(tcpProcessor) serverConnection.waitRead() should be(expectedOutput) } @@ -275,7 +275,7 @@ class TcpFlowSpec extends AkkaSpec { serverConnection.write(testData) tcpReadProbe.read(5) should be(testData) serverConnection.confirmedClose() - tcpReadProbe.consumerProbe.expectComplete() + tcpReadProbe.subscriberProbe.expectComplete() } "stop reading when the input stream is cancelled" in { @@ -289,7 +289,7 @@ class TcpFlowSpec extends AkkaSpec { tcpReadProbe.close() // FIXME: expect PeerClosed on server serverConnection.write(testData) - tcpReadProbe.consumerProbe.expectNoMsg(1.second) + tcpReadProbe.subscriberProbe.expectNoMsg(1.second) serverConnection.read(5) tcpWriteProbe.write(testData) serverConnection.waitRead() should be(testData) @@ -308,7 +308,7 @@ class TcpFlowSpec extends AkkaSpec { // there should be a chitchat and non-chitchat version serverConnection.confirmedClose() - tcpReadProbe.consumerProbe.expectComplete() + tcpReadProbe.subscriberProbe.expectComplete() serverConnection.read(5) tcpWriteProbe.write(testData) @@ -328,7 +328,7 @@ class TcpFlowSpec extends AkkaSpec { val tcpReadProbe = new TcpReadProbe(tcpProcessor) serverConnection.abort() - tcpReadProbe.consumerProbe.expectError() + tcpReadProbe.subscriberProbe.expectError() tcpWriteProbe.tcpWriteSubscription.expectCancellation() } @@ -349,7 +349,7 @@ class TcpFlowSpec extends AkkaSpec { val testInput = Iterator.range(0, 256).map(ByteString(_)) val expectedOutput = ByteString(Array.tabulate(256)(_.asInstanceOf[Byte])) - Flow(testInput).toProducer(materializer).produceTo(conn.outputStream) + Flow(testInput).toPublisher(materializer).subscribe(conn.outputStream) val resultFuture = Flow(conn.inputStream).fold(ByteString.empty)((acc, in) ⇒ acc ++ in).toFuture(materializer) Await.result(resultFuture, 3.seconds) should be(expectedOutput) @@ -368,9 +368,9 @@ class TcpFlowSpec extends AkkaSpec { val testInput = Iterator.range(0, 256).map(ByteString(_)) val expectedOutput = ByteString(Array.tabulate(256)(_.asInstanceOf[Byte])) - Flow(testInput).toProducer(materializer).produceTo(conn1.outputStream) - conn1.inputStream.produceTo(conn2.outputStream) - conn2.inputStream.produceTo(conn3.outputStream) + Flow(testInput).toPublisher(materializer).subscribe(conn1.outputStream) + conn1.inputStream.subscribe(conn2.outputStream) + conn2.inputStream.subscribe(conn3.outputStream) val resultFuture = Flow(conn3.inputStream).fold(ByteString.empty)((acc, in) ⇒ acc ++ in).toFuture(materializer) Await.result(resultFuture, 3.seconds) should be(expectedOutput) diff --git a/akka-stream/src/test/scala/akka/stream/testkit/AkkaConsumerProbe.scala b/akka-stream/src/test/scala/akka/stream/testkit/AkkaConsumerProbe.scala deleted file mode 100644 index f08e773737..0000000000 --- a/akka-stream/src/test/scala/akka/stream/testkit/AkkaConsumerProbe.scala +++ /dev/null @@ -1,10 +0,0 @@ -/** - * Copyright (C) 2014 Typesafe Inc. - */ -package akka.stream.testkit - -import akka.testkit.TestProbe - -trait AkkaConsumerProbe[I] extends ConsumerProbe[I] { - def probe: TestProbe -} diff --git a/akka-stream/src/test/scala/akka/stream/testkit/AkkaProducerProbe.scala b/akka-stream/src/test/scala/akka/stream/testkit/AkkaProducerProbe.scala deleted file mode 100644 index 7a5785e0ba..0000000000 --- a/akka-stream/src/test/scala/akka/stream/testkit/AkkaProducerProbe.scala +++ /dev/null @@ -1,10 +0,0 @@ -/** - * Copyright (C) 2014 Typesafe Inc. - */ -package akka.stream.testkit - -import akka.testkit.TestProbe - -trait AkkaProducerProbe[I] extends ProducerProbe[I] { - def probe: TestProbe -} diff --git a/akka-stream/src/test/scala/akka/stream/testkit/ChainSetup.scala b/akka-stream/src/test/scala/akka/stream/testkit/ChainSetup.scala index f984edcb8c..3928616c1f 100644 --- a/akka-stream/src/test/scala/akka/stream/testkit/ChainSetup.scala +++ b/akka-stream/src/test/scala/akka/stream/testkit/ChainSetup.scala @@ -8,12 +8,12 @@ import akka.actor.ActorSystem import akka.stream.scaladsl.Flow class ChainSetup[I, O](stream: Flow[I] ⇒ Flow[O], val settings: MaterializerSettings)(implicit val system: ActorSystem) { - val upstream = StreamTestKit.producerProbe[I]() - val downstream = StreamTestKit.consumerProbe[O]() + val upstream = StreamTestKit.PublisherProbe[I]() + val downstream = StreamTestKit.SubscriberProbe[O]() private val s = stream(Flow(upstream)) - val producer = s.toProducer(FlowMaterializer(settings)) + val publisher = s.toPublisher(FlowMaterializer(settings)) val upstreamSubscription = upstream.expectSubscription() - producer.produceTo(downstream) + publisher.subscribe(downstream) val downstreamSubscription = downstream.expectSubscription() } diff --git a/akka-stream/src/test/scala/akka/stream/testkit/ConsumerProbe.scala b/akka-stream/src/test/scala/akka/stream/testkit/ConsumerProbe.scala deleted file mode 100644 index 94a9af32c5..0000000000 --- a/akka-stream/src/test/scala/akka/stream/testkit/ConsumerProbe.scala +++ /dev/null @@ -1,31 +0,0 @@ -/** - * Copyright (C) 2014 Typesafe Inc. - */ -package akka.stream.testkit - -import scala.concurrent.duration.FiniteDuration -import org.reactivestreams.api.Consumer -import org.reactivestreams.spi.Subscription - -sealed trait ConsumerEvent -case class OnSubscribe(subscription: Subscription) extends ConsumerEvent -case class OnNext[I](element: I) extends ConsumerEvent -case object OnComplete extends ConsumerEvent -case class OnError(cause: Throwable) extends ConsumerEvent - -trait ConsumerProbe[I] extends Consumer[I] { - def expectSubscription(): Subscription - def expectEvent(event: ConsumerEvent): Unit - def expectNext(element: I): Unit - def expectNext(e1: I, e2: I, es: I*): Unit - def expectNext(): I - def expectError(cause: Throwable): Unit - def expectError(): Throwable - def expectErrorOrSubscriptionFollowedByError(cause: Throwable): Unit - def expectErrorOrSubscriptionFollowedByError(): Throwable - def expectCompletedOrSubscriptionFollowedByComplete() - def expectComplete(): Unit - - def expectNoMsg(): Unit - def expectNoMsg(max: FiniteDuration): Unit -} diff --git a/akka-stream/src/test/scala/akka/stream/testkit/ProducerProbe.scala b/akka-stream/src/test/scala/akka/stream/testkit/ProducerProbe.scala deleted file mode 100644 index dd9a6c008c..0000000000 --- a/akka-stream/src/test/scala/akka/stream/testkit/ProducerProbe.scala +++ /dev/null @@ -1,35 +0,0 @@ -/** - * Copyright (C) 2014 Typesafe Inc. - */ -package akka.stream.testkit - -import org.reactivestreams.spi.{ Subscriber, Subscription } -import org.reactivestreams.api.Producer -import scala.concurrent.duration.FiniteDuration -import org.reactivestreams.api.Consumer - -sealed trait ProducerEvent -case class Subscribe(subscription: Subscription) extends ProducerEvent -case class CancelSubscription(subscription: Subscription) extends ProducerEvent -case class RequestMore(subscription: Subscription, elements: Int) extends ProducerEvent - -abstract case class ActiveSubscription[I](subscriber: Subscriber[I]) extends Subscription { - def sendNext(element: I): Unit - def sendComplete(): Unit - def sendError(cause: Exception): Unit - - def expectCancellation(): Unit - def expectRequestMore(n: Int): Unit - def expectRequestMore(): Int -} - -trait ProducerProbe[I] extends Producer[I] { - def expectSubscription(): ActiveSubscription[I] - def expectRequestMore(subscription: Subscription, n: Int): Unit - - def expectNoMsg(): Unit - def expectNoMsg(max: FiniteDuration): Unit - - def produceTo(consumer: Consumer[I]): Unit = - getPublisher.subscribe(consumer.getSubscriber) -} diff --git a/akka-stream/src/test/scala/akka/stream/testkit/ScriptedTest.scala b/akka-stream/src/test/scala/akka/stream/testkit/ScriptedTest.scala index 90c6dfb532..291d54b8ef 100644 --- a/akka-stream/src/test/scala/akka/stream/testkit/ScriptedTest.scala +++ b/akka-stream/src/test/scala/akka/stream/testkit/ScriptedTest.scala @@ -4,15 +4,15 @@ package akka.stream.testkit import akka.actor.ActorSystem -import scala.annotation.tailrec -import scala.collection.immutable -import scala.concurrent.forkjoin.ThreadLocalRandom -import scala.concurrent.duration._ -import scala.util.control.NonFatal -import akka.stream.scaladsl.Flow import akka.stream.MaterializerSettings +import akka.stream.scaladsl.Flow +import akka.stream.testkit.StreamTestKit._ import org.scalatest.Matchers +import scala.annotation.tailrec +import scala.concurrent.duration._ +import scala.concurrent.forkjoin.ThreadLocalRandom + trait ScriptedTest extends Matchers { class ScriptException(msg: String) extends RuntimeException(msg) @@ -107,9 +107,9 @@ trait ScriptedTest extends Matchers { def debugLog(msg: String): Unit = _debugLog :+= msg - def requestMore(demand: Int): Unit = { + def request(demand: Int): Unit = { debugLog(s"test environment requests $demand") - downstreamSubscription.requestMore(demand) + downstreamSubscription.request(demand) outstandingDemand += demand } @@ -158,7 +158,7 @@ trait ScriptedTest extends Matchers { upstreamSubscription.sendNext(input) doRun(nextIdle) } else if (mayRequestMore && (!mayProvideInput || !tieBreak)) { - requestMore(getNextDemand()) + request(getNextDemand()) doRun(nextIdle) } else { if (currentScript.noInsPending && !completed) { @@ -174,7 +174,7 @@ trait ScriptedTest extends Matchers { try { debugLog(s"running $script") - requestMore(getNextDemand()) + request(getNextDemand()) doRun(0) } catch { case e: Throwable ⇒ diff --git a/akka-stream/src/test/scala/akka/stream/testkit/StreamTestKit.scala b/akka-stream/src/test/scala/akka/stream/testkit/StreamTestKit.scala index 67d8df987e..29afcd49e7 100644 --- a/akka-stream/src/test/scala/akka/stream/testkit/StreamTestKit.scala +++ b/akka-stream/src/test/scala/akka/stream/testkit/StreamTestKit.scala @@ -3,117 +3,19 @@ */ package akka.stream.testkit -import akka.testkit.TestProbe -import org.reactivestreams.spi.{ Publisher, Subscriber, Subscription } -import org.reactivestreams.tck._ import akka.actor.ActorSystem +import akka.stream.impl.{ EmptyPublisher, ErrorPublisher } +import akka.testkit.TestProbe +import org.reactivestreams.{ Publisher, Subscriber, Subscription } + import scala.concurrent.duration.FiniteDuration -import scala.annotation.tailrec -import akka.stream.impl.EmptyProducer -import org.reactivestreams.api.Producer -import akka.stream.impl.ErrorProducer -import org.reactivestreams.api.Consumer object StreamTestKit { - def consumerProbe[I]()(implicit system: ActorSystem): AkkaConsumerProbe[I] = - new AkkaConsumerProbe[I] with Subscriber[I] { outer ⇒ - lazy val probe = TestProbe() - - def expectSubscription(): Subscription = probe.expectMsgType[OnSubscribe].subscription - def expectEvent(event: ConsumerEvent): Unit = probe.expectMsg(event) - def expectNext(element: I): Unit = probe.expectMsg(OnNext(element)) - def expectNext(e1: I, e2: I, es: I*): Unit = { - val all = e1 +: e2 +: es - all.foreach(e ⇒ probe.expectMsg(OnNext(e))) - } - - def expectNext(): I = probe.expectMsgType[OnNext[I]].element - def expectComplete(): Unit = probe.expectMsg(OnComplete) - - def expectError(cause: Throwable): Unit = probe.expectMsg(OnError(cause)) - def expectError(): Throwable = probe.expectMsgType[OnError].cause - - def expectErrorOrSubscriptionFollowedByError(cause: Throwable): Unit = { - val t = expectErrorOrSubscriptionFollowedByError() - assert(t == cause, s"expected $cause, found $cause") - } - - def expectErrorOrSubscriptionFollowedByError(): Throwable = - probe.expectMsgPF() { - case s: OnSubscribe ⇒ - s.subscription.requestMore(1) - expectError() - case OnError(cause) ⇒ cause - } - - def expectCompletedOrSubscriptionFollowedByComplete(): Unit = { - probe.expectMsgPF() { - case s: OnSubscribe ⇒ - s.subscription.requestMore(1) - expectComplete() - case OnComplete ⇒ - } - } - - def expectNoMsg(): Unit = probe.expectNoMsg() - def expectNoMsg(max: FiniteDuration): Unit = probe.expectNoMsg(max) - - def onSubscribe(subscription: Subscription): Unit = probe.ref ! OnSubscribe(subscription) - def onNext(element: I): Unit = probe.ref ! OnNext(element) - def onComplete(): Unit = probe.ref ! OnComplete - def onError(cause: Throwable): Unit = probe.ref ! OnError(cause) - - def getSubscriber: Subscriber[I] = this - } - - def producerProbe[I]()(implicit system: ActorSystem): AkkaProducerProbe[I] = - new AkkaProducerProbe[I] with Publisher[I] { - lazy val probe: TestProbe = TestProbe() - - def subscribe(subscriber: Subscriber[I]): Unit = { - lazy val subscription: ActiveSubscription[I] = new ActiveSubscription[I](subscriber) { - def requestMore(elements: Int): Unit = probe.ref ! RequestMore(subscription, elements) - def cancel(): Unit = probe.ref ! CancelSubscription(subscription) - - def expectRequestMore(n: Int): Unit = probe.expectMsg(RequestMore(subscription, n)) - def expectRequestMore(): Int = probe.expectMsgPF() { - case RequestMore(`subscription`, n) ⇒ n - } - def expectCancellation(): Unit = probe.fishForMessage() { - case CancelSubscription(`subscription`) ⇒ true - case RequestMore(`subscription`, _) ⇒ false - } - - def sendNext(element: I): Unit = subscriber.onNext(element) - def sendComplete(): Unit = subscriber.onComplete() - def sendError(cause: Exception): Unit = subscriber.onError(cause) - } - probe.ref ! Subscribe(subscription) - subscriber.onSubscribe(subscription) - } - - def expectSubscription(): ActiveSubscription[I] = - probe.expectMsgType[Subscribe].subscription.asInstanceOf[ActiveSubscription[I]] - - def expectRequestMore(subscription: Subscription, n: Int): Unit = probe.expectMsg(RequestMore(subscription, n)) - - def expectNoMsg(): Unit = probe.expectNoMsg() - def expectNoMsg(max: FiniteDuration): Unit = probe.expectNoMsg(max) - - def getPublisher: Publisher[I] = this - } /** - * Completes subscribers immediately, before handing out subscription. + * Subscribes the subscriber and completes after the first request. */ - def emptyProducer[T]: Producer[T] = EmptyProducer.asInstanceOf[Producer[T]] - - /** - * Subscribes the subscriber and completes after the first requestMore. - */ - def lazyEmptyProducer[T]: Producer[T] = new Producer[T] with Publisher[T] { - override def getPublisher = this - override def produceTo(consumer: Consumer[T]): Unit = getPublisher.subscribe(consumer.getSubscriber) + def lazyEmptyPublisher[T]: Publisher[T] = new Publisher[T] { override def subscribe(subscriber: Subscriber[T]): Unit = subscriber.onSubscribe(CompletedSubscription(subscriber)) } @@ -121,36 +23,139 @@ object StreamTestKit { /** * Signals error to subscribers immediately, before handing out subscription. */ - def errorProducer[T](cause: Throwable): Producer[T] = ErrorProducer(cause: Throwable).asInstanceOf[Producer[T]] + def errorPublisher[T](cause: Throwable): Publisher[T] = ErrorPublisher(cause: Throwable).asInstanceOf[Publisher[T]] + + def emptyPublisher[T](): Publisher[T] = EmptyPublisher.asInstanceOf[Publisher[T]] /** - * Subscribes the subscriber and signals error after the first requestMore. + * Subscribes the subscriber and signals error after the first request. */ - def lazyErrorProducer[T](cause: Throwable): Producer[T] = new Producer[T] with Publisher[T] { - override def getPublisher = this - override def produceTo(consumer: Consumer[T]): Unit = getPublisher.subscribe(consumer.getSubscriber) + def lazyErrorPublisher[T](cause: Throwable): Publisher[T] = new Publisher[T] { override def subscribe(subscriber: Subscriber[T]): Unit = subscriber.onSubscribe(FailedSubscription(subscriber, cause)) } private case class FailedSubscription[T](subscriber: Subscriber[T], cause: Throwable) extends Subscription { - override def requestMore(elements: Int): Unit = subscriber.onError(cause) + override def request(elements: Int): Unit = subscriber.onError(cause) override def cancel(): Unit = () } private case class CompletedSubscription[T](subscriber: Subscriber[T]) extends Subscription { - override def requestMore(elements: Int): Unit = subscriber.onComplete() + override def request(elements: Int): Unit = subscriber.onComplete() override def cancel(): Unit = () } - class AutoProducer[T](probe: ProducerProbe[T], initialPendingRequests: Int = 0) { + class AutoPublisher[T](probe: PublisherProbe[T], initialPendingRequests: Int = 0) { val subscription = probe.expectSubscription() var pendingRequests = initialPendingRequests def sendNext(elem: T): Unit = { - if (pendingRequests == 0) pendingRequests = subscription.expectRequestMore() + if (pendingRequests == 0) pendingRequests = subscription.expectRequest() pendingRequests -= 1 subscription.sendNext(elem) } } + + sealed trait SubscriberEvent + case class OnSubscribe(subscription: Subscription) extends SubscriberEvent + case class OnNext[I](element: I) extends SubscriberEvent + case object OnComplete extends SubscriberEvent + case class OnError(cause: Throwable) extends SubscriberEvent + + sealed trait PublisherEvent + case class Subscribe(subscription: Subscription) extends PublisherEvent + case class CancelSubscription(subscription: Subscription) extends PublisherEvent + case class RequestMore(subscription: Subscription, elements: Int) extends PublisherEvent + + case class PublisherProbeSubscription[I](subscriber: Subscriber[I], publisherProbe: TestProbe) extends Subscription { + def request(elements: Int): Unit = publisherProbe.ref ! RequestMore(this, elements) + def cancel(): Unit = publisherProbe.ref ! CancelSubscription(this) + + def expectRequest(n: Int): Unit = publisherProbe.expectMsg(RequestMore(this, n)) + def expectRequest(): Int = publisherProbe.expectMsgPF() { + case RequestMore(sub, n) if sub eq this ⇒ n + } + + def expectCancellation(): Unit = publisherProbe.fishForMessage() { + case CancelSubscription(sub) if sub eq this ⇒ true + case RequestMore(sub, _) if sub eq this ⇒ false + } + + def sendNext(element: I): Unit = subscriber.onNext(element) + def sendComplete(): Unit = subscriber.onComplete() + def sendError(cause: Exception): Unit = subscriber.onError(cause) + } + + case class SubscriberProbe[I]()(implicit system: ActorSystem) extends Subscriber[I] { + val probe = TestProbe() + + def expectSubscription(): Subscription = probe.expectMsgType[OnSubscribe].subscription + def expectEvent(event: SubscriberEvent): Unit = probe.expectMsg(event) + def expectNext(element: I): Unit = probe.expectMsg(OnNext(element)) + def expectNext(e1: I, e2: I, es: I*): Unit = { + val all = e1 +: e2 +: es + all.foreach(e ⇒ probe.expectMsg(OnNext(e))) + } + + def expectNext(): I = probe.expectMsgType[OnNext[I]].element + def expectComplete(): Unit = probe.expectMsg(OnComplete) + + def expectError(cause: Throwable): Unit = probe.expectMsg(OnError(cause)) + def expectError(): Throwable = probe.expectMsgType[OnError].cause + + def expectErrorOrSubscriptionFollowedByError(cause: Throwable): Unit = { + val t = expectErrorOrSubscriptionFollowedByError() + assert(t == cause, s"expected $cause, found $cause") + } + + def expectErrorOrSubscriptionFollowedByError(): Throwable = + probe.expectMsgPF() { + case s: OnSubscribe ⇒ + s.subscription.request(1) + expectError() + case OnError(cause) ⇒ cause + } + + def expectCompletedOrSubscriptionFollowedByComplete(): Unit = { + probe.expectMsgPF() { + case s: OnSubscribe ⇒ + s.subscription.request(1) + expectComplete() + case OnComplete ⇒ + } + } + + def expectNoMsg(): Unit = probe.expectNoMsg() + def expectNoMsg(max: FiniteDuration): Unit = probe.expectNoMsg(max) + + def onSubscribe(subscription: Subscription): Unit = probe.ref ! OnSubscribe(subscription) + def onNext(element: I): Unit = probe.ref ! OnNext(element) + def onComplete(): Unit = probe.ref ! OnComplete + def onError(cause: Throwable): Unit = probe.ref ! OnError(cause) + + // Keeping equality + // FIXME: This and PublisherProbe should not be a case class so that we don't need this equality reversal + override def equals(that: Any): Boolean = this eq that.asInstanceOf[AnyRef] + override def hashCode(): Int = System.identityHashCode(this) + } + + case class PublisherProbe[I]()(implicit system: ActorSystem) extends Publisher[I] { + val probe: TestProbe = TestProbe() + + def subscribe(subscriber: Subscriber[I]): Unit = { + val subscription: PublisherProbeSubscription[I] = new PublisherProbeSubscription[I](subscriber, probe) + probe.ref ! Subscribe(subscription) + subscriber.onSubscribe(subscription) + } + + def expectSubscription(): PublisherProbeSubscription[I] = + probe.expectMsgType[Subscribe].subscription.asInstanceOf[PublisherProbeSubscription[I]] + + def expectRequest(subscription: Subscription, n: Int): Unit = probe.expectMsg(RequestMore(subscription, n)) + + def expectNoMsg(): Unit = probe.expectNoMsg() + def expectNoMsg(max: FiniteDuration): Unit = probe.expectNoMsg(max) + + def getPublisher: Publisher[I] = this + } }