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 6869d9c469..0b57ac2254 100644 --- a/akka-http-core/src/main/scala/akka/http/HttpManager.scala +++ b/akka-http-core/src/main/scala/akka/http/HttpManager.scala @@ -11,7 +11,7 @@ import akka.http.engine.client._ import akka.http.engine.server.{ HttpServerPipeline, ServerSettings } import akka.io.IO import akka.pattern.ask -import akka.stream.scaladsl2.{ PublisherDrain, Source, FlowMaterializer } +import akka.stream.scaladsl2.{ Sink, Source, FlowMaterializer } import akka.stream.io.StreamTcp import akka.util.Timeout @@ -65,7 +65,7 @@ private[http] class HttpManager(httpSettings: HttpExt#Settings) extends Actor wi val httpServerPipeline = new HttpServerPipeline(effectiveSettings, log) val httpConnectionStream = Source(connectionStream) .map(httpServerPipeline) - .runWith(PublisherDrain()) + .runWith(Sink.publisher) commander ! Http.ServerBinding(localAddress, httpConnectionStream, tcpServerBinding) case Failure(error) ⇒ diff --git a/akka-http-core/src/main/scala/akka/http/engine/client/HttpClientPipeline.scala b/akka-http-core/src/main/scala/akka/http/engine/client/HttpClientPipeline.scala index fc1f5d1d19..807c0f06cb 100644 --- a/akka-http-core/src/main/scala/akka/http/engine/client/HttpClientPipeline.scala +++ b/akka-http-core/src/main/scala/akka/http/engine/client/HttpClientPipeline.scala @@ -39,11 +39,11 @@ private[http] class HttpClientPipeline(effectiveSettings: ClientConnectionSettin val requestMethodByPass = new RequestMethodByPass(tcpConn.remoteAddress) - val userIn = SubscriberTap[(HttpRequest, Any)]() - val userOut = PublisherDrain[(HttpResponse, Any)]() + val userIn = Source.subscriber[(HttpRequest, Any)] + val userOut = Sink.publisher[(HttpResponse, Any)] - val netOut = SubscriberDrain(tcpConn.outputStream) - val netIn = PublisherTap(tcpConn.inputStream) + val netOut = Sink(tcpConn.outputStream) + val netIn = Source(tcpConn.inputStream) val pipeline = FlowGraph { implicit b ⇒ val bypassFanout = Broadcast[(HttpRequest, Any)]("bypassFanout") @@ -76,8 +76,8 @@ private[http] class HttpClientPipeline(effectiveSettings: ClientConnectionSettin Http.OutgoingConnection( tcpConn.remoteAddress, tcpConn.localAddress, - pipeline.materializedDrain(userOut), - pipeline.materializedTap(userIn)) + pipeline.get(userOut), + pipeline.get(userIn)) } class RequestMethodByPass(serverAddress: InetSocketAddress) diff --git a/akka-http-core/src/main/scala/akka/http/engine/rendering/RenderSupport.scala b/akka-http-core/src/main/scala/akka/http/engine/rendering/RenderSupport.scala index 92aa4f63a1..ef54fa28ee 100644 --- a/akka-http-core/src/main/scala/akka/http/engine/rendering/RenderSupport.scala +++ b/akka-http-core/src/main/scala/akka/http/engine/rendering/RenderSupport.scala @@ -33,9 +33,9 @@ private object RenderSupport { // This hooks into the materialization to cancel the not needed second source. This helper class // allows us to not take a FlowMaterializer but delegate the cancellation to the point when the whole stream // materializes - private case class CancelSecond[T](first: Source[T], second: Source[T]) extends SimpleTap[T] { + private case class CancelSecond[T](first: Source[T], second: Source[T]) extends SimpleActorFlowSource[T] { override def attach(flowSubscriber: Subscriber[T], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = { - first.connect(SubscriberDrain(flowSubscriber)).run()(materializer) + first.connect(Sink(flowSubscriber)).run()(materializer) second.connect(Sink.cancelled).run()(materializer) } } diff --git a/akka-http-core/src/main/scala/akka/http/engine/server/HttpServerPipeline.scala b/akka-http-core/src/main/scala/akka/http/engine/server/HttpServerPipeline.scala index 2e8a4b33f6..83149b0255 100644 --- a/akka-http-core/src/main/scala/akka/http/engine/server/HttpServerPipeline.scala +++ b/akka-http-core/src/main/scala/akka/http/engine/server/HttpServerPipeline.scala @@ -32,11 +32,11 @@ private[http] class HttpServerPipeline(settings: ServerSettings, log: LoggingAda def apply(tcpConn: StreamTcp.IncomingTcpConnection): Http.IncomingConnection = { import FlowGraphImplicits._ - val networkIn = PublisherTap(tcpConn.inputStream) - val networkOut = SubscriberDrain(tcpConn.outputStream) + val networkIn = Source(tcpConn.inputStream) + val networkOut = Sink(tcpConn.outputStream) - val userIn = PublisherDrain[HttpRequest]() - val userOut = SubscriberTap[HttpResponse]() + val userIn = Sink.publisher[HttpRequest] + val userOut = Source.subscriber[HttpResponse] val pipeline = FlowGraph { implicit b ⇒ val bypassFanout = Broadcast[(RequestOutput, Source[RequestOutput])]("bypassFanout") @@ -73,7 +73,7 @@ private[http] class HttpServerPipeline(settings: ServerSettings, log: LoggingAda }.run() - Http.IncomingConnection(tcpConn.remoteAddress, pipeline.materializedDrain(userIn), pipeline.materializedTap(userOut)) + Http.IncomingConnection(tcpConn.remoteAddress, pipeline.get(userIn), pipeline.get(userOut)) } /** 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 837ff9fad8..c6b5dc6229 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 @@ -60,7 +60,7 @@ sealed trait HttpEntity extends japi.HttpEntity { throw new java.util.concurrent.TimeoutException( s"HttpEntity.toStrict timed out after $timeout while still waiting for outstanding data") } - dataBytes.timerTransform("toStrict", transformer).runWith(FutureDrain()) + dataBytes.timerTransform("toStrict", transformer).runWith(Sink.future) } /** 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 fb170f4210..5977988185 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 @@ -7,7 +7,7 @@ package akka.http.model import java.io.File import scala.concurrent.{ Future, ExecutionContext } import scala.collection.immutable -import akka.stream.scaladsl2.{ FutureDrain, FlowMaterializer, Source } +import akka.stream.scaladsl2.{ FlowMaterializer, Sink, Source } import akka.stream.impl.SynchronousPublisherFromIterable import akka.http.util.FastFuture import FastFuture._ @@ -56,7 +56,7 @@ case class MultipartFormData(parts: Source[BodyPart]) extends MultipartParts { * hint. */ def toStrict(maxFieldCount: Int = 1000)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[StrictMultipartFormData] = - parts.grouped(maxFieldCount).runWith(FutureDrain()).fast.map(new StrictMultipartFormData(_)) + parts.grouped(maxFieldCount).runWith(Sink.future).fast.map(new StrictMultipartFormData(_)) } /** 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 6605551cad..5946831043 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 @@ -12,8 +12,8 @@ import akka.dispatch.Foreach; import akka.japi.Function; import akka.japi.Procedure; import akka.stream.javadsl.Flow; +import akka.stream.javadsl.Sink; import akka.stream.javadsl.Source; -import akka.stream.javadsl.SubscriberDrain; import akka.stream.scaladsl2.FlowMaterializer; import scala.concurrent.Future; @@ -46,7 +46,7 @@ public abstract class JavaTestServer { System.out.println("Handling request to " + request.getUri()); return JavaApiTestCases.handleRequest(request); } - }).runWith(SubscriberDrain.create(conn.getResponseSubscriber()), materializer); + }).runWith(Sink.subscriber(conn.getResponseSubscriber()), materializer); } }, 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 9dbd1a494c..a9c08b1464 100644 --- a/akka-http-core/src/test/scala/akka/http/ClientServerSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/ClientServerSpec.scala @@ -107,7 +107,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll { 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") - Await.result(chunkStream.grouped(4).runWith(FutureDrain()), 100.millis) shouldEqual chunks + Await.result(chunkStream.grouped(4).runWith(Sink.future), 100.millis) shouldEqual chunks val serverOutSub = serverOut.expectSubscription() serverOutSub.sendNext(HttpResponse(206, List(RawHeader("Age", "42")), chunkedEntity)) @@ -116,7 +116,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll { 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(chunkStream2.grouped(1000).runWith(FutureDrain()), 100.millis) shouldEqual chunks + Await.result(chunkStream2.grouped(1000).runWith(Sink.future), 100.millis) shouldEqual chunks } } diff --git a/akka-http-core/src/test/scala/akka/http/TestClient.scala b/akka-http-core/src/test/scala/akka/http/TestClient.scala index b3372cd2e5..1ce610bd97 100644 --- a/akka-http-core/src/test/scala/akka/http/TestClient.scala +++ b/akka-http-core/src/test/scala/akka/http/TestClient.scala @@ -12,7 +12,7 @@ import scala.util.{ Failure, Success } import akka.actor.ActorSystem import akka.pattern.ask import akka.util.Timeout -import akka.stream.scaladsl2.{ FutureDrain, SubscriberDrain, Source, FlowMaterializer } +import akka.stream.scaladsl2.{ FlowMaterializer, Sink, Source } import akka.io.IO import akka.http.model.HttpMethods._ import akka.http.model._ @@ -38,9 +38,9 @@ object TestClient extends App { def sendRequest(request: HttpRequest, connection: Http.OutgoingConnection): Future[HttpResponse] = { Source(List(HttpRequest() -> 'NoContext)) - .connect(SubscriberDrain(connection.requestSubscriber)) + .connect(Sink(connection.requestSubscriber)) .run() - Source(connection.responsePublisher).map(_._1).runWith(FutureDrain()) + Source(connection.responsePublisher).map(_._1).runWith(Sink.future) } result onComplete { 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 a349698d77..44cda92e5a 100644 --- a/akka-http-core/src/test/scala/akka/http/TestServer.scala +++ b/akka-http-core/src/test/scala/akka/http/TestServer.scala @@ -6,7 +6,7 @@ package akka.http import com.typesafe.config.{ ConfigFactory, Config } import scala.concurrent.duration._ -import akka.stream.scaladsl2.{ SubscriberDrain, Source, FlowMaterializer } +import akka.stream.scaladsl2.{ Sink, Source, FlowMaterializer } import akka.io.IO import akka.util.Timeout import akka.actor.ActorSystem @@ -38,7 +38,7 @@ object TestServer extends App { Source(connectionStream).foreach { case Http.IncomingConnection(remoteAddress, requestPublisher, responseSubscriber) ⇒ println("Accepted new connection from " + remoteAddress) - Source(requestPublisher).map(requestHandler).connect(SubscriberDrain(responseSubscriber)).run() + Source(requestPublisher).map(requestHandler).connect(Sink(responseSubscriber)).run() } } diff --git a/akka-http-core/src/test/scala/akka/http/engine/parsing/RequestParserSpec.scala b/akka-http-core/src/test/scala/akka/http/engine/parsing/RequestParserSpec.scala index c312c42d58..3a81af9f8f 100644 --- a/akka-http-core/src/test/scala/akka/http/engine/parsing/RequestParserSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/engine/parsing/RequestParserSpec.scala @@ -418,7 +418,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { } .flatten(FlattenStrategy.concat) .map(strictEqualify) - .grouped(1000).runWith(FutureDrain()) + .grouped(1000).runWith(Sink.future) Await.result(future, 250.millis) } @@ -432,7 +432,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { } private def compactEntityChunks(data: Source[ChunkStreamPart]): Future[Seq[ChunkStreamPart]] = - data.grouped(1000).runWith(FutureDrain()) + data.grouped(1000).runWith(Sink.future) .fast.recover { case _: NoSuchElementException ⇒ Nil } def prep(response: String) = response.stripMarginWithNewline("\r\n") diff --git a/akka-http-core/src/test/scala/akka/http/engine/parsing/ResponseParserSpec.scala b/akka-http-core/src/test/scala/akka/http/engine/parsing/ResponseParserSpec.scala index 9eec7d2d82..a0ecb5dbdf 100644 --- a/akka-http-core/src/test/scala/akka/http/engine/parsing/ResponseParserSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/engine/parsing/ResponseParserSpec.scala @@ -270,7 +270,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { } .flatten(FlattenStrategy.concat) .map(strictEqualify) - .grouped(1000).runWith(FutureDrain()) + .grouped(1000).runWith(Sink.future) Await.result(future, 500.millis) } @@ -288,7 +288,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll { } private def compactEntityChunks(data: Source[ChunkStreamPart]): Future[Source[ChunkStreamPart]] = - data.grouped(1000).runWith(FutureDrain()) + data.grouped(1000).runWith(Sink.future) .fast.map(source(_: _*)) .fast.recover { case _: NoSuchElementException ⇒ source() } diff --git a/akka-http-core/src/test/scala/akka/http/engine/rendering/RequestRendererSpec.scala b/akka-http-core/src/test/scala/akka/http/engine/rendering/RequestRendererSpec.scala index f1801d3d9f..ba153c725e 100644 --- a/akka-http-core/src/test/scala/akka/http/engine/rendering/RequestRendererSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/engine/rendering/RequestRendererSpec.scala @@ -223,7 +223,7 @@ class RequestRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll equal(expected.stripMarginWithNewline("\r\n")).matcher[String] compose { request ⇒ val renderer = newRenderer val byteStringSource :: Nil = renderer.onNext(RequestRenderingContext(request, serverAddress)) - val future = byteStringSource.grouped(1000).runWith(FutureDrain()).map(_.reduceLeft(_ ++ _).utf8String) + val future = byteStringSource.grouped(1000).runWith(Sink.future).map(_.reduceLeft(_ ++ _).utf8String) Await.result(future, 250.millis) } } diff --git a/akka-http-core/src/test/scala/akka/http/engine/rendering/ResponseRendererSpec.scala b/akka-http-core/src/test/scala/akka/http/engine/rendering/ResponseRendererSpec.scala index 82a86a714c..27bfc788e4 100644 --- a/akka-http-core/src/test/scala/akka/http/engine/rendering/ResponseRendererSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/engine/rendering/ResponseRendererSpec.scala @@ -369,7 +369,7 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll equal(expected.stripMarginWithNewline("\r\n") -> close).matcher[(String, Boolean)] compose { ctx ⇒ val renderer = newRenderer val byteStringSource :: Nil = renderer.onNext(ctx) - val future = byteStringSource.grouped(1000).runWith(FutureDrain()).map(_.reduceLeft(_ ++ _).utf8String) + val future = byteStringSource.grouped(1000).runWith(Sink.future).map(_.reduceLeft(_ ++ _).utf8String) Await.result(future, 250.millis) -> renderer.isComplete } diff --git a/akka-http-core/src/test/scala/akka/http/engine/server/HttpServerPipelineSpec.scala b/akka-http-core/src/test/scala/akka/http/engine/server/HttpServerPipelineSpec.scala index dedad1a2a9..02c5d198a8 100644 --- a/akka-http-core/src/test/scala/akka/http/engine/server/HttpServerPipelineSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/engine/server/HttpServerPipelineSpec.scala @@ -39,7 +39,7 @@ class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterA inside(expectRequest) { case HttpRequest(HttpMethods.POST, _, _, HttpEntity.Default(_, 12, data), _) ⇒ val dataProbe = StreamTestKit.SubscriberProbe[ByteString] - data.connect(SubscriberDrain(dataProbe)).run() + data.connect(Sink(dataProbe)).run() val sub = dataProbe.expectSubscription() sub.request(10) dataProbe.expectNoMsg(50.millis) @@ -75,7 +75,7 @@ class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterA inside(expectRequest) { case HttpRequest(HttpMethods.POST, _, _, HttpEntity.Chunked(_, data), _) ⇒ val dataProbe = StreamTestKit.SubscriberProbe[ChunkStreamPart] - data.connect(SubscriberDrain(dataProbe)).run() + data.connect(Sink(dataProbe)).run() val sub = dataProbe.expectSubscription() sub.request(10) dataProbe.expectNext(Chunk(ByteString("abcdef"))) @@ -111,7 +111,7 @@ class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterA inside(expectRequest) { case HttpRequest(HttpMethods.POST, _, _, HttpEntity.Default(_, 12, data), _) ⇒ val dataProbe = StreamTestKit.SubscriberProbe[ByteString] - data.connect(SubscriberDrain(dataProbe)).run() + data.connect(Sink(dataProbe)).run() val sub = dataProbe.expectSubscription() sub.request(10) dataProbe.expectNext(ByteString("abcdef")) @@ -133,7 +133,7 @@ class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterA inside(expectRequest) { case HttpRequest(HttpMethods.POST, _, _, HttpEntity.Chunked(_, data), _) ⇒ val dataProbe = StreamTestKit.SubscriberProbe[ChunkStreamPart] - data.connect(SubscriberDrain(dataProbe)).run() + data.connect(Sink(dataProbe)).run() val sub = dataProbe.expectSubscription() sub.request(10) dataProbe.expectNext(Chunk(ByteString("abcdef"))) @@ -181,7 +181,7 @@ class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterA inside(expectRequest) { case HttpRequest(HttpMethods.POST, _, _, HttpEntity.Default(_, 12, data), _) ⇒ val dataProbe = StreamTestKit.SubscriberProbe[ByteString] - data.connect(SubscriberDrain(dataProbe)).run() + data.connect(Sink(dataProbe)).run() val sub = dataProbe.expectSubscription() sub.request(10) dataProbe.expectNext(ByteString("abcdef")) @@ -217,7 +217,7 @@ class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterA inside(expectRequest) { case HttpRequest(HttpMethods.POST, _, _, HttpEntity.Chunked(_, data), _) ⇒ val dataProbe = StreamTestKit.SubscriberProbe[ChunkStreamPart] - data.connect(SubscriberDrain(dataProbe)).run() + data.connect(Sink(dataProbe)).run() val sub = dataProbe.expectSubscription() sub.request(10) dataProbe.expectNext(Chunk(ByteString("abcdef"))) @@ -253,7 +253,7 @@ class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterA inside(expectRequest) { case HttpRequest(HttpMethods.POST, _, _, HttpEntity.Default(_, 12, data), _) ⇒ val dataProbe = StreamTestKit.SubscriberProbe[ByteString] - data.connect(SubscriberDrain(dataProbe)).run() + data.connect(Sink(dataProbe)).run() val sub = dataProbe.expectSubscription() sub.request(10) dataProbe.expectNext(ByteString("abcdef")) @@ -275,7 +275,7 @@ class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterA inside(expectRequest) { case HttpRequest(HttpMethods.POST, _, _, HttpEntity.Chunked(_, data), _) ⇒ val dataProbe = StreamTestKit.SubscriberProbe[ChunkStreamPart] - data.connect(SubscriberDrain(dataProbe)).run() + data.connect(Sink(dataProbe)).run() val sub = dataProbe.expectSubscription() sub.request(10) dataProbe.expectNext(Chunk(ByteString("abcdef"))) @@ -297,7 +297,7 @@ class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterA inside(expectRequest) { case HttpRequest(HttpMethods.POST, _, _, HttpEntity.Default(_, 12, data), _) ⇒ val dataProbe = StreamTestKit.SubscriberProbe[ByteString] - data.connect(SubscriberDrain(dataProbe)).run() + data.connect(Sink(dataProbe)).run() val sub = dataProbe.expectSubscription() sub.request(10) dataProbe.expectNext(ByteString("abcdef")) @@ -319,7 +319,7 @@ class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterA inside(expectRequest) { case HttpRequest(HttpMethods.POST, _, _, HttpEntity.Chunked(_, data), _) ⇒ val dataProbe = StreamTestKit.SubscriberProbe[ChunkStreamPart] - data.connect(SubscriberDrain(dataProbe)).run() + data.connect(Sink(dataProbe)).run() val sub = dataProbe.expectSubscription() sub.request(10) dataProbe.expectNext(Chunk(ByteString("abcdef"))) 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 e75b1d48c9..d1de8fa71d 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 @@ -106,7 +106,7 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll { def collectBytesTo(bytes: ByteString*): Matcher[HttpEntity] = equal(bytes.toVector).matcher[Seq[ByteString]].compose { entity ⇒ - val future = entity.dataBytes.grouped(1000).runWith(FutureDrain()) + val future = entity.dataBytes.grouped(1000).runWith(Sink.future) Await.result(future, 250.millis) } diff --git a/akka-http-testkit/src/main/scala/akka/http/testkit/RouteTestResultComponent.scala b/akka-http-testkit/src/main/scala/akka/http/testkit/RouteTestResultComponent.scala index 8ba2d84642..5ca76d8e13 100644 --- a/akka-http-testkit/src/main/scala/akka/http/testkit/RouteTestResultComponent.scala +++ b/akka-http-testkit/src/main/scala/akka/http/testkit/RouteTestResultComponent.scala @@ -94,6 +94,6 @@ trait RouteTestResultComponent { failTest("Request was neither completed nor rejected within " + timeout) private def awaitAllElements[T](data: Source[T]): immutable.Seq[T] = - Await.result(data.grouped(Int.MaxValue).runWith(FutureDrain()), timeout) + Await.result(data.grouped(Int.MaxValue).runWith(Sink.future), timeout) } } \ No newline at end of file diff --git a/akka-http-tests/src/test/scala/akka/http/unmarshalling/UnmarshallingSpec.scala b/akka-http-tests/src/test/scala/akka/http/unmarshalling/UnmarshallingSpec.scala index c93f831899..7d793272a1 100644 --- a/akka-http-tests/src/test/scala/akka/http/unmarshalling/UnmarshallingSpec.scala +++ b/akka-http-tests/src/test/scala/akka/http/unmarshalling/UnmarshallingSpec.scala @@ -95,7 +95,7 @@ class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll { |test@there.com |-----""".stripMarginWithNewline("\r\n"))) .to[MultipartContent], 1.second) - Await.result(mpc.parts.runWith(FutureDrain()).failed, 1.second).getMessage shouldEqual + Await.result(mpc.parts.runWith(Sink.future).failed, 1.second).getMessage shouldEqual "multipart part must not contain more than one Content-Type header" } @@ -173,14 +173,14 @@ class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll { def haveParts[T <: MultipartParts](parts: BodyPart*): Matcher[Future[T]] = equal(parts).matcher[Seq[BodyPart]] compose { x ⇒ Await.result(x - .fast.flatMap(x ⇒ x.parts.grouped(100).runWith(FutureDrain())) + .fast.flatMap(x ⇒ x.parts.grouped(100).runWith(Sink.future)) .fast.recover { case _: NoSuchElementException ⇒ Nil }, 1.second) } def haveFormData(fields: (String, BodyPart)*): Matcher[Future[MultipartFormData]] = equal(fields).matcher[Seq[(String, BodyPart)]] compose { x ⇒ Await.result(x - .fast.flatMap(x ⇒ x.parts.grouped(100).runWith(FutureDrain())) + .fast.flatMap(x ⇒ x.parts.grouped(100).runWith(Sink.future)) .fast.recover { case _: NoSuchElementException ⇒ Nil } .fast.map { _ map { part ⇒ diff --git a/akka-http/src/main/scala/akka/http/server/ScalaRoutingDSL.scala b/akka-http/src/main/scala/akka/http/server/ScalaRoutingDSL.scala index f2dfacbc18..ab31474805 100644 --- a/akka-http/src/main/scala/akka/http/server/ScalaRoutingDSL.scala +++ b/akka-http/src/main/scala/akka/http/server/ScalaRoutingDSL.scala @@ -55,7 +55,7 @@ trait ScalaRoutingDSL extends Directives { val runner = f(setup) Source(requestProducer) .mapAsync(request ⇒ runner(request)) - .connect(SubscriberDrain(responseConsumer)).run()(fm) + .connect(Sink(responseConsumer)).run()(fm) } } } diff --git a/akka-stream-tck/src/test/scala/akka/stream/tck/AkkaIdentityProcessorVerification.scala b/akka-stream-tck/src/test/scala/akka/stream/tck/AkkaIdentityProcessorVerification.scala index b5aeb8dcc8..2e903fec63 100644 --- a/akka-stream-tck/src/test/scala/akka/stream/tck/AkkaIdentityProcessorVerification.scala +++ b/akka-stream-tck/src/test/scala/akka/stream/tck/AkkaIdentityProcessorVerification.scala @@ -6,7 +6,7 @@ package akka.stream.tck import scala.collection.immutable import akka.actor.ActorSystem import akka.stream.scaladsl2.FlowMaterializer -import akka.stream.scaladsl2.PublisherDrain +import akka.stream.scaladsl2.Sink import akka.stream.scaladsl2.Source import akka.stream.testkit.AkkaSpec import akka.stream.testkit.StreamTestKit @@ -48,7 +48,7 @@ abstract class AkkaIdentityProcessorVerification[T](val system: ActorSystem, env if (elements == Long.MaxValue) 1 to Int.MaxValue else 0 until elements.toInt - Source(iterable).runWith(PublisherDrain()) + Source(iterable).runWith(Sink.publisher) } /** By default Akka Publishers do not support Fanout! */ diff --git a/akka-stream-tck/src/test/scala/akka/stream/tck/AkkaSubscriberVerification.scala b/akka-stream-tck/src/test/scala/akka/stream/tck/AkkaSubscriberVerification.scala index a12bce8c48..cda58e13a4 100644 --- a/akka-stream-tck/src/test/scala/akka/stream/tck/AkkaSubscriberVerification.scala +++ b/akka-stream-tck/src/test/scala/akka/stream/tck/AkkaSubscriberVerification.scala @@ -8,7 +8,7 @@ import scala.concurrent.duration._ import akka.actor.ActorSystem import akka.stream.MaterializerSettings import akka.stream.scaladsl2.FlowMaterializer -import akka.stream.scaladsl2.PublisherDrain +import akka.stream.scaladsl2.Sink import akka.stream.scaladsl2.Source import akka.stream.testkit.AkkaSpec import org.reactivestreams.Publisher @@ -62,7 +62,7 @@ trait AkkaSubscriberVerificationLike { if (elements == Long.MaxValue) 1 to Int.MaxValue else 0 until elements.toInt - Source(iterable).runWith(PublisherDrain()) + Source(iterable).runWith(Sink.publisher) } @AfterClass diff --git a/akka-stream-tck/src/test/scala/akka/stream/tck/FanoutPublisherTest.scala b/akka-stream-tck/src/test/scala/akka/stream/tck/FanoutPublisherTest.scala index d2db157d80..af3ff862d1 100644 --- a/akka-stream-tck/src/test/scala/akka/stream/tck/FanoutPublisherTest.scala +++ b/akka-stream-tck/src/test/scala/akka/stream/tck/FanoutPublisherTest.scala @@ -4,7 +4,7 @@ package akka.stream.tck import scala.collection.immutable -import akka.stream.scaladsl2.FanoutPublisherDrain +import akka.stream.scaladsl2.Sink import akka.stream.scaladsl2.Source import org.reactivestreams.Publisher @@ -15,7 +15,7 @@ class FanoutPublisherTest extends AkkaPublisherVerification[Int] { if (elements == 0) new immutable.Iterable[Int] { override def iterator = Iterator from 0 } else 0 until elements.toInt - Source(iterable).runWith(FanoutPublisherDrain(initialBufferSize = 2, maximumBufferSize = 4)) + Source(iterable).runWith(Sink.fanoutPublisher(initialBufferSize = 2, maximumBufferSize = 4)) } } diff --git a/akka-stream-tck/src/test/scala/akka/stream/tck/IterablePublisherTest.scala b/akka-stream-tck/src/test/scala/akka/stream/tck/IterablePublisherTest.scala index 994ff60641..71101dd2d8 100644 --- a/akka-stream-tck/src/test/scala/akka/stream/tck/IterablePublisherTest.scala +++ b/akka-stream-tck/src/test/scala/akka/stream/tck/IterablePublisherTest.scala @@ -4,7 +4,7 @@ package akka.stream.tck import scala.collection.immutable -import akka.stream.scaladsl2.PublisherDrain +import akka.stream.scaladsl2.Sink import akka.stream.scaladsl2.Source import org.reactivestreams._ @@ -17,7 +17,7 @@ class IterablePublisherTest extends AkkaPublisherVerification[Int] { else 0 until elements.toInt - Source(iterable).runWith(PublisherDrain()) + Source(iterable).runWith(Sink.publisher) } } \ No newline at end of file diff --git a/akka-stream-tck/src/test/scala/akka/stream/tck/IteratorPublisherTest.scala b/akka-stream-tck/src/test/scala/akka/stream/tck/IteratorPublisherTest.scala index 2ff8e48a21..184c060c59 100644 --- a/akka-stream-tck/src/test/scala/akka/stream/tck/IteratorPublisherTest.scala +++ b/akka-stream-tck/src/test/scala/akka/stream/tck/IteratorPublisherTest.scala @@ -4,7 +4,7 @@ package akka.stream.tck import scala.collection.immutable -import akka.stream.scaladsl2.PublisherDrain +import akka.stream.scaladsl2.Sink import akka.stream.scaladsl2.Source import org.reactivestreams.Publisher @@ -15,7 +15,7 @@ class IteratorPublisherTest extends AkkaPublisherVerification[Int](true) { if (elements == 0) new immutable.Iterable[Int] { override def iterator = Iterator from 0 } else 0 until elements.toInt - Source(iterable).runWith(PublisherDrain()) + Source(iterable).runWith(Sink.publisher) } } \ No newline at end of file diff --git a/akka-stream-tck/src/test/scala/akka/stream/tck/SimpleCallbackPublisherTest.scala b/akka-stream-tck/src/test/scala/akka/stream/tck/SimpleCallbackPublisherTest.scala index 8d654638fd..4b854373b7 100644 --- a/akka-stream-tck/src/test/scala/akka/stream/tck/SimpleCallbackPublisherTest.scala +++ b/akka-stream-tck/src/test/scala/akka/stream/tck/SimpleCallbackPublisherTest.scala @@ -3,7 +3,7 @@ */ package akka.stream.tck -import akka.stream.scaladsl2.PublisherDrain +import akka.stream.scaladsl2.Sink import akka.stream.scaladsl2.Source import org.reactivestreams._ @@ -12,7 +12,7 @@ class SimpleCallbackPublisherTest extends AkkaPublisherVerification[Int] { def createPublisher(elements: Long): Publisher[Int] = { val iter = Iterator from 0 val iter2 = if (elements > 0) iter take elements.toInt else iter - Source(() ⇒ if (iter2.hasNext) Some(iter2.next()) else None).runWith(PublisherDrain()) + Source(() ⇒ if (iter2.hasNext) Some(iter2.next()) else None).runWith(Sink.publisher) } } \ No newline at end of file diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit2/ScriptedTest.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit2/ScriptedTest.scala index 2a947832c6..c77b0d5d82 100644 --- a/akka-stream-testkit/src/test/scala/akka/stream/testkit2/ScriptedTest.scala +++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit2/ScriptedTest.scala @@ -5,21 +5,20 @@ package akka.stream.testkit2 import akka.actor.ActorSystem import akka.stream.MaterializerSettings -import akka.stream.scaladsl2.{ FlowMaterializer, Source, Flow } +import akka.stream.scaladsl2.{ FlowMaterializer, Sink, Source, Flow } import akka.stream.testkit.StreamTestKit._ import org.reactivestreams.Publisher import org.scalatest.Matchers import scala.annotation.tailrec import scala.concurrent.duration._ import scala.concurrent.forkjoin.ThreadLocalRandom -import akka.stream.scaladsl2.PublisherDrain trait ScriptedTest extends Matchers { class ScriptException(msg: String) extends RuntimeException(msg) def toPublisher[In, Out]: (Source[Out], FlowMaterializer) ⇒ Publisher[Out] = - (f, m) ⇒ f.runWith(PublisherDrain())(m) + (f, m) ⇒ f.runWith(Sink.publisher)(m) object Script { def apply[In, Out](phases: (Seq[In], Seq[Out])*): Script[In, Out] = { diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit2/TwoStreamsSetup.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit2/TwoStreamsSetup.scala index 9c2b316e51..314be53990 100644 --- a/akka-stream-testkit/src/test/scala/akka/stream/testkit2/TwoStreamsSetup.scala +++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit2/TwoStreamsSetup.scala @@ -30,7 +30,7 @@ abstract class TwoStreamsSetup extends AkkaSpec { import FlowGraphImplicits._ val left = operationUnderTestLeft() val right = operationUnderTestRight() - val x = Source(p1) ~> left ~> Flow[Outputs] ~> SubscriberDrain(subscriber) + val x = Source(p1) ~> left ~> Flow[Outputs] ~> Sink(subscriber) Source(p2) ~> right }.run() @@ -41,7 +41,7 @@ abstract class TwoStreamsSetup extends AkkaSpec { def completedPublisher[T]: Publisher[T] = StreamTestKit.emptyPublisher[T] - def nonemptyPublisher[T](elems: Iterator[T]): Publisher[T] = Source(elems).runWith(PublisherDrain()) + def nonemptyPublisher[T](elems: Iterator[T]): Publisher[T] = Source(elems).runWith(Sink.publisher) def soonToFailPublisher[T]: Publisher[T] = StreamTestKit.lazyErrorPublisher[T](TestException) diff --git a/akka-stream-tests/src/test/java/akka/stream/actor/ActorSubscriberTest.java b/akka-stream-tests/src/test/java/akka/stream/actor/ActorSubscriberTest.java index 8de4d8f0f5..c33222c3e4 100644 --- a/akka-stream-tests/src/test/java/akka/stream/actor/ActorSubscriberTest.java +++ b/akka-stream-tests/src/test/java/akka/stream/actor/ActorSubscriberTest.java @@ -5,9 +5,8 @@ import akka.actor.ActorSystem; import akka.actor.Props; import akka.stream.MaterializerSettings; import akka.stream.javadsl.AkkaJUnitActorSystemResource; -import akka.stream.javadsl.Drain; +import akka.stream.javadsl.Sink; import akka.stream.javadsl.Source; -import akka.stream.javadsl.SubscriberDrain; import akka.stream.scaladsl2.FlowMaterializer; import akka.stream.testkit.AkkaSpec; import akka.testkit.JavaTestKit; @@ -70,7 +69,7 @@ public class ActorSubscriberTest { final Subscriber subscriber = UntypedActorSubscriber.create(ref); final java.util.Iterator input = Arrays.asList(1, 2, 3).iterator(); - Source.from(input).runWith(SubscriberDrain.create(subscriber), materializer); + Source.from(input).runWith(Sink.subscriber(subscriber), materializer); ref.tell("run", null); probe.expectMsgEquals(1); diff --git a/akka-stream-tests/src/test/java/akka/stream/javadsl/FlowTest.java b/akka-stream-tests/src/test/java/akka/stream/javadsl/FlowTest.java index 5be1d927c1..d9f7398afe 100644 --- a/akka-stream-tests/src/test/java/akka/stream/javadsl/FlowTest.java +++ b/akka-stream-tests/src/test/java/akka/stream/javadsl/FlowTest.java @@ -18,7 +18,6 @@ import akka.testkit.JavaTestKit; import org.junit.ClassRule; import org.junit.Test; import org.reactivestreams.Publisher; -import scala.Function1; import scala.Option; import scala.collection.immutable.Seq; import scala.concurrent.Await; @@ -26,7 +25,6 @@ import scala.concurrent.Future; import scala.concurrent.duration.Duration; import scala.concurrent.duration.FiniteDuration; import scala.runtime.BoxedUnit; -import scala.util.Success; import scala.util.Try; import java.util.*; @@ -317,10 +315,10 @@ public class FlowTest { final Flow f2 = Flow.of(String.class).transform("f2", this.op()); // javadsl final Flow f3 = Flow.of(String.class).transform("f2", this.op()); // javadsl - final IterableTap in1 = IterableTap.create(Arrays.asList("a", "b", "c")); - final IterableTap in2 = IterableTap.create(Arrays.asList("d", "e", "f")); + final Source in1 = Source.from(Arrays.asList("a", "b", "c")); + final Source in2 = Source.from(Arrays.asList("d", "e", "f")); - final PublisherDrain publisher = PublisherDrain.create(); + final KeyedSink> publisher = Sink.publisher(); // this is red in intellij, but actually valid, scalac generates bridge methods for Java, so inference *works* final Merge merge = Merge.create(); @@ -333,8 +331,8 @@ public class FlowTest { run(materializer); // collecting - final Publisher pub = m.materializedDrain(publisher); - final Future> all = Source.from(pub).grouped(100).runWith(FutureDrain.>create(), materializer); + final Publisher pub = m.get(publisher); + final Future> all = Source.from(pub).grouped(100).runWith(Sink.>future(), materializer); final List result = Await.result(all, Duration.apply(200, TimeUnit.MILLISECONDS)); assertEquals(new HashSet(Arrays.asList("a", "b", "c", "d", "e", "f")), new HashSet(result)); @@ -411,7 +409,7 @@ public class FlowTest { final java.lang.Iterable input = Arrays.asList("A", "B", "C"); Source.from(input) - .runWith(OnCompleteDrain.create( + .runWith(Sink.onComplete( new OnComplete() { @Override public void onComplete(Throwable failure, BoxedUnit success) throws Throwable { probe.getRef().tell(success, ActorRef.noSender()); @@ -431,7 +429,7 @@ public class FlowTest { public String apply(String arg0) throws Exception { throw new RuntimeException("simulated err"); } - }).runWith(FutureDrain.create(), materializer) + }).runWith(Sink.future(), materializer) .onComplete(new OnSuccess>() { @Override public void onSuccess(Try e) throws Throwable { if (e == null) { @@ -449,7 +447,7 @@ public class FlowTest { public void mustBeAbleToUseToFuture() throws Exception { final JavaTestKit probe = new JavaTestKit(system); final java.lang.Iterable input = Arrays.asList("A", "B", "C"); - Future future = Source.from(input).runWith(FutureDrain.create(), materializer); + Future future = Source.from(input).runWith(Sink.future(), materializer); String result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS))); assertEquals("A", result); } @@ -461,11 +459,11 @@ public class FlowTest { Future, Source>> future = Source .from(input) .prefixAndTail(3) - .runWith(FutureDrain., Source>>create(), materializer); + .runWith(Sink., Source>>future(), materializer); Pair, Source> result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS))); assertEquals(Arrays.asList(1, 2, 3), result.first()); - Future> tailFuture = result.second().grouped(4).runWith(FutureDrain.>create(), materializer); + Future> tailFuture = result.second().grouped(4).runWith(Sink.>future(), materializer); List tailResult = Await.result(tailFuture, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS))); assertEquals(Arrays.asList(4, 5, 6), tailResult); } @@ -484,7 +482,7 @@ public class FlowTest { .from(mainInputs) .flatten(akka.stream.javadsl.FlattenStrategy.concat()) .grouped(6) - .runWith(FutureDrain.>create(), materializer); + .runWith(Sink.>future(), materializer); List result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS))); @@ -497,7 +495,7 @@ public class FlowTest { final List input = Arrays.asList("A", "B", "C"); Future> future = Source.from(input) .buffer(2, OverflowStrategy.backpressure()).grouped(4) - .runWith(FutureDrain.>create(), materializer); + .runWith(Sink.>future(), materializer); List result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS))); assertEquals(input, result); @@ -543,7 +541,7 @@ public class FlowTest { public Pair apply(String in) throws Exception { return new Pair(in, in); } - }).runWith(FutureDrain.create(), materializer); + }).runWith(Sink.future(), materializer); String result = Await.result(future, probe.dilated(FiniteDuration.create(3, TimeUnit.SECONDS))); assertEquals("A", result); } diff --git a/akka-stream-tests/src/test/scala/akka/stream/DslConsistencySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/DslConsistencySpec.scala index 61ab400689..ef03eabcb5 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/DslConsistencySpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/DslConsistencySpec.scala @@ -4,6 +4,7 @@ package akka.stream import java.lang.reflect.Method +import java.lang.reflect.Modifier import org.scalatest.Matchers import org.scalatest.WordSpec @@ -61,6 +62,7 @@ class DslConsistencySpec extends WordSpec with Matchers { (for { c ← classes m ← c.getMethods + if !Modifier.isStatic(m.getModifiers) if !ignore(m.getName) if !m.getName.contains("$") if !materializing(m) @@ -75,6 +77,7 @@ class DslConsistencySpec extends WordSpec with Matchers { (for { c ← classes m ← c.getMethods + if !Modifier.isStatic(m.getModifiers) if !ignore(m.getName) if !m.getName.contains("$") if materializing(m) diff --git a/akka-stream-tests/src/test/scala/akka/stream/io2/TcpFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io2/TcpFlowSpec.scala index b483148d8f..10a56d2ecd 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io2/TcpFlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io2/TcpFlowSpec.scala @@ -47,9 +47,9 @@ class TcpFlowSpec extends AkkaSpec with TcpHelper { val expectedOutput = ByteString(Array.tabulate(256)(_.asInstanceOf[Byte])) serverConnection.read(256) - Source(tcpPublisher).connect(BlackholeDrain).run() + Source(tcpPublisher).connect(Sink.ignore).run() - Source(testInput).connect(SubscriberDrain(tcpSubscriber)).run() + Source(testInput).connect(Sink(tcpSubscriber)).run() serverConnection.waitRead() should be(expectedOutput) server.close() @@ -161,7 +161,7 @@ class TcpFlowSpec extends AkkaSpec with TcpHelper { val testInput = Iterator.range(0, 256).map(ByteString(_)) val expectedOutput = ByteString(Array.tabulate(256)(_.asInstanceOf[Byte])) - Source(testInput).connect(SubscriberDrain(tcpSubscriber)).run() + Source(testInput).connect(Sink(tcpSubscriber)).run() val resultFuture = Source(tcpPublisher).fold(ByteString.empty) { case (res, elem) ⇒ res ++ elem } Await.result(resultFuture, 3.seconds) should be(expectedOutput) @@ -180,7 +180,7 @@ class TcpFlowSpec extends AkkaSpec with TcpHelper { val testInput = Iterator.range(0, 256).map(ByteString(_)) val expectedOutput = ByteString(Array.tabulate(256)(_.asInstanceOf[Byte])) - Source(testInput).connect(SubscriberDrain(tcpSubscriber1)).run() + Source(testInput).connect(Sink(tcpSubscriber1)).run() tcpPublisher1.subscribe(tcpSubscriber2) tcpPublisher2.subscribe(tcpSubscriber3) val resultFuture = Source(tcpPublisher3).fold(ByteString.empty) { case (res, elem) ⇒ res ++ elem } diff --git a/akka-stream-tests/src/test/scala/akka/stream/io2/TcpHelper.scala b/akka-stream-tests/src/test/scala/akka/stream/io2/TcpHelper.scala index dabaa543d7..ba1b3a6e99 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io2/TcpHelper.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io2/TcpHelper.scala @@ -201,13 +201,13 @@ trait TcpHelper { this: TestKitBase ⇒ def connect(serverAddress: InetSocketAddress): (Subscriber[ByteString], Publisher[ByteString]) = { val tcpProbe = TestProbe() - val outbound = SubscriberTap[ByteString] - val inbound = PublisherDrain[ByteString] + val outbound = Source.subscriber[ByteString] + val inbound = Sink.publisher[ByteString] tcpProbe.send(IO(StreamTcp), StreamTcp.Connect(outbound, inbound, serverAddress)) val outgoingConnection = tcpProbe.expectMsgType[StreamTcp.OutgoingTcpConnection] - (outgoingConnection.outbound.materializedTap(outbound), outgoingConnection.inbound.materializedDrain(inbound)) + (outgoingConnection.outbound.get(outbound), outgoingConnection.inbound.get(inbound)) } def bind(connectionHandler: Sink[StreamTcp.IncomingTcpConnection], @@ -218,10 +218,10 @@ trait TcpHelper { this: TestKitBase ⇒ } def echoServer(serverAddress: InetSocketAddress = temporaryServerAddress): EchoServer = { - val foreachDrain = ForeachDrain[IncomingTcpConnection] { conn ⇒ + val foreachSink = Sink.foreach[IncomingTcpConnection] { conn ⇒ conn.inbound.connect(conn.outbound).run() } - val binding = bind(Flow[IncomingTcpConnection].connect(foreachDrain), serverAddress) - new EchoServer(binding.connection.materializedDrain(foreachDrain), binding) + val binding = bind(Flow[IncomingTcpConnection].connect(foreachSink), serverAddress) + new EchoServer(binding.connection.get(foreachSink), binding) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowAppendSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowAppendSpec.scala index bc3e8f4546..322f6aa116 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowAppendSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowAppendSpec.scala @@ -17,11 +17,11 @@ class FlowAppendSpec extends AkkaSpec with River { "Flow" should { "append Flow" in riverOf[String] { subscriber ⇒ val flow = Flow[Int].connect(otherFlow) - Source(elements).connect(flow).connect(SubscriberDrain(subscriber)).run() + Source(elements).connect(flow).connect(Sink(subscriber)).run() } "append Sink" in riverOf[String] { subscriber ⇒ - val sink = Flow[Int].connect(otherFlow.connect(SubscriberDrain(subscriber))) + val sink = Flow[Int].connect(otherFlow.connect(Sink(subscriber))) Source(elements).connect(sink).run() } } @@ -30,12 +30,12 @@ class FlowAppendSpec extends AkkaSpec with River { "append Flow" in riverOf[String] { subscriber ⇒ Source(elements) .connect(otherFlow) - .connect(SubscriberDrain(subscriber)).run() + .connect(Sink(subscriber)).run() } "append Sink" in riverOf[String] { subscriber ⇒ Source(elements) - .connect(otherFlow.connect(SubscriberDrain(subscriber))) + .connect(otherFlow.connect(Sink(subscriber))) .run() } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowBufferSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowBufferSpec.scala index dfc928e630..11e4290940 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowBufferSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowBufferSpec.scala @@ -22,14 +22,14 @@ class FlowBufferSpec extends AkkaSpec { "pass elements through normally in backpressured mode" in { val future: Future[Seq[Int]] = Source((1 to 1000).iterator).buffer(100, overflowStrategy = OverflowStrategy.backpressure).grouped(1001). - runWith(FutureDrain()) + runWith(Sink.future) Await.result(future, 3.seconds) should be(1 to 1000) } "pass elements through normally in backpressured mode with buffer size one" in { - val futureDrain = FutureDrain[Seq[Int]] + val futureSink = Sink.future[Seq[Int]] val future = Source((1 to 1000).iterator).buffer(1, overflowStrategy = OverflowStrategy.backpressure).grouped(1001). - runWith(FutureDrain()) + runWith(Sink.future) Await.result(future, 3.seconds) should be(1 to 1000) } @@ -42,7 +42,7 @@ class FlowBufferSpec extends AkkaSpec { .buffer(5, overflowStrategy = OverflowStrategy.backpressure) .buffer(128, overflowStrategy = OverflowStrategy.backpressure) .grouped(1001) - .runWith(FutureDrain()) + .runWith(Sink.future) Await.result(future, 3.seconds) should be(1 to 1000) } @@ -50,7 +50,7 @@ class FlowBufferSpec extends AkkaSpec { val publisher = StreamTestKit.PublisherProbe[Int]() val subscriber = StreamTestKit.SubscriberProbe[Int]() - Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.backpressure).connect(SubscriberDrain(subscriber)).run() + Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.backpressure).connect(Sink(subscriber)).run() val autoPublisher = new StreamTestKit.AutoPublisher(publisher) val sub = subscriber.expectSubscription() @@ -70,7 +70,7 @@ class FlowBufferSpec extends AkkaSpec { val publisher = StreamTestKit.PublisherProbe[Int]() val subscriber = StreamTestKit.SubscriberProbe[Int]() - Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropHead).connect(SubscriberDrain(subscriber)).run() + Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropHead).connect(Sink(subscriber)).run() val autoPublisher = new StreamTestKit.AutoPublisher(publisher) val sub = subscriber.expectSubscription() @@ -98,7 +98,7 @@ class FlowBufferSpec extends AkkaSpec { val publisher = StreamTestKit.PublisherProbe[Int]() val subscriber = StreamTestKit.SubscriberProbe[Int]() - Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropTail).connect(SubscriberDrain(subscriber)).run() + Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropTail).connect(Sink(subscriber)).run() val autoPublisher = new StreamTestKit.AutoPublisher(publisher) val sub = subscriber.expectSubscription() @@ -129,7 +129,7 @@ class FlowBufferSpec extends AkkaSpec { val publisher = StreamTestKit.PublisherProbe[Int] val subscriber = StreamTestKit.SubscriberProbe[Int]() - Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropBuffer).connect(SubscriberDrain(subscriber)).run() + Source(publisher).buffer(100, overflowStrategy = OverflowStrategy.dropBuffer).connect(Sink(subscriber)).run() val autoPublisher = new StreamTestKit.AutoPublisher(publisher) val sub = subscriber.expectSubscription() @@ -160,7 +160,7 @@ class FlowBufferSpec extends AkkaSpec { val publisher = StreamTestKit.PublisherProbe[Int] val subscriber = StreamTestKit.SubscriberProbe[Int]() - Source(publisher).buffer(1, overflowStrategy = strategy).connect(SubscriberDrain(subscriber)).run() + Source(publisher).buffer(1, overflowStrategy = strategy).connect(Sink(subscriber)).run() val autoPublisher = new StreamTestKit.AutoPublisher(publisher) val sub = subscriber.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowCompileSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowCompileSpec.scala index 52c10258e4..84df7d4fb0 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowCompileSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowCompileSpec.scala @@ -10,11 +10,11 @@ import scala.concurrent.Future class FlowCompileSpec extends AkkaSpec { - val intSeq = IterableTap(Seq(1, 2, 3)) - val strSeq = IterableTap(Seq("a", "b", "c")) + val intSeq = Source(Seq(1, 2, 3)) + val strSeq = Source(Seq("a", "b", "c")) import scala.concurrent.ExecutionContext.Implicits.global - val intFut = FutureTap(Future { 3 }) + val intFut = Source(Future { 3 }) implicit val materializer = FlowMaterializer(MaterializerSettings(system)) "Flow" should { @@ -37,38 +37,38 @@ class FlowCompileSpec extends AkkaSpec { val closedSource: Source[Int] = intSeq.connect(open3) "closedSource.run()" shouldNot compile - val closedSink: Sink[Int] = open3.connect(PublisherDrain[Int]) + val closedSink: Sink[Int] = open3.connect(Sink.publisher[Int]) "closedSink.run()" shouldNot compile - closedSource.connect(PublisherDrain[Int]).run() + closedSource.connect(Sink.publisher[Int]).run() intSeq.connect(closedSink).run() } "append Sink" in { val open: Flow[Int, String] = Flow[Int].map(_.toString) - val closedDrain: Sink[String] = Flow[String].map(_.hashCode).connect(PublisherDrain[Int]) - val appended: Sink[Int] = open.connect(closedDrain) + val closedSink: Sink[String] = Flow[String].map(_.hashCode).connect(Sink.publisher[Int]) + val appended: Sink[Int] = open.connect(closedSink) "appended.run()" shouldNot compile - "appended.connect(FutureDrain[Int])" shouldNot compile + "appended.connect(Sink.future[Int])" shouldNot compile intSeq.connect(appended).run } "be appended to Source" in { val open: Flow[Int, String] = Flow[Int].map(_.toString) - val closedTap: Source[Int] = strSeq.connect(Flow[String].map(_.hashCode)) - val closedSource: Source[String] = closedTap.connect(open) - "closedSource.run()" shouldNot compile - "strSeq.connect(closedSource)" shouldNot compile - closedSource.connect(PublisherDrain[String]).run + val closedSource: Source[Int] = strSeq.connect(Flow[String].map(_.hashCode)) + val closedSource2: Source[String] = closedSource.connect(open) + "closedSource2.run()" shouldNot compile + "strSeq.connect(closedSource2)" shouldNot compile + closedSource2.connect(Sink.publisher[String]).run } } "Sink" should { val openSource: Sink[Int] = - Flow[Int].map(_.toString).connect(PublisherDrain[String]) + Flow[Int].map(_.toString).connect(Sink.publisher[String]) "accept Source" in { intSeq.connect(openSource) } "not accept Sink" in { - "openSource.connect(FutureDrain[String])" shouldNot compile + "openSource.connect(Sink.future[String])" shouldNot compile } "not run()" in { "openSource.run()" shouldNot compile @@ -79,7 +79,7 @@ class FlowCompileSpec extends AkkaSpec { val openSource: Source[String] = Source(Seq(1, 2, 3)).map(_.toString) "accept Sink" in { - openSource.connect(PublisherDrain[String]) + openSource.connect(Sink.publisher[String]) } "not be accepted by Source" in { "openSource.connect(intSeq)" shouldNot compile @@ -90,9 +90,9 @@ class FlowCompileSpec extends AkkaSpec { } "RunnableFlow" should { - FutureDrain[String] + Sink.future[String] val closed: RunnableFlow = - Source(Seq(1, 2, 3)).map(_.toString).connect(PublisherDrain[String]) + Source(Seq(1, 2, 3)).map(_.toString).connect(Sink.publisher[String]) "run" in { closed.run() } @@ -101,7 +101,7 @@ class FlowCompileSpec extends AkkaSpec { } "not accept Sink" in { - "closed.connect(FutureDrain[String])" shouldNot compile + "closed.connect(Sink.future[String])" shouldNot compile } } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowConcatAllSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowConcatAllSpec.scala index 5ede7d6b07..a4beb51b29 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowConcatAllSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowConcatAllSpec.scala @@ -32,7 +32,7 @@ class FlowConcatAllSpec extends AkkaSpec { val main = Source(List(s1, s2, s3, s4, s5)) val subscriber = StreamTestKit.SubscriberProbe[Int]() - main.flatten(FlattenStrategy.concat).connect(SubscriberDrain(subscriber)).run() + main.flatten(FlattenStrategy.concat).connect(Sink(subscriber)).run() val subscription = subscriber.expectSubscription() subscription.request(10) subscriber.probe.receiveN(10) should be((1 to 10).map(StreamTestKit.OnNext(_))) @@ -42,7 +42,7 @@ class FlowConcatAllSpec extends AkkaSpec { "work together with SplitWhen" in { val subscriber = StreamTestKit.SubscriberProbe[Int]() - Source((1 to 10).iterator).splitWhen(_ % 2 == 0).flatten(FlattenStrategy.concat).connect(SubscriberDrain(subscriber)).run() + Source((1 to 10).iterator).splitWhen(_ % 2 == 0).flatten(FlattenStrategy.concat).connect(Sink(subscriber)).run() val subscription = subscriber.expectSubscription() subscription.request(10) subscriber.probe.receiveN(10) should be((1 to 10).map(StreamTestKit.OnNext(_))) @@ -53,7 +53,7 @@ class FlowConcatAllSpec extends AkkaSpec { "on onError on master stream cancel the current open substream and signal error" in { val publisher = StreamTestKit.PublisherProbe[Source[Int]]() val subscriber = StreamTestKit.SubscriberProbe[Int]() - Source(publisher).flatten(FlattenStrategy.concat).connect(SubscriberDrain(subscriber)).run() + Source(publisher).flatten(FlattenStrategy.concat).connect(Sink(subscriber)).run() val upstream = publisher.expectSubscription() val downstream = subscriber.expectSubscription() @@ -73,7 +73,7 @@ class FlowConcatAllSpec extends AkkaSpec { "on onError on open substream, cancel the master stream and signal error " in { val publisher = StreamTestKit.PublisherProbe[Source[Int]]() val subscriber = StreamTestKit.SubscriberProbe[Int]() - Source(publisher).flatten(FlattenStrategy.concat).connect(SubscriberDrain(subscriber)).run() + Source(publisher).flatten(FlattenStrategy.concat).connect(Sink(subscriber)).run() val upstream = publisher.expectSubscription() val downstream = subscriber.expectSubscription() @@ -93,7 +93,7 @@ class FlowConcatAllSpec extends AkkaSpec { "on cancellation cancel the current open substream and the master stream" in { val publisher = StreamTestKit.PublisherProbe[Source[Int]]() val subscriber = StreamTestKit.SubscriberProbe[Int]() - Source(publisher).flatten(FlattenStrategy.concat).connect(SubscriberDrain(subscriber)).run() + Source(publisher).flatten(FlattenStrategy.concat).connect(Sink(subscriber)).run() val upstream = publisher.expectSubscription() val downstream = subscriber.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowConflateSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowConflateSpec.scala index ca86176ba1..46b82ce734 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowConflateSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowConflateSpec.scala @@ -23,7 +23,7 @@ class FlowConflateSpec extends AkkaSpec { val publisher = StreamTestKit.PublisherProbe[Int]() val subscriber = StreamTestKit.SubscriberProbe[Int]() - Source(publisher).conflate[Int](seed = i ⇒ i, aggregate = (sum, i) ⇒ sum + i).connect(SubscriberDrain(subscriber)).run() + Source(publisher).conflate[Int](seed = i ⇒ i, aggregate = (sum, i) ⇒ sum + i).connect(Sink(subscriber)).run() val autoPublisher = new StreamTestKit.AutoPublisher(publisher) val sub = subscriber.expectSubscription() @@ -41,7 +41,7 @@ class FlowConflateSpec extends AkkaSpec { val publisher = StreamTestKit.PublisherProbe[Int]() val subscriber = StreamTestKit.SubscriberProbe[Int]() - Source(publisher).conflate[Int](seed = i ⇒ i, aggregate = (sum, i) ⇒ sum + i).connect(SubscriberDrain(subscriber)).run() + Source(publisher).conflate[Int](seed = i ⇒ i, aggregate = (sum, i) ⇒ sum + i).connect(Sink(subscriber)).run() val autoPublisher = new StreamTestKit.AutoPublisher(publisher) val sub = subscriber.expectSubscription() @@ -67,7 +67,7 @@ class FlowConflateSpec extends AkkaSpec { val publisher = StreamTestKit.PublisherProbe[Int]() val subscriber = StreamTestKit.SubscriberProbe[Int]() - Source(publisher).conflate[Int](seed = i ⇒ i, aggregate = (sum, i) ⇒ sum + i).connect(SubscriberDrain(subscriber)).run() + Source(publisher).conflate[Int](seed = i ⇒ i, aggregate = (sum, i) ⇒ sum + i).connect(Sink(subscriber)).run() val autoPublisher = new StreamTestKit.AutoPublisher(publisher) val sub = subscriber.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowDispatcherSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowDispatcherSpec.scala index 8f561e9b72..aeccc75bdc 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowDispatcherSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowDispatcherSpec.scala @@ -16,7 +16,7 @@ class FlowDispatcherSpec extends AkkaSpec { val probe = TestProbe() val p = Source(List(1, 2, 3)).map(i ⇒ { probe.ref ! Thread.currentThread().getName(); i }). - connect(BlackholeDrain).run() + connect(Sink.ignore).run() probe.receiveN(3) foreach { case s: String ⇒ s should startWith(system.name + "-akka.test.stream-dispatcher") } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowDropSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowDropSpec.scala index 8a71742310..c56d68b47f 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowDropSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowDropSpec.scala @@ -29,7 +29,7 @@ class FlowDropSpec extends AkkaSpec with ScriptedTest { "not drop anything for negative n" in { val probe = StreamTestKit.SubscriberProbe[Int]() - Source(List(1, 2, 3)).drop(-1).connect(SubscriberDrain(probe)).run() + Source(List(1, 2, 3)).drop(-1).connect(Sink(probe)).run() probe.expectSubscription().request(10) probe.expectNext(1) probe.expectNext(2) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowDropWithinSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowDropWithinSpec.scala index 1902e3adad..a157d0f429 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowDropWithinSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowDropWithinSpec.scala @@ -18,7 +18,7 @@ class FlowDropWithinSpec extends AkkaSpec { val input = Iterator.from(1) val p = StreamTestKit.PublisherProbe[Int]() val c = StreamTestKit.SubscriberProbe[Int]() - Source(p).dropWithin(1.second).connect(SubscriberDrain(c)).run() + Source(p).dropWithin(1.second).connect(Sink(c)).run() val pSub = p.expectSubscription val cSub = c.expectSubscription cSub.request(100) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowExpandSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowExpandSpec.scala index 8635e5819c..c4eb2f46e3 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowExpandSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowExpandSpec.scala @@ -24,7 +24,7 @@ class FlowExpandSpec extends AkkaSpec { val subscriber = StreamTestKit.SubscriberProbe[Int]() // Simply repeat the last element as an extrapolation step - Source(publisher).expand[Int, Int](seed = i ⇒ i, extrapolate = i ⇒ (i, i)).connect(SubscriberDrain(subscriber)).run() + Source(publisher).expand[Int, Int](seed = i ⇒ i, extrapolate = i ⇒ (i, i)).connect(Sink(subscriber)).run() val autoPublisher = new StreamTestKit.AutoPublisher(publisher) val sub = subscriber.expectSubscription() @@ -44,7 +44,7 @@ class FlowExpandSpec extends AkkaSpec { val subscriber = StreamTestKit.SubscriberProbe[Int]() // Simply repeat the last element as an extrapolation step - Source(publisher).expand[Int, Int](seed = i ⇒ i, extrapolate = i ⇒ (i, i)).connect(SubscriberDrain(subscriber)).run() + Source(publisher).expand[Int, Int](seed = i ⇒ i, extrapolate = i ⇒ (i, i)).connect(Sink(subscriber)).run() val autoPublisher = new StreamTestKit.AutoPublisher(publisher) val sub = subscriber.expectSubscription() @@ -76,7 +76,7 @@ class FlowExpandSpec extends AkkaSpec { val publisher = StreamTestKit.PublisherProbe[Int]() val subscriber = StreamTestKit.SubscriberProbe[Int]() - Source(publisher).expand[Int, Int](seed = i ⇒ i, extrapolate = i ⇒ (i, i)).connect(SubscriberDrain(subscriber)).run() + Source(publisher).expand[Int, Int](seed = i ⇒ i, extrapolate = i ⇒ (i, i)).connect(Sink(subscriber)).run() val autoPublisher = new StreamTestKit.AutoPublisher(publisher) val sub = subscriber.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowFilterSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowFilterSpec.scala index bebe4baa90..22c6a6f9b1 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowFilterSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowFilterSpec.scala @@ -30,7 +30,7 @@ class FlowFilterSpec extends AkkaSpec with ScriptedTest { val probe = StreamTestKit.SubscriberProbe[Int]() Source(Iterator.fill(1000)(0) ++ List(1)).filter(_ != 0). - connect(SubscriberDrain(probe)).run() + connect(Sink(probe)).run() val subscription = probe.expectSubscription() for (_ ← 1 to 10000) { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowFromFutureSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowFromFutureSpec.scala index efde0635ff..3043af52be 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowFromFutureSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowFromFutureSpec.scala @@ -17,7 +17,7 @@ class FlowFromFutureSpec extends AkkaSpec { "A Flow based on a Future" must { "produce one element from already successful Future" in { - val p = Source(Future.successful(1)).runWith(PublisherDrain()) + val p = Source(Future.successful(1)).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() @@ -29,7 +29,7 @@ class FlowFromFutureSpec extends AkkaSpec { "produce error from already failed Future" in { val ex = new RuntimeException("test") with NoStackTrace - val p = Source(Future.failed[Int](ex)).runWith(PublisherDrain()) + val p = Source(Future.failed[Int](ex)).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) c.expectError(ex) @@ -37,7 +37,7 @@ class FlowFromFutureSpec extends AkkaSpec { "produce one element when Future is completed" in { val promise = Promise[Int]() - val p = Source(promise.future).runWith(PublisherDrain()) + val p = Source(promise.future).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() @@ -51,7 +51,7 @@ class FlowFromFutureSpec extends AkkaSpec { "produce one element when Future is completed but not before request" in { val promise = Promise[Int]() - val p = Source(promise.future).runWith(PublisherDrain()) + val p = Source(promise.future).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() @@ -64,7 +64,7 @@ class FlowFromFutureSpec extends AkkaSpec { "produce elements with multiple subscribers" in { val promise = Promise[Int]() - val p = Source(promise.future).runWith(PublisherDrain()) + val p = Source(promise.future).runWith(Sink.publisher) val c1 = StreamTestKit.SubscriberProbe[Int]() val c2 = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c1) @@ -82,7 +82,7 @@ class FlowFromFutureSpec extends AkkaSpec { "produce elements to later subscriber" in { val promise = Promise[Int]() - val p = Source(promise.future).runWith(PublisherDrain()) + val p = Source(promise.future).runWith(Sink.publisher) val keepAlive = StreamTestKit.SubscriberProbe[Int]() val c1 = StreamTestKit.SubscriberProbe[Int]() val c2 = StreamTestKit.SubscriberProbe[Int]() @@ -103,7 +103,7 @@ class FlowFromFutureSpec extends AkkaSpec { "allow cancel before receiving element" in { val promise = Promise[Int]() - val p = Source(promise.future).runWith(PublisherDrain()) + val p = Source(promise.future).runWith(Sink.publisher) val keepAlive = StreamTestKit.SubscriberProbe[Int]() val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(keepAlive) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowGraphCompileSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowGraphCompileSpec.scala index 787e64f656..42c01f7ef2 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowGraphCompileSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowGraphCompileSpec.scala @@ -30,10 +30,10 @@ class FlowGraphCompileSpec extends AkkaSpec { val f5 = Flow[String].transform("f5", op[String, String]) val f6 = Flow[String].transform("f6", op[String, String]) - val in1 = IterableTap(List("a", "b", "c")) - val in2 = IterableTap(List("d", "e", "f")) - val out1 = PublisherDrain[String] - val out2 = FutureDrain[String] + val in1 = Source(List("a", "b", "c")) + val in2 = Source(List("d", "e", "f")) + val out1 = Sink.publisher[String] + val out2 = Sink.future[String] "FlowGraph" should { "build simple merge" in { @@ -153,12 +153,12 @@ class FlowGraphCompileSpec extends AkkaSpec { val m9 = Merge[String] val m10 = Merge[String] val m11 = Merge[String] - val in3 = IterableTap(List("b")) - val in5 = IterableTap(List("b")) - val in7 = IterableTap(List("a")) - val out2 = PublisherDrain[String] - val out9 = PublisherDrain[String] - val out10 = PublisherDrain[String] + val in3 = Source(List("b")) + val in5 = Source(List("b")) + val in7 = Source(List("a")) + val out2 = Sink.publisher[String] + val out9 = Sink.publisher[String] + val out10 = Sink.publisher[String] def f(s: String) = Flow[String].transform(s, op[String, String]) import FlowGraphImplicits._ @@ -188,7 +188,7 @@ class FlowGraphCompileSpec extends AkkaSpec { b.attachSink(undefinedSink1, out1) }.run() - mg.materializedDrain(out1) should not be (null) + mg.get(out1) should not be (null) } "build partial flow graphs" in { @@ -210,19 +210,19 @@ class FlowGraphCompileSpec extends AkkaSpec { import FlowGraphImplicits._ b.attachSource(undefinedSource1, in1) b.attachSource(undefinedSource2, in2) - bcast ~> f5 ~> UndefinedSink[String]("drain2") + bcast ~> f5 ~> UndefinedSink[String]("sink2") } partial2.undefinedSources should be(Set.empty) - partial2.undefinedSinks should be(Set(undefinedSink1, UndefinedSink[String]("drain2"))) + partial2.undefinedSinks should be(Set(undefinedSink1, UndefinedSink[String]("sink2"))) FlowGraph(partial2) { b ⇒ b.attachSink(undefinedSink1, out1) - b.attachSink(UndefinedSink[String]("drain2"), out2) + b.attachSink(UndefinedSink[String]("sink2"), out2) }.run() FlowGraph(partial2) { b ⇒ b.attachSink(undefinedSink1, f1.connect(out1)) - b.attachSink(UndefinedSink[String]("drain2"), f2.connect(out2)) + b.attachSink(UndefinedSink[String]("sink2"), f2.connect(out2)) }.run() FlowGraph(partial1) { implicit b ⇒ @@ -248,7 +248,7 @@ class FlowGraphCompileSpec extends AkkaSpec { "chain input and output ports" in { FlowGraph { implicit b ⇒ val zip = Zip[Int, String] - val out = PublisherDrain[(Int, String)] + val out = Sink.publisher[(Int, String)] import FlowGraphImplicits._ Source(List(1, 2, 3)) ~> zip.left ~> out Source(List("a", "b", "c")) ~> zip.right @@ -259,7 +259,7 @@ class FlowGraphCompileSpec extends AkkaSpec { FlowGraph { implicit b ⇒ val zip = Zip[Int, String] val unzip = Unzip[Int, String] - val out = PublisherDrain[(Int, String)] + val out = Sink.publisher[(Int, String)] import FlowGraphImplicits._ Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in unzip.left ~> Flow[Int].map(_ * 2) ~> zip.left @@ -273,8 +273,8 @@ class FlowGraphCompileSpec extends AkkaSpec { FlowGraph { implicit b ⇒ val zip = Zip[Int, String] val unzip = Unzip[Int, String] - val wrongOut = PublisherDrain[(Int, Int)] - val whatever = PublisherDrain[Any] + val wrongOut = Sink.publisher[(Int, Int)] + val whatever = Sink.publisher[Any] "Flow(List(1, 2, 3)) ~> zip.left ~> wrongOut" shouldNot compile """Flow(List("a", "b", "c")) ~> zip.left""" shouldNot compile """Flow(List("a", "b", "c")) ~> zip.out""" shouldNot compile @@ -308,7 +308,7 @@ class FlowGraphCompileSpec extends AkkaSpec { } "build with variance" in { - val out = SubscriberDrain(SubscriberProbe[Fruit]()) + val out = Sink(SubscriberProbe[Fruit]()) FlowGraph { b ⇒ val merge = Merge[Fruit] b. @@ -320,13 +320,13 @@ class FlowGraphCompileSpec extends AkkaSpec { "build with implicits and variance" in { PartialFlowGraph { implicit b ⇒ - val inA = PublisherTap(PublisherProbe[Fruit]()) - val inB = PublisherTap(PublisherProbe[Apple]()) - val outA = SubscriberDrain(SubscriberProbe[Fruit]()) - val outB = SubscriberDrain(SubscriberProbe[Fruit]()) + val inA = Source(PublisherProbe[Fruit]()) + val inB = Source(PublisherProbe[Apple]()) + val outA = Sink(SubscriberProbe[Fruit]()) + val outB = Sink(SubscriberProbe[Fruit]()) val merge = Merge[Fruit] val unzip = Unzip[Int, String] - val whatever = PublisherDrain[Any] + val whatever = Sink.publisher[Any] import FlowGraphImplicits._ Source[Fruit](() ⇒ Some(new Apple)) ~> merge Source[Apple](() ⇒ Some(new Apple)) ~> merge diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowGroupBySpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowGroupBySpec.scala index 5e0b6c9870..0dd6570471 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowGroupBySpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowGroupBySpec.scala @@ -32,8 +32,8 @@ class FlowGroupBySpec extends AkkaSpec { } class SubstreamsSupport(groupCount: Int = 2, elementCount: Int = 6) { - val tap = Source((1 to elementCount).iterator).runWith(PublisherDrain()) - val groupStream = Source(tap).groupBy(_ % groupCount).runWith(PublisherDrain()) + val source = Source((1 to elementCount).iterator).runWith(Sink.publisher) + val groupStream = Source(source).groupBy(_ % groupCount).runWith(Sink.publisher) val masterSubscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int])]() groupStream.subscribe(masterSubscriber) @@ -56,7 +56,7 @@ class FlowGroupBySpec extends AkkaSpec { "groupBy" must { "work in the happy case" in new SubstreamsSupport(groupCount = 2) { - val s1 = StreamPuppet(getSubFlow(1).runWith(PublisherDrain())) + val s1 = StreamPuppet(getSubFlow(1).runWith(Sink.publisher)) masterSubscriber.expectNoMsg(100.millis) s1.expectNoMsg(100.millis) @@ -64,7 +64,7 @@ class FlowGroupBySpec extends AkkaSpec { s1.expectNext(1) s1.expectNoMsg(100.millis) - val s2 = StreamPuppet(getSubFlow(0).runWith(PublisherDrain())) + val s2 = StreamPuppet(getSubFlow(0).runWith(Sink.publisher)) s2.expectNoMsg(100.millis) s2.request(2) @@ -92,9 +92,9 @@ class FlowGroupBySpec extends AkkaSpec { } "accept cancellation of substreams" in new SubstreamsSupport(groupCount = 2) { - StreamPuppet(getSubFlow(1).runWith(PublisherDrain())).cancel() + StreamPuppet(getSubFlow(1).runWith(Sink.publisher)).cancel() - val substream = StreamPuppet(getSubFlow(0).runWith(PublisherDrain())) + val substream = StreamPuppet(getSubFlow(0).runWith(Sink.publisher)) substream.request(2) substream.expectNext(2) substream.expectNext(4) @@ -110,7 +110,7 @@ class FlowGroupBySpec extends AkkaSpec { "accept cancellation of master stream when not consumed anything" in { val publisherProbeProbe = StreamTestKit.PublisherProbe[Int]() - val publisher = Source(publisherProbeProbe).groupBy(_ % 2).runWith(PublisherDrain()) + val publisher = Source(publisherProbeProbe).groupBy(_ % 2).runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int])]() publisher.subscribe(subscriber) @@ -141,7 +141,7 @@ class FlowGroupBySpec extends AkkaSpec { } "work with empty input stream" in { - val publisher = Source(List.empty[Int]).groupBy(_ % 2).runWith(PublisherDrain()) + val publisher = Source(List.empty[Int]).groupBy(_ % 2).runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int])]() publisher.subscribe(subscriber) @@ -150,7 +150,7 @@ class FlowGroupBySpec extends AkkaSpec { "abort on onError from upstream" in { val publisherProbeProbe = StreamTestKit.PublisherProbe[Int]() - val publisher = Source(publisherProbeProbe).groupBy(_ % 2).runWith(PublisherDrain()) + val publisher = Source(publisherProbeProbe).groupBy(_ % 2).runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int])]() publisher.subscribe(subscriber) @@ -167,7 +167,7 @@ class FlowGroupBySpec extends AkkaSpec { "abort on onError from upstream when substreams are running" in { val publisherProbeProbe = StreamTestKit.PublisherProbe[Int]() - val publisher = Source(publisherProbeProbe).groupBy(_ % 2).runWith(PublisherDrain()) + val publisher = Source(publisherProbeProbe).groupBy(_ % 2).runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[(Int, Source[Int])]() publisher.subscribe(subscriber) @@ -179,7 +179,7 @@ class FlowGroupBySpec extends AkkaSpec { upstreamSubscription.sendNext(1) val (_, substream) = subscriber.expectNext() - val substreamPuppet = StreamPuppet(substream.runWith(PublisherDrain())) + val substreamPuppet = StreamPuppet(substream.runWith(Sink.publisher)) substreamPuppet.request(1) substreamPuppet.expectNext(1) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowGroupedWithinSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowGroupedWithinSpec.scala index 14b988090c..b39519e2bf 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowGroupedWithinSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowGroupedWithinSpec.scala @@ -24,7 +24,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest { val input = Iterator.from(1) val p = StreamTestKit.PublisherProbe[Int]() val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]() - Source(p).groupedWithin(1000, 1.second).connect(SubscriberDrain(c)).run() + Source(p).groupedWithin(1000, 1.second).connect(Sink(c)).run() val pSub = p.expectSubscription val cSub = c.expectSubscription cSub.request(100) @@ -49,7 +49,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest { "deliver bufferd elements onComplete before the timeout" in { val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]() - Source(1 to 3).groupedWithin(1000, 10.second).connect(SubscriberDrain(c)).run() + Source(1 to 3).groupedWithin(1000, 10.second).connect(Sink(c)).run() val cSub = c.expectSubscription cSub.request(100) c.expectNext((1 to 3).toList) @@ -61,7 +61,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest { val input = Iterator.from(1) val p = StreamTestKit.PublisherProbe[Int]() val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]() - Source(p).groupedWithin(1000, 1.second).connect(SubscriberDrain(c)).run() + Source(p).groupedWithin(1000, 1.second).connect(Sink(c)).run() val pSub = p.expectSubscription val cSub = c.expectSubscription cSub.request(1) @@ -81,7 +81,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest { "drop empty groups" in { val p = StreamTestKit.PublisherProbe[Int]() val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]() - Source(p).groupedWithin(1000, 500.millis).connect(SubscriberDrain(c)).run() + Source(p).groupedWithin(1000, 500.millis).connect(Sink(c)).run() val pSub = p.expectSubscription val cSub = c.expectSubscription cSub.request(2) @@ -103,7 +103,7 @@ class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest { val input = Iterator.from(1) val p = StreamTestKit.PublisherProbe[Int]() val c = StreamTestKit.SubscriberProbe[immutable.Seq[Int]]() - Source(p).groupedWithin(3, 2.second).connect(SubscriberDrain(c)).run() + Source(p).groupedWithin(3, 2.second).connect(Sink(c)).run() val pSub = p.expectSubscription val cSub = c.expectSubscription cSub.request(4) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowIterableSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowIterableSpec.scala index 03c3c45d27..57e6d14e45 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowIterableSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowIterableSpec.scala @@ -18,7 +18,7 @@ class FlowIterableSpec extends AkkaSpec { "A Flow based on an iterable" must { "produce elements" in { - val p = Source(List(1, 2, 3)).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3)).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() @@ -32,7 +32,7 @@ class FlowIterableSpec extends AkkaSpec { } "complete empty" in { - val p = Source(List.empty[Int]).runWith(PublisherDrain()) + val p = Source(List.empty[Int]).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) c.expectComplete() @@ -44,7 +44,7 @@ class FlowIterableSpec extends AkkaSpec { } "produce elements with multiple subscribers" in { - val p = Source(List(1, 2, 3)).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3)).runWith(Sink.publisher) val c1 = StreamTestKit.SubscriberProbe[Int]() val c2 = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c1) @@ -68,7 +68,7 @@ class FlowIterableSpec extends AkkaSpec { } "produce elements to later subscriber" in { - val p = Source(List(1, 2, 3)).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3)).runWith(Sink.publisher) val c1 = StreamTestKit.SubscriberProbe[Int]() val c2 = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c1) @@ -94,7 +94,7 @@ class FlowIterableSpec extends AkkaSpec { } "produce elements with one transformation step" in { - val p = Source(List(1, 2, 3)).map(_ * 2).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3)).map(_ * 2).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() @@ -106,7 +106,7 @@ class FlowIterableSpec extends AkkaSpec { } "produce elements with two transformation steps" in { - val p = Source(List(1, 2, 3, 4)).filter(_ % 2 == 0).map(_ * 2).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3, 4)).filter(_ % 2 == 0).map(_ * 2).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() @@ -118,7 +118,7 @@ class FlowIterableSpec extends AkkaSpec { "allow cancel before receiving all elements" in { val count = 100000 - val p = Source(1 to count).runWith(PublisherDrain()) + val p = Source(1 to count).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() @@ -134,19 +134,19 @@ class FlowIterableSpec extends AkkaSpec { } "have value equality of publisher" in { - val p1 = Source(List(1, 2, 3)).runWith(PublisherDrain()) - val p2 = Source(List(1, 2, 3)).runWith(PublisherDrain()) + val p1 = Source(List(1, 2, 3)).runWith(Sink.publisher) + val p2 = Source(List(1, 2, 3)).runWith(Sink.publisher) p1 should be(p2) p2 should be(p1) - val p3 = Source(List(1, 2, 3, 4)).runWith(PublisherDrain()) + val p3 = Source(List(1, 2, 3, 4)).runWith(Sink.publisher) p1 should not be (p3) p3 should not be (p1) - val p4 = Source(Vector.empty[String]).runWith(PublisherDrain()) - val p5 = Source(Set.empty[String]).runWith(PublisherDrain()) + val p4 = Source(Vector.empty[String]).runWith(Sink.publisher) + val p5 = Source(Set.empty[String]).runWith(Sink.publisher) p1 should not be (p4) p4 should be(p5) p5 should be(p4) - val p6 = Source(List(1, 2, 3).iterator).runWith(PublisherDrain()) + val p6 = Source(List(1, 2, 3).iterator).runWith(Sink.publisher) p1 should not be (p6) p6 should not be (p1) } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowIteratorSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowIteratorSpec.scala index 38540f13d9..fdf5a37d63 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowIteratorSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowIteratorSpec.scala @@ -22,7 +22,7 @@ class FlowIteratorSpec extends AkkaSpec { "A Flow based on an iterator" must { "produce elements" in { - val p = Source(List(1, 2, 3).iterator).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3).iterator).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() @@ -36,7 +36,7 @@ class FlowIteratorSpec extends AkkaSpec { } "complete empty" in { - val p = Source(List.empty[Int].iterator).runWith(PublisherDrain()) + val p = Source(List.empty[Int].iterator).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) c.expectComplete() @@ -48,7 +48,7 @@ class FlowIteratorSpec extends AkkaSpec { } "produce elements with multiple subscribers" in { - val p = Source(List(1, 2, 3).iterator).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3).iterator).runWith(Sink.publisher) val c1 = StreamTestKit.SubscriberProbe[Int]() val c2 = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c1) @@ -72,7 +72,7 @@ class FlowIteratorSpec extends AkkaSpec { } "produce elements to later subscriber" in { - val p = Source(List(1, 2, 3).iterator).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3).iterator).runWith(Sink.publisher) val c1 = StreamTestKit.SubscriberProbe[Int]() val c2 = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c1) @@ -95,7 +95,7 @@ class FlowIteratorSpec extends AkkaSpec { } "produce elements with one transformation step" in { - val p = Source(List(1, 2, 3).iterator).map(_ * 2).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3).iterator).map(_ * 2).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() @@ -107,7 +107,7 @@ class FlowIteratorSpec extends AkkaSpec { } "produce elements with two transformation steps" in { - val p = Source(List(1, 2, 3, 4).iterator).filter(_ % 2 == 0).map(_ * 2).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3, 4).iterator).filter(_ % 2 == 0).map(_ * 2).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() @@ -119,7 +119,7 @@ class FlowIteratorSpec extends AkkaSpec { "allow cancel before receiving all elements" in { val count = 100000 - val p = Source((1 to count).iterator).runWith(PublisherDrain()) + val p = Source((1 to count).iterator).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowMapAsyncSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowMapAsyncSpec.scala index 6497d972c2..e11b6f9ea9 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowMapAsyncSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowMapAsyncSpec.scala @@ -23,7 +23,7 @@ class FlowMapAsyncSpec extends AkkaSpec { "produce future elements" in { val c = StreamTestKit.SubscriberProbe[Int]() implicit val ec = system.dispatcher - val p = Source(1 to 3).mapAsync(n ⇒ Future(n)).connect(SubscriberDrain(c)).run() + val p = Source(1 to 3).mapAsync(n ⇒ Future(n)).connect(Sink(c)).run() val sub = c.expectSubscription() sub.request(2) c.expectNext(1) @@ -40,7 +40,7 @@ class FlowMapAsyncSpec extends AkkaSpec { val p = Source(1 to 50).mapAsync(n ⇒ Future { Thread.sleep(ThreadLocalRandom.current().nextInt(1, 10)) n - }).connect(SubscriberDrain(c)).run() + }).connect(Sink(c)).run() val sub = c.expectSubscription() sub.request(1000) for (n ← 1 to 50) c.expectNext(n) @@ -54,7 +54,7 @@ class FlowMapAsyncSpec extends AkkaSpec { val p = Source(1 to 20).mapAsync(n ⇒ Future { probe.ref ! n n - }).connect(SubscriberDrain(c)).run() + }).connect(Sink(c)).run() val sub = c.expectSubscription() // nothing before requested probe.expectNoMsg(500.millis) @@ -82,7 +82,7 @@ class FlowMapAsyncSpec extends AkkaSpec { Await.ready(latch, 10.seconds) n } - }).connect(SubscriberDrain(c)).run() + }).connect(Sink(c)).run() val sub = c.expectSubscription() sub.request(10) c.expectError.getMessage should be("err1") @@ -101,7 +101,7 @@ class FlowMapAsyncSpec extends AkkaSpec { n } }). - connect(SubscriberDrain(c)).run() + connect(Sink(c)).run() val sub = c.expectSubscription() sub.request(10) c.expectError.getMessage should be("err2") diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowMapAsyncUnorderedSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowMapAsyncUnorderedSpec.scala index abaf2ce7da..4ba9eb84ab 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowMapAsyncUnorderedSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowMapAsyncUnorderedSpec.scala @@ -26,7 +26,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec { val p = Source(1 to 4).mapAsyncUnordered(n ⇒ Future { Await.ready(latch(n), 5.seconds) n - }).connect(SubscriberDrain(c)).run() + }).connect(Sink(c)).run() val sub = c.expectSubscription() sub.request(5) latch(2).countDown() @@ -47,7 +47,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec { val p = Source(1 to 20).mapAsyncUnordered(n ⇒ Future { probe.ref ! n n - }).connect(SubscriberDrain(c)).run() + }).connect(Sink(c)).run() val sub = c.expectSubscription() // nothing before requested probe.expectNoMsg(500.millis) @@ -76,7 +76,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec { Await.ready(latch, 10.seconds) n } - }).connect(SubscriberDrain(c)).run() + }).connect(Sink(c)).run() val sub = c.expectSubscription() sub.request(10) c.expectError.getMessage should be("err1") @@ -95,7 +95,7 @@ class FlowMapAsyncUnorderedSpec extends AkkaSpec { n } }). - connect(SubscriberDrain(c)).run() + connect(Sink(c)).run() val sub = c.expectSubscription() sub.request(10) c.expectError.getMessage should be("err2") diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowMapSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowMapSpec.scala index 14269c3399..c29bd6343a 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowMapSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowMapSpec.scala @@ -28,7 +28,7 @@ class FlowMapSpec extends AkkaSpec with ScriptedTest { val probe = StreamTestKit.SubscriberProbe[Int]() Source(List(1).iterator). map(_ + 1).map(_ + 1).map(_ + 1).map(_ + 1).map(_ + 1). - runWith(PublisherDrain()).subscribe(probe) + runWith(Sink.publisher).subscribe(probe) val subscription = probe.expectSubscription() for (_ ← 1 to 10000) { diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowOnCompleteSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowOnCompleteSpec.scala index ff3a35bca4..e36653ef22 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowOnCompleteSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowOnCompleteSpec.scala @@ -27,7 +27,7 @@ class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest { "invoke callback on normal completion" in { val onCompleteProbe = TestProbe() val p = StreamTestKit.PublisherProbe[Int]() - Source(p).connect(OnCompleteDrain[Int](onCompleteProbe.ref ! _)).run() + Source(p).connect(Sink.onComplete[Int](onCompleteProbe.ref ! _)).run() val proc = p.expectSubscription proc.expectRequest() proc.sendNext(42) @@ -39,7 +39,7 @@ class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest { "yield the first error" in { val onCompleteProbe = TestProbe() val p = StreamTestKit.PublisherProbe[Int]() - Source(p).connect(OnCompleteDrain[Int](onCompleteProbe.ref ! _)).run() + Source(p).connect(Sink.onComplete[Int](onCompleteProbe.ref ! _)).run() val proc = p.expectSubscription proc.expectRequest() val ex = new RuntimeException("ex") with NoStackTrace @@ -51,7 +51,7 @@ class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest { "invoke callback for an empty stream" in { val onCompleteProbe = TestProbe() val p = StreamTestKit.PublisherProbe[Int]() - Source(p).connect(OnCompleteDrain[Int](onCompleteProbe.ref ! _)).run() + Source(p).connect(Sink.onComplete[Int](onCompleteProbe.ref ! _)).run() val proc = p.expectSubscription proc.expectRequest() proc.sendComplete() @@ -63,13 +63,13 @@ class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest { val onCompleteProbe = TestProbe() val p = StreamTestKit.PublisherProbe[Int]() import system.dispatcher // for the Future.onComplete - val foreachDrain = ForeachDrain[Int] { + val foreachSink = Sink.foreach[Int] { x ⇒ onCompleteProbe.ref ! ("foreach-" + x) } val future = Source(p).map { x ⇒ onCompleteProbe.ref ! ("map-" + x) x - }.runWith(foreachDrain) + }.runWith(foreachSink) future onComplete { onCompleteProbe.ref ! _ } val proc = p.expectSubscription proc.expectRequest() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowPrefixAndTailSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowPrefixAndTailSpec.scala index 8f84a6bc06..a47579eaea 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowPrefixAndTailSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowPrefixAndTailSpec.scala @@ -24,69 +24,69 @@ class FlowPrefixAndTailSpec extends AkkaSpec { val testException = new Exception("test") with NoStackTrace - def newFutureDrain = FutureDrain[(immutable.Seq[Int], Source[Int])] + def newFutureSink = Sink.future[(immutable.Seq[Int], Source[Int])] "work on empty input" in { - val futureDrain = newFutureDrain - val fut = Source.empty.prefixAndTail(10).runWith(futureDrain) + val futureSink = newFutureSink + val fut = Source.empty.prefixAndTail(10).runWith(futureSink) val (prefix, tailFlow) = Await.result(fut, 3.seconds) prefix should be(Nil) val tailSubscriber = SubscriberProbe[Int] - tailFlow.connect(SubscriberDrain(tailSubscriber)).run() + tailFlow.connect(Sink(tailSubscriber)).run() tailSubscriber.expectComplete() } "work on short input" in { - val futureDrain = newFutureDrain - val fut = Source(List(1, 2, 3)).prefixAndTail(10).runWith(futureDrain) + val futureSink = newFutureSink + val fut = Source(List(1, 2, 3)).prefixAndTail(10).runWith(futureSink) val (prefix, tailFlow) = Await.result(fut, 3.seconds) prefix should be(List(1, 2, 3)) val tailSubscriber = SubscriberProbe[Int] - tailFlow.connect(SubscriberDrain(tailSubscriber)).run() + tailFlow.connect(Sink(tailSubscriber)).run() tailSubscriber.expectComplete() } "work on longer inputs" in { - val futureDrain = newFutureDrain - val fut = Source((1 to 10).iterator).prefixAndTail(5).runWith(futureDrain) + val futureSink = newFutureSink + val fut = Source((1 to 10).iterator).prefixAndTail(5).runWith(futureSink) val (takes, tail) = Await.result(fut, 3.seconds) takes should be(1 to 5) - val futureDrain2 = FutureDrain[immutable.Seq[Int]] - val fut2 = tail.grouped(6).runWith(futureDrain2) + val futureSink2 = Sink.future[immutable.Seq[Int]] + val fut2 = tail.grouped(6).runWith(futureSink2) Await.result(fut2, 3.seconds) should be(6 to 10) } "handle zero take count" in { - val futureDrain = newFutureDrain - val fut = Source((1 to 10).iterator).prefixAndTail(0).runWith(futureDrain) + val futureSink = newFutureSink + val fut = Source((1 to 10).iterator).prefixAndTail(0).runWith(futureSink) val (takes, tail) = Await.result(fut, 3.seconds) takes should be(Nil) - val futureDrain2 = FutureDrain[immutable.Seq[Int]] - val fut2 = tail.grouped(11).runWith(futureDrain2) + val futureSink2 = Sink.future[immutable.Seq[Int]] + val fut2 = tail.grouped(11).runWith(futureSink2) Await.result(fut2, 3.seconds) should be(1 to 10) } "handle negative take count" in { - val futureDrain = newFutureDrain - val fut = Source((1 to 10).iterator).prefixAndTail(-1).runWith(futureDrain) + val futureSink = newFutureSink + val fut = Source((1 to 10).iterator).prefixAndTail(-1).runWith(futureSink) val (takes, tail) = Await.result(fut, 3.seconds) takes should be(Nil) - val futureDrain2 = FutureDrain[immutable.Seq[Int]] - val fut2 = tail.grouped(11).runWith(futureDrain2) + val futureSink2 = Sink.future[immutable.Seq[Int]] + val fut2 = tail.grouped(11).runWith(futureSink2) Await.result(fut2, 3.seconds) should be(1 to 10) } "work if size of take is equal to stream size" in { - val futureDrain = newFutureDrain - val fut = Source((1 to 10).iterator).prefixAndTail(10).runWith(futureDrain) + val futureSink = newFutureSink + val fut = Source((1 to 10).iterator).prefixAndTail(10).runWith(futureSink) val (takes, tail) = Await.result(fut, 3.seconds) takes should be(1 to 10) val subscriber = StreamTestKit.SubscriberProbe[Int]() - tail.connect(SubscriberDrain(subscriber)).run() + tail.connect(Sink(subscriber)).run() subscriber.expectCompletedOrSubscriptionFollowedByComplete() } @@ -94,7 +94,7 @@ class FlowPrefixAndTailSpec extends AkkaSpec { val publisher = StreamTestKit.PublisherProbe[Int]() val subscriber = StreamTestKit.SubscriberProbe[(immutable.Seq[Int], Source[Int])]() - Source(publisher).prefixAndTail(3).connect(SubscriberDrain(subscriber)).run() + Source(publisher).prefixAndTail(3).connect(Sink(subscriber)).run() val upstream = publisher.expectSubscription() val downstream = subscriber.expectSubscription() @@ -112,7 +112,7 @@ class FlowPrefixAndTailSpec extends AkkaSpec { val publisher = StreamTestKit.PublisherProbe[Int]() val subscriber = StreamTestKit.SubscriberProbe[(immutable.Seq[Int], Source[Int])]() - Source(publisher).prefixAndTail(1).connect(SubscriberDrain(subscriber)).run() + Source(publisher).prefixAndTail(1).connect(Sink(subscriber)).run() val upstream = publisher.expectSubscription() val downstream = subscriber.expectSubscription() @@ -127,7 +127,7 @@ class FlowPrefixAndTailSpec extends AkkaSpec { subscriber.expectComplete() val substreamSubscriber = StreamTestKit.SubscriberProbe[Int]() - tail.connect(SubscriberDrain(substreamSubscriber)).run() + tail.connect(Sink(substreamSubscriber)).run() substreamSubscriber.expectSubscription() upstream.sendError(testException) @@ -139,7 +139,7 @@ class FlowPrefixAndTailSpec extends AkkaSpec { val publisher = StreamTestKit.PublisherProbe[Int]() val subscriber = StreamTestKit.SubscriberProbe[(immutable.Seq[Int], Source[Int])]() - Source(publisher).prefixAndTail(3).connect(SubscriberDrain(subscriber)).run() + Source(publisher).prefixAndTail(3).connect(Sink(subscriber)).run() val upstream = publisher.expectSubscription() val downstream = subscriber.expectSubscription() @@ -157,7 +157,7 @@ class FlowPrefixAndTailSpec extends AkkaSpec { val publisher = StreamTestKit.PublisherProbe[Int]() val subscriber = StreamTestKit.SubscriberProbe[(immutable.Seq[Int], Source[Int])]() - Source(publisher).prefixAndTail(1).connect(SubscriberDrain(subscriber)).run() + Source(publisher).prefixAndTail(1).connect(Sink(subscriber)).run() val upstream = publisher.expectSubscription() val downstream = subscriber.expectSubscription() @@ -172,7 +172,7 @@ class FlowPrefixAndTailSpec extends AkkaSpec { subscriber.expectComplete() val substreamSubscriber = StreamTestKit.SubscriberProbe[Int]() - tail.connect(SubscriberDrain(substreamSubscriber)).run() + tail.connect(Sink(substreamSubscriber)).run() substreamSubscriber.expectSubscription().cancel() upstream.expectCancellation() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowPublishToSubscriberSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowPublishToSubscriberSpec.scala index 445272c946..9a3fa1af10 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowPublishToSubscriberSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowPublishToSubscriberSpec.scala @@ -15,11 +15,11 @@ class FlowPublishToSubscriberSpec extends AkkaSpec { implicit val materializer = FlowMaterializer(settings) - "A Flow with SubscriberDrain" must { + "A Flow with Sink" must { "publish elements to the subscriber" in { val c = StreamTestKit.SubscriberProbe[Int]() - Source(List(1, 2, 3)).connect(SubscriberDrain(c)).run() + Source(List(1, 2, 3)).connect(Sink(c)).run() val s = c.expectSubscription() s.request(3) c.expectNext(1) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowSpec.scala index 877678d65b..aaa0871e51 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowSpec.scala @@ -83,14 +83,14 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece val identity2: Flow[Any, Any] ⇒ Flow[Any, Any] = in ⇒ identity(in) val toPublisher: (Source[Any], FlowMaterializer) ⇒ Publisher[Any] = - (f, m) ⇒ f.runWith(PublisherDrain())(m) + (f, m) ⇒ f.runWith(Sink.publisher)(m) def toFanoutPublisher[In, Out](initialBufferSize: Int, maximumBufferSize: Int): (Source[Out], FlowMaterializer) ⇒ Publisher[Out] = - (f, m) ⇒ f.runWith(FanoutPublisherDrain(initialBufferSize, maximumBufferSize))(m) + (f, m) ⇒ f.runWith(Sink.fanoutPublisher(initialBufferSize, maximumBufferSize))(m) def materializeIntoSubscriberAndPublisher[In, Out](flow: Flow[In, Out]): (Subscriber[In], Publisher[Out]) = { - val tap = SubscriberTap[In] - val drain = PublisherDrain[Out] - flow.runWith(tap, drain) + val source = Source.subscriber[In] + val sink = Sink.publisher[Out] + flow.runWith(source, sink) } "A Flow" must { @@ -173,8 +173,8 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece val c1 = StreamTestKit.SubscriberProbe[String]() flowOut.subscribe(c1) - val tap: Publisher[String] = Source(List("1", "2", "3")).runWith(PublisherDrain()) - tap.subscribe(flowIn) + val source: Publisher[String] = Source(List("1", "2", "3")).runWith(Sink.publisher) + source.subscribe(flowIn) val sub1 = c1.expectSubscription sub1.request(3) @@ -194,8 +194,8 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece sub1.request(3) c1.expectNoMsg(200.millis) - val tap: Publisher[Int] = Source(List(1, 2, 3)).runWith(PublisherDrain()) - tap.subscribe(flowIn) + val source: Publisher[Int] = Source(List(1, 2, 3)).runWith(Sink.publisher) + source.subscribe(flowIn) c1.expectNext("1") c1.expectNext("2") @@ -213,8 +213,8 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece sub1.request(3) c1.expectNoMsg(200.millis) - val tap: Publisher[Int] = Source(List(1, 2, 3)).runWith(PublisherDrain()) - tap.subscribe(flowIn) + val source: Publisher[Int] = Source(List(1, 2, 3)).runWith(Sink.publisher) + source.subscribe(flowIn) c1.expectNext("elem-1") c1.expectNext("elem-2") @@ -225,8 +225,8 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece "subscribe Subscriber" in { val flow: Flow[String, String] = Flow[String] val c1 = StreamTestKit.SubscriberProbe[String]() - val sink: Sink[String] = flow.connect(SubscriberDrain(c1)) - val publisher: Publisher[String] = Source(List("1", "2", "3")).runWith(PublisherDrain()) + val sink: Sink[String] = flow.connect(Sink(c1)) + val publisher: Publisher[String] = Source(List("1", "2", "3")).runWith(Sink.publisher) Source(publisher).connect(sink).run() val sub1 = c1.expectSubscription @@ -240,8 +240,8 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece "perform transformation operation" in { val flow = Flow[Int].map(i ⇒ { testActor ! i.toString; i.toString }) - val publisher = Source(List(1, 2, 3)).runWith(PublisherDrain()) - Source(publisher).connect(flow).connect(BlackholeDrain).run() + val publisher = Source(List(1, 2, 3)).runWith(Sink.publisher) + Source(publisher).connect(flow).connect(Sink.ignore).run() expectMsg("1") expectMsg("2") @@ -251,8 +251,8 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece "perform transformation operation and subscribe Subscriber" in { val flow = Flow[Int].map(_.toString) val c1 = StreamTestKit.SubscriberProbe[String]() - val sink: Sink[Int] = flow.connect(SubscriberDrain(c1)) - val publisher: Publisher[Int] = Source(List(1, 2, 3)).runWith(PublisherDrain()) + val sink: Sink[Int] = flow.connect(Sink(c1)) + val publisher: Publisher[Int] = Source(List(1, 2, 3)).runWith(Sink.publisher) Source(publisher).connect(sink).run() val sub1 = c1.expectSubscription @@ -265,8 +265,8 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece "be materializable several times with fanout publisher" in { val flow = Source(List(1, 2, 3)).map(_.toString) - val p1 = flow.runWith(FanoutPublisherDrain(2, 2)) - val p2 = flow.runWith(FanoutPublisherDrain(2, 2)) + val p1 = flow.runWith(Sink.fanoutPublisher(2, 2)) + val p2 = flow.runWith(Sink.fanoutPublisher(2, 2)) val s1 = StreamTestKit.SubscriberProbe[String]() val s2 = StreamTestKit.SubscriberProbe[String]() val s3 = StreamTestKit.SubscriberProbe[String]() @@ -298,7 +298,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece "be covariant" in { val f1: Source[Fruit] = Source[Fruit](() ⇒ Some(new Apple)) - val p1: Publisher[Fruit] = Source[Fruit](() ⇒ Some(new Apple)).runWith(PublisherDrain()) + val p1: Publisher[Fruit] = Source[Fruit](() ⇒ Some(new Apple)).runWith(Sink.publisher) val f2: Source[Source[Fruit]] = Source[Fruit](() ⇒ Some(new Apple)).splitWhen(_ ⇒ true) val f3: Source[(Boolean, Source[Fruit])] = Source[Fruit](() ⇒ Some(new Apple)).groupBy(_ ⇒ true) val f4: Source[(immutable.Seq[Fruit], Source[Fruit])] = Source[Fruit](() ⇒ Some(new Apple)).prefixAndTail(1) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowSplitWhenSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowSplitWhenSpec.scala index 560f5883d1..2ef905e0b0 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowSplitWhenSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowSplitWhenSpec.scala @@ -31,8 +31,8 @@ class FlowSplitWhenSpec extends AkkaSpec { } class SubstreamsSupport(splitWhen: Int = 3, elementCount: Int = 6) { - val tap = Source((1 to elementCount).iterator) - val groupStream = tap.splitWhen(_ == splitWhen).runWith(PublisherDrain()) + val source = Source((1 to elementCount).iterator) + val groupStream = source.splitWhen(_ == splitWhen).runWith(Sink.publisher) val masterSubscriber = StreamTestKit.SubscriberProbe[Source[Int]]() groupStream.subscribe(masterSubscriber) @@ -53,7 +53,7 @@ class FlowSplitWhenSpec extends AkkaSpec { "splitWhen" must { "work in the happy case" in new SubstreamsSupport(elementCount = 4) { - val s1 = StreamPuppet(getSubFlow().runWith(PublisherDrain())) + val s1 = StreamPuppet(getSubFlow().runWith(Sink.publisher)) masterSubscriber.expectNoMsg(100.millis) s1.request(2) @@ -62,7 +62,7 @@ class FlowSplitWhenSpec extends AkkaSpec { s1.request(1) s1.expectComplete() - val s2 = StreamPuppet(getSubFlow().runWith(PublisherDrain())) + val s2 = StreamPuppet(getSubFlow().runWith(Sink.publisher)) s2.request(1) s2.expectNext(3) @@ -77,9 +77,9 @@ class FlowSplitWhenSpec extends AkkaSpec { } "support cancelling substreams" in new SubstreamsSupport(splitWhen = 5, elementCount = 8) { - val s1 = StreamPuppet(getSubFlow().runWith(PublisherDrain())) + val s1 = StreamPuppet(getSubFlow().runWith(Sink.publisher)) s1.cancel() - val s2 = StreamPuppet(getSubFlow().runWith(PublisherDrain())) + val s2 = StreamPuppet(getSubFlow().runWith(Sink.publisher)) s2.request(4) s2.expectNext(5) @@ -94,7 +94,7 @@ class FlowSplitWhenSpec extends AkkaSpec { } "support cancelling the master stream" in new SubstreamsSupport(splitWhen = 5, elementCount = 8) { - val s1 = StreamPuppet(getSubFlow().runWith(PublisherDrain())) + val s1 = StreamPuppet(getSubFlow().runWith(Sink.publisher)) masterSubscription.cancel() s1.request(4) s1.expectNext(1) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTakeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTakeSpec.scala index 81f231cc34..590b1704fa 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTakeSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTakeSpec.scala @@ -34,7 +34,7 @@ class FlowTakeSpec extends AkkaSpec with ScriptedTest { "not take anything for negative n" in { val probe = StreamTestKit.SubscriberProbe[Int]() - Source(List(1, 2, 3)).take(-1).connect(SubscriberDrain(probe)).run() + Source(List(1, 2, 3)).take(-1).connect(Sink(probe)).run() probe.expectSubscription().request(10) probe.expectComplete() } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTakeWithinSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTakeWithinSpec.scala index 2cbf617f1c..155f910a2e 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTakeWithinSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTakeWithinSpec.scala @@ -18,7 +18,7 @@ class FlowTakeWithinSpec extends AkkaSpec { val input = Iterator.from(1) val p = StreamTestKit.PublisherProbe[Int]() val c = StreamTestKit.SubscriberProbe[Int]() - Source(p).takeWithin(1.second).connect(SubscriberDrain(c)).run() + Source(p).takeWithin(1.second).connect(Sink(c)).run() val pSub = p.expectSubscription() val cSub = c.expectSubscription() cSub.request(100) @@ -38,7 +38,7 @@ class FlowTakeWithinSpec extends AkkaSpec { "deliver bufferd elements onComplete before the timeout" in { val c = StreamTestKit.SubscriberProbe[Int]() - Source(1 to 3).takeWithin(1.second).connect(SubscriberDrain(c)).run() + Source(1 to 3).takeWithin(1.second).connect(Sink(c)).run() val cSub = c.expectSubscription() c.expectNoMsg(200.millis) cSub.request(100) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowThunkSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowThunkSpec.scala index 50fda95a6e..ae02222048 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowThunkSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowThunkSpec.scala @@ -18,7 +18,7 @@ class FlowThunkSpec extends AkkaSpec { "produce elements" in { val iter = List(1, 2, 3).iterator - val p = Source(() ⇒ if (iter.hasNext) Some(iter.next()) else None).map(_ + 10).runWith(PublisherDrain()) + val p = Source(() ⇒ if (iter.hasNext) Some(iter.next()) else None).map(_ + 10).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() @@ -32,7 +32,7 @@ class FlowThunkSpec extends AkkaSpec { } "complete empty" in { - val p = Source(() ⇒ None).runWith(PublisherDrain()) + val p = Source(() ⇒ None).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() @@ -44,7 +44,7 @@ class FlowThunkSpec extends AkkaSpec { "allow cancel before receiving all elements" in { val count = 100000 val iter = (1 to count).iterator - val p = Source(() ⇒ if (iter.hasNext) Some(iter.next()) else None).runWith(PublisherDrain()) + val p = Source(() ⇒ if (iter.hasNext) Some(iter.next()) else None).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTimerTransformerSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTimerTransformerSpec.scala index 41bcf18730..70c9bc371e 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTimerTransformerSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTimerTransformerSpec.scala @@ -28,7 +28,7 @@ class FlowTimerTransformerSpec extends AkkaSpec { } override def isComplete: Boolean = !isTimerActive("tick") }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(subscriber) val subscription = subscriber.expectSubscription() @@ -54,7 +54,7 @@ class FlowTimerTransformerSpec extends AkkaSpec { } override def isComplete: Boolean = !isTimerActive("tick") }). - connect(BlackholeDrain).run() + connect(Sink.ignore).run() val pSub = p.expectSubscription() expectMsg("tick-1") expectMsg("tick-2") @@ -72,7 +72,7 @@ class FlowTimerTransformerSpec extends AkkaSpec { def onNext(element: Int) = Nil override def onTimer(timerKey: Any) = throw exception - }).runWith(PublisherDrain()) + }).runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(subscriber) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowToFutureSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowToFutureSpec.scala index 8c3c54c01b..54898cd446 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowToFutureSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowToFutureSpec.scala @@ -20,11 +20,11 @@ class FlowToFutureSpec extends AkkaSpec with ScriptedTest { implicit val materializer = FlowMaterializer(settings) - "A Flow with FutureDrain" must { + "A Flow with Sink.future" must { "yield the first value" in { val p = StreamTestKit.PublisherProbe[Int]() - val f: Future[Int] = Source(p).map(identity).runWith(FutureDrain()) + val f: Future[Int] = Source(p).map(identity).runWith(Sink.future) val proc = p.expectSubscription proc.expectRequest() proc.sendNext(42) @@ -34,20 +34,20 @@ class FlowToFutureSpec extends AkkaSpec with ScriptedTest { "yield the first value when actively constructing" in { val p = StreamTestKit.PublisherProbe[Int]() - val f = FutureDrain[Int] - val s = SubscriberTap[Int] + val f = Sink.future[Int] + val s = Source.subscriber[Int] val m = s.connect(f).run() - p.subscribe(m.materializedTap(s)) + p.subscribe(m.get(s)) val proc = p.expectSubscription proc.expectRequest() proc.sendNext(42) - Await.result(m.materializedDrain(f), 100.millis) should be(42) + Await.result(m.get(f), 100.millis) should be(42) proc.expectCancellation() } "yield the first error" in { val p = StreamTestKit.PublisherProbe[Int]() - val f = Source(p).runWith(FutureDrain()) + val f = Source(p).runWith(Sink.future) val proc = p.expectSubscription proc.expectRequest() val ex = new RuntimeException("ex") @@ -58,7 +58,7 @@ class FlowToFutureSpec extends AkkaSpec with ScriptedTest { "yield NoSuchElementExcption for empty stream" in { val p = StreamTestKit.PublisherProbe[Int]() - val f = Source(p).runWith(FutureDrain()) + val f = Source(p).runWith(Sink.future) val proc = p.expectSubscription proc.expectRequest() proc.sendComplete() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTransformRecoverSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTransformRecoverSpec.scala index 7ffb2246e1..8ea079cbc0 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTransformRecoverSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/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 = Source(List(1, 2, 3).iterator).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3).iterator).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[Int, Int] { var tot = 0 @@ -54,7 +54,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { case Some(_) ⇒ List(-1) } }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(subscriber) val subscription = subscriber.expectSubscription() @@ -68,7 +68,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { } "produce one-to-several transformation as expected" in { - val p = Source(List(1, 2, 3).iterator).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3).iterator).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[Int, Int] { var tot = 0 @@ -82,7 +82,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { case Some(_) ⇒ List(-1) } }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(subscriber) val subscription = subscriber.expectSubscription() @@ -99,7 +99,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { } "produce dropping transformation as expected" in { - val p = Source(List(1, 2, 3, 4).iterator).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3, 4).iterator).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[Int, Int] { var tot = 0 @@ -113,7 +113,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { case Some(_) ⇒ List(-1) } }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(subscriber) val subscription = subscriber.expectSubscription() @@ -127,7 +127,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { } "produce multi-step transformation as expected" in { - val p = Source(List("a", "bc", "def").iterator).runWith(PublisherDrain()) + val p = Source(List("a", "bc", "def").iterator).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new TryRecoveryTransformer[String, Int] { var concat = "" @@ -147,7 +147,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { case None ⇒ Nil case Some(_) ⇒ List(-1) } - }).runWith(FanoutPublisherDrain(1, 1)) + }).runWith(Sink.fanoutPublisher(1, 1)) val c1 = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(c1) val sub1 = c1.expectSubscription() @@ -170,7 +170,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { } "invoke onComplete when done" in { - val p = Source(List("a").iterator).runWith(PublisherDrain()) + val p = Source(List("a").iterator).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new TryRecoveryTransformer[String, String] { var s = "" @@ -180,7 +180,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { } override def onTermination(e: Option[Throwable]) = List(s + "B") }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[String]() p2.subscribe(c) val s = c.expectSubscription() @@ -200,7 +200,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { } override def isComplete = s == "Success(1)" }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val proc = p.expectSubscription val c = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(c) @@ -225,7 +225,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { override def isComplete = s == "Success(1)" override def onTermination(e: Option[Throwable]) = List(s.length + 10) }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val proc = p.expectSubscription val c = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(c) @@ -240,7 +240,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { } "report error when exception is thrown" in { - val p = Source(List(1, 2, 3).iterator).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3).iterator).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[Int, Int] { override def onNext(elem: Int) = { @@ -249,7 +249,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { } override def onError(e: Throwable) = List(-1) }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(subscriber) val subscription = subscriber.expectSubscription() @@ -279,7 +279,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { case Some(_) ⇒ List(-1, -2, -3) } }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(subscriber) val subscription = subscriber.expectSubscription() @@ -332,7 +332,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { } } }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val proc = p.expectSubscription() val c = StreamTestKit.SubscriberProbe[String]() p2.subscribe(c) @@ -353,7 +353,7 @@ class FlowTransformRecoverSpec extends AkkaSpec { override def onNext(in: Int) = List(in) override def onError(e: Throwable) = throw e }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val proc = p.expectSubscription() val c = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(c) @@ -366,13 +366,13 @@ class FlowTransformRecoverSpec extends AkkaSpec { } "support cancel as expected" in { - val p = Source(List(1, 2, 3).iterator).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3).iterator).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[Int, Int] { override def onNext(elem: Int) = List(elem, elem) override def onError(e: Throwable) = List(-1) }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(subscriber) val subscription = subscriber.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTransformSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTransformSpec.scala index 6642f802c7..62e306bd74 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTransformSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/FlowTransformSpec.scala @@ -23,7 +23,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 = Source(List(1, 2, 3)).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3)).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[Int, Int] { var tot = 0 @@ -32,7 +32,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d List(tot) } }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(subscriber) val subscription = subscriber.expectSubscription() @@ -46,7 +46,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } "produce one-to-several transformation as expected" in { - val p = Source(List(1, 2, 3)).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3)).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[Int, Int] { var tot = 0 @@ -55,7 +55,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d Vector.fill(elem)(tot) } }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(subscriber) val subscription = subscriber.expectSubscription() @@ -72,7 +72,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } "produce dropping transformation as expected" in { - val p = Source(List(1, 2, 3, 4)).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3, 4)).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[Int, Int] { var tot = 0 @@ -85,7 +85,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } } }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(subscriber) val subscription = subscriber.expectSubscription() @@ -99,7 +99,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } "produce multi-step transformation as expected" in { - val p = Source(List("a", "bc", "def")).runWith(PublisherDrain()) + val p = Source(List("a", "bc", "def")).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[String, Int] { var concat = "" @@ -115,7 +115,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d List(tot) } }). - runWith(FanoutPublisherDrain(2, 2)) + runWith(Sink.fanoutPublisher(2, 2)) val c1 = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(c1) val sub1 = c1.expectSubscription() @@ -138,7 +138,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } "invoke onComplete when done" in { - val p = Source(List("a")).runWith(PublisherDrain()) + val p = Source(List("a")).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[String, String] { var s = "" @@ -148,7 +148,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } override def onTermination(e: Option[Throwable]) = List(s + "B") }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[String]() p2.subscribe(c) val s = c.expectSubscription() @@ -159,7 +159,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d "invoke cleanup when done" in { val cleanupProbe = TestProbe() - val p = Source(List("a")).runWith(PublisherDrain()) + val p = Source(List("a")).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[String, String] { var s = "" @@ -170,7 +170,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d override def onTermination(e: Option[Throwable]) = List(s + "B") override def cleanup() = cleanupProbe.ref ! s }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[String]() p2.subscribe(c) val s = c.expectSubscription() @@ -182,7 +182,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d "invoke cleanup when done consume" in { val cleanupProbe = TestProbe() - val p = Source(List("a")).runWith(PublisherDrain()) + val p = Source(List("a")).runWith(Sink.publisher) Source(p). transform("transform", () ⇒ new Transformer[String, String] { var s = "x" @@ -192,13 +192,13 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } override def cleanup() = cleanupProbe.ref ! s }). - connect(BlackholeDrain).run() + connect(Sink.ignore).run() cleanupProbe.expectMsg("a") } "invoke cleanup when done after error" in { val cleanupProbe = TestProbe() - val p = Source(List("a", "b", "c")).runWith(PublisherDrain()) + val p = Source(List("a", "b", "c")).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[String, String] { var s = "" @@ -214,7 +214,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d override def onTermination(e: Option[Throwable]) = List(s + "B") override def cleanup() = cleanupProbe.ref ! s }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[String]() p2.subscribe(c) val s = c.expectSubscription() @@ -236,7 +236,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } override def isComplete = s == "1" }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val proc = p.expectSubscription val c = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(c) @@ -263,7 +263,7 @@ 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 }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val proc = p.expectSubscription val c = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(c) @@ -279,7 +279,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } "report error when exception is thrown" in { - val p = Source(List(1, 2, 3)).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3)).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[Int, Int] { override def onNext(elem: Int) = { @@ -290,7 +290,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } } }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(subscriber) val subscription = subscriber.expectSubscription() @@ -304,12 +304,12 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } "support cancel as expected" in { - val p = Source(List(1, 2, 3)).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3)).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[Int, Int] { override def onNext(elem: Int) = List(elem, elem) }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(subscriber) val subscription = subscriber.expectSubscription() @@ -323,13 +323,13 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d } "support producing elements from empty inputs" in { - val p = Source(List.empty[Int]).runWith(PublisherDrain()) + val p = Source(List.empty[Int]).runWith(Sink.publisher) val p2 = Source(p). transform("transform", () ⇒ new Transformer[Int, Int] { override def onNext(elem: Int) = Nil override def onTermination(e: Option[Throwable]) = List(1, 2, 3) }). - runWith(PublisherDrain()) + runWith(Sink.publisher) val subscriber = StreamTestKit.SubscriberProbe[Int]() p2.subscribe(subscriber) val subscription = subscriber.expectSubscription() @@ -363,7 +363,7 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d case _ ⇒ Nil } } - }).connect(SubscriberDrain(subscriber)).run() + }).connect(Sink(subscriber)).run() val subscription = subscriber.expectSubscription() subscription.request(10) @@ -386,13 +386,13 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d }) val s1 = StreamTestKit.SubscriberProbe[Int]() - flow.connect(SubscriberDrain(s1)).run() + flow.connect(Sink(s1)).run() s1.expectSubscription().request(3) s1.expectNext(1, 2, 3) s1.expectComplete() val s2 = StreamTestKit.SubscriberProbe[Int]() - flow.connect(SubscriberDrain(s2)).run() + flow.connect(Sink(s2)).run() s2.expectSubscription().request(3) s2.expectNext(1, 2, 3) s2.expectComplete() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphBalanceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphBalanceSpec.scala index 6937e588ea..f5c03e663d 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphBalanceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphBalanceSpec.scala @@ -24,8 +24,8 @@ class GraphBalanceSpec extends AkkaSpec { FlowGraph { implicit b ⇒ val balance = Balance[Int]("balance") Source(List(1, 2, 3)) ~> balance - balance ~> SubscriberDrain(c1) - balance ~> SubscriberDrain(c2) + balance ~> Sink(c1) + balance ~> Sink(c2) }.run() val sub1 = c1.expectSubscription() @@ -43,11 +43,11 @@ class GraphBalanceSpec extends AkkaSpec { } "work with 5-way balance" in { - val f1 = FutureDrain[Seq[Int]] - val f2 = FutureDrain[Seq[Int]] - val f3 = FutureDrain[Seq[Int]] - val f4 = FutureDrain[Seq[Int]] - val f5 = FutureDrain[Seq[Int]] + val f1 = Sink.future[Seq[Int]] + val f2 = Sink.future[Seq[Int]] + val f3 = Sink.future[Seq[Int]] + val f4 = Sink.future[Seq[Int]] + val f5 = Sink.future[Seq[Int]] val g = FlowGraph { implicit b ⇒ val balance = Balance[Int]("balance") @@ -59,12 +59,12 @@ class GraphBalanceSpec extends AkkaSpec { balance ~> Flow[Int].grouped(15) ~> f5 }.run() - Set(f1, f2, f3, f4, f5) flatMap (sink ⇒ Await.result(g.materializedDrain(sink), 3.seconds)) should be((0 to 14).toSet) + Set(f1, f2, f3, f4, f5) flatMap (sink ⇒ Await.result(g.get(sink), 3.seconds)) should be((0 to 14).toSet) } "fairly balance between three outputs" in { val numElementsForSink = 10000 - val f1, f2, f3 = FoldDrain[Int, Int](0)(_ + _) + val f1, f2, f3 = Sink.fold[Int, Int](0)(_ + _) val g = FlowGraph { implicit b ⇒ val balance = Balance[Int]("balance") Source(Stream.fill(10000 * 3)(1)) ~> balance ~> f1 @@ -73,7 +73,7 @@ class GraphBalanceSpec extends AkkaSpec { }.run() Seq(f1, f2, f3) map { sink ⇒ - Await.result(g.materializedDrain(sink), 3.seconds) should be(numElementsForSink +- 1000) + Await.result(g.get(sink), 3.seconds) should be(numElementsForSink +- 1000) } } @@ -84,8 +84,8 @@ class GraphBalanceSpec extends AkkaSpec { FlowGraph { implicit b ⇒ val balance = Balance[Int]("balance") Source(List(1, 2, 3)) ~> balance - balance ~> Flow[Int] ~> SubscriberDrain(c1) - balance ~> Flow[Int] ~> SubscriberDrain(c2) + balance ~> Flow[Int] ~> Sink(c1) + balance ~> Flow[Int] ~> Sink(c2) }.run() val sub1 = c1.expectSubscription() @@ -105,8 +105,8 @@ class GraphBalanceSpec extends AkkaSpec { FlowGraph { implicit b ⇒ val balance = Balance[Int]("balance") Source(List(1, 2, 3)) ~> balance - balance ~> Flow[Int] ~> SubscriberDrain(c1) - balance ~> Flow[Int] ~> SubscriberDrain(c2) + balance ~> Flow[Int] ~> Sink(c1) + balance ~> Flow[Int] ~> Sink(c2) }.run() val sub1 = c1.expectSubscription() @@ -127,8 +127,8 @@ class GraphBalanceSpec extends AkkaSpec { FlowGraph { implicit b ⇒ val balance = Balance[Int]("balance") Source(p1.getPublisher) ~> balance - balance ~> Flow[Int] ~> SubscriberDrain(c1) - balance ~> Flow[Int] ~> SubscriberDrain(c2) + balance ~> Flow[Int] ~> Sink(c1) + balance ~> Flow[Int] ~> Sink(c2) }.run() val bsub = p1.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphBroadcastSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphBroadcastSpec.scala index c429bbacb6..864c1f872c 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphBroadcastSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphBroadcastSpec.scala @@ -23,8 +23,8 @@ class GraphBroadcastSpec extends AkkaSpec { FlowGraph { implicit b ⇒ val bcast = Broadcast[Int]("broadcast") Source(List(1, 2, 3)) ~> bcast - bcast ~> Flow[Int].buffer(16, OverflowStrategy.backpressure) ~> SubscriberDrain(c1) - bcast ~> Flow[Int].buffer(16, OverflowStrategy.backpressure) ~> SubscriberDrain(c2) + bcast ~> Flow[Int].buffer(16, OverflowStrategy.backpressure) ~> Sink(c1) + bcast ~> Flow[Int].buffer(16, OverflowStrategy.backpressure) ~> Sink(c2) }.run() val sub1 = c1.expectSubscription() @@ -46,11 +46,11 @@ class GraphBroadcastSpec extends AkkaSpec { } "work with n-way broadcast" in { - val f1 = FutureDrain[Seq[Int]] - val f2 = FutureDrain[Seq[Int]] - val f3 = FutureDrain[Seq[Int]] - val f4 = FutureDrain[Seq[Int]] - val f5 = FutureDrain[Seq[Int]] + val f1 = Sink.future[Seq[Int]] + val f2 = Sink.future[Seq[Int]] + val f3 = Sink.future[Seq[Int]] + val f4 = Sink.future[Seq[Int]] + val f5 = Sink.future[Seq[Int]] val g = FlowGraph { implicit b ⇒ val bcast = Broadcast[Int]("broadcast") @@ -62,11 +62,11 @@ class GraphBroadcastSpec extends AkkaSpec { bcast ~> Flow[Int].grouped(5) ~> f5 }.run() - Await.result(g.materializedDrain(f1), 3.seconds) should be(List(1, 2, 3)) - Await.result(g.materializedDrain(f2), 3.seconds) should be(List(1, 2, 3)) - Await.result(g.materializedDrain(f3), 3.seconds) should be(List(1, 2, 3)) - Await.result(g.materializedDrain(f4), 3.seconds) should be(List(1, 2, 3)) - Await.result(g.materializedDrain(f5), 3.seconds) should be(List(1, 2, 3)) + Await.result(g.get(f1), 3.seconds) should be(List(1, 2, 3)) + Await.result(g.get(f2), 3.seconds) should be(List(1, 2, 3)) + Await.result(g.get(f3), 3.seconds) should be(List(1, 2, 3)) + Await.result(g.get(f4), 3.seconds) should be(List(1, 2, 3)) + Await.result(g.get(f5), 3.seconds) should be(List(1, 2, 3)) } "produce to other even though downstream cancels" in { @@ -76,8 +76,8 @@ class GraphBroadcastSpec extends AkkaSpec { FlowGraph { implicit b ⇒ val bcast = Broadcast[Int]("broadcast") Source(List(1, 2, 3)) ~> bcast - bcast ~> Flow[Int] ~> SubscriberDrain(c1) - bcast ~> Flow[Int] ~> SubscriberDrain(c2) + bcast ~> Flow[Int] ~> Sink(c1) + bcast ~> Flow[Int] ~> Sink(c2) }.run() val sub1 = c1.expectSubscription() @@ -97,8 +97,8 @@ class GraphBroadcastSpec extends AkkaSpec { FlowGraph { implicit b ⇒ val bcast = Broadcast[Int]("broadcast") Source(List(1, 2, 3)) ~> bcast - bcast ~> Flow[Int] ~> SubscriberDrain(c1) - bcast ~> Flow[Int] ~> SubscriberDrain(c2) + bcast ~> Flow[Int] ~> Sink(c1) + bcast ~> Flow[Int] ~> Sink(c2) }.run() val sub1 = c1.expectSubscription() @@ -119,8 +119,8 @@ class GraphBroadcastSpec extends AkkaSpec { FlowGraph { implicit b ⇒ val bcast = Broadcast[Int]("broadcast") Source(p1.getPublisher) ~> bcast - bcast ~> Flow[Int] ~> SubscriberDrain(c1) - bcast ~> Flow[Int] ~> SubscriberDrain(c2) + bcast ~> Flow[Int] ~> Sink(c1) + bcast ~> Flow[Int] ~> Sink(c2) }.run() val bsub = p1.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphConcatSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphConcatSpec.scala index 3a891c1a40..26d8fce583 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphConcatSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphConcatSpec.scala @@ -32,7 +32,7 @@ class GraphConcatSpec extends TwoStreamsSetup { concat1.out ~> concat2.first Source((5 to 10).iterator) ~> concat2.second - concat2.out ~> SubscriberDrain(probe) + concat2.out ~> Sink(probe) }.run() val subscription = probe.expectSubscription() @@ -111,7 +111,7 @@ class GraphConcatSpec extends TwoStreamsSetup { val concat = Concat[Int] Source(List(1, 2, 3)) ~> concat.first Source(promise.future) ~> concat.second - concat.out ~> SubscriberDrain(subscriber) + concat.out ~> Sink(subscriber) }.run() val subscription = subscriber.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphFlexiMergeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphFlexiMergeSpec.scala index 1af5f80821..ced3dfc9a8 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphFlexiMergeSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphFlexiMergeSpec.scala @@ -219,7 +219,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { val in1 = Source(List("a", "b", "c", "d")) val in2 = Source(List("e", "f")) - val out1 = PublisherDrain[String] + val out1 = Sink.publisher[String] "FlexiMerge" must { @@ -231,7 +231,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { }.run() val s = SubscriberProbe[String] - val p = m.materializedDrain(out1) + val p = m.get(out1) p.subscribe(s) val sub = s.expectSubscription() sub.request(10) @@ -249,7 +249,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { }.run() val s = SubscriberProbe[String] - val p = m.materializedDrain(out1) + val p = m.get(out1) p.subscribe(s) val sub = s.expectSubscription() sub.request(10) @@ -263,7 +263,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { } "build simple zip merge" in { - val output = PublisherDrain[(Int, String)] + val output = Sink.publisher[(Int, String)] val m = FlowGraph { implicit b ⇒ val merge = new Zip[Int, String] Source(List(1, 2, 3, 4)) ~> merge.input1 @@ -272,7 +272,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { }.run() val s = SubscriberProbe[(Int, String)] - val p = m.materializedDrain(output) + val p = m.get(output) p.subscribe(s) val sub = s.expectSubscription() sub.request(10) @@ -283,7 +283,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { } "build simple ordered merge 1" in { - val output = PublisherDrain[Int] + val output = Sink.publisher[Int] val m = FlowGraph { implicit b ⇒ val merge = new OrderedMerge Source(List(3, 5, 6, 7, 8)) ~> merge.input1 @@ -292,7 +292,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { }.run() val s = SubscriberProbe[Int] - val p = m.materializedDrain(output) + val p = m.get(output) p.subscribe(s) val sub = s.expectSubscription() sub.request(100) @@ -302,7 +302,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { } "build simple ordered merge 2" in { - val output = PublisherDrain[Int] + val output = Sink.publisher[Int] val m = FlowGraph { implicit b ⇒ val merge = new OrderedMerge Source(List(3, 5, 6, 7, 8)) ~> merge.input1 @@ -311,7 +311,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { }.run() val s = SubscriberProbe[Int] - val p = m.materializedDrain(output) + val p = m.get(output) p.subscribe(s) val sub = s.expectSubscription() sub.request(100) @@ -340,7 +340,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { }.run() val s = SubscriberProbe[String] - val p = m.materializedDrain(out1) + val p = m.get(out1) p.subscribe(s) val autoPublisher = new AutoPublisher(publisher) @@ -376,7 +376,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { }.run() val s = SubscriberProbe[String] - val p = m.materializedDrain(out1) + val p = m.get(out1) p.subscribe(s) val autoPublisher1 = new AutoPublisher(publisher1) @@ -409,7 +409,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { }.run() val s = SubscriberProbe[String] - val p = m.materializedDrain(out1) + val p = m.get(out1) p.subscribe(s) val sub = s.expectSubscription() sub.request(10) @@ -433,7 +433,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { }.run() val s = SubscriberProbe[String] - val p = m.materializedDrain(out1) + val p = m.get(out1) p.subscribe(s) s.expectErrorOrSubscriptionFollowedByError().getMessage should be("ERROR") } @@ -448,7 +448,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { }.run() val s = SubscriberProbe[String] - val p = m.materializedDrain(out1) + val p = m.get(out1) p.subscribe(s) val sub = s.expectSubscription() sub.request(10) @@ -467,7 +467,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { }.run() val s = SubscriberProbe[String] - val p = m.materializedDrain(out1) + val p = m.get(out1) p.subscribe(s) val sub = s.expectSubscription() sub.request(10) @@ -486,7 +486,7 @@ class GraphFlexiMergeSpec extends AkkaSpec { }.run() val s = SubscriberProbe[String] - val p = m.materializedDrain(out1) + val p = m.get(out1) p.subscribe(s) val sub = s.expectSubscription() sub.request(10) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphFlowSpec.scala index 0930ad8ba6..1ff27c4746 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphFlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphFlowSpec.scala @@ -8,21 +8,21 @@ import akka.stream.testkit.StreamTestKit.SubscriberProbe import akka.stream.testkit.{ StreamTestKit, AkkaSpec } object GraphFlowSpec { - val tap1 = Source(0 to 3) + val source1 = Source(0 to 3) val inMerge = Merge[Int]("m1") val outMerge = Merge[String]("m3") val partialGraph = PartialFlowGraph { implicit b ⇒ import FlowGraphImplicits._ - val tap2 = Source(4 to 9) - val tap3 = Source.empty[Int] - val tap4 = Source.empty[String] + val source2 = Source(4 to 9) + val source3 = Source.empty[Int] + val source4 = Source.empty[String] val m2 = Merge[Int]("m2") inMerge ~> Flow[Int].map(_ * 2) ~> m2 ~> Flow[Int].map(_ / 2).map(i ⇒ (i + 1).toString) ~> outMerge - tap2 ~> inMerge - tap3 ~> m2 - tap4 ~> outMerge + source2 ~> inMerge + source3 ~> m2 + source4 ~> outMerge } val stdRequests = 10 @@ -54,7 +54,7 @@ class GraphFlowSpec extends AkkaSpec { "FlowGraphs" when { "turned into flows" should { - "work with a Tap and Drain" in { + "work with a Source and Sink" in { val in = UndefinedSource[Int] val out = UndefinedSink[Int] val probe = StreamTestKit.SubscriberProbe[Int]() @@ -66,7 +66,7 @@ class GraphFlowSpec extends AkkaSpec { in -> out } - tap1.connect(flow).connect(Sink(probe)).run() + source1.connect(flow).connect(Sink(probe)).run() validateProbe(probe, stdRequests, stdResult) } @@ -84,7 +84,7 @@ class GraphFlowSpec extends AkkaSpec { in -> out } - tap1.connect(flow).map(_.toInt).connect(Sink(probe)).run() + source1.connect(flow).map(_.toInt).connect(Sink(probe)).run() validateProbe(probe, stdRequests, stdResult) } @@ -111,7 +111,7 @@ class GraphFlowSpec extends AkkaSpec { in2 -> out2 } - tap1.connect(flow1).connect(flow2).connect(Sink(probe)).run() + source1.connect(flow1).connect(flow2).connect(Sink(probe)).run() validateProbe(probe, stdRequests, stdResult) } @@ -137,13 +137,13 @@ class GraphFlowSpec extends AkkaSpec { } "turned into sources" should { - "work with a Drain" in { + "work with a Sink" in { val out = UndefinedSink[Int] val probe = StreamTestKit.SubscriberProbe[Int]() val source = Source(partialGraph) { implicit b ⇒ import FlowGraphImplicits._ - tap1 ~> inMerge + source1 ~> inMerge outMerge ~> Flow[String].map(_.toInt) ~> out out } @@ -160,7 +160,7 @@ class GraphFlowSpec extends AkkaSpec { val source = Source[String](partialGraph) { implicit b ⇒ import FlowGraphImplicits._ - tap1 ~> inMerge + source1 ~> inMerge outMerge ~> out out } @@ -180,7 +180,7 @@ class GraphFlowSpec extends AkkaSpec { val source = Source(partialGraph) { implicit b ⇒ import FlowGraphImplicits._ - tap1 ~> inMerge + source1 ~> inMerge outMerge ~> out1 out1 } @@ -217,7 +217,7 @@ class GraphFlowSpec extends AkkaSpec { } } "turned into sinks" should { - "work with a Tap" in { + "work with a Source" in { val in = UndefinedSource[Int] val probe = StreamTestKit.SubscriberProbe[Int]() @@ -228,7 +228,7 @@ class GraphFlowSpec extends AkkaSpec { in } - tap1.connect(sink).run() + source1.connect(sink).run() validateProbe(probe, stdRequests, stdResult) } @@ -246,7 +246,7 @@ class GraphFlowSpec extends AkkaSpec { } val iSink = Flow[Int].map(_.toString).connect(sink) - tap1.connect(iSink).run() + source1.connect(iSink).run() validateProbe(probe, stdRequests, stdResult) } @@ -272,7 +272,7 @@ class GraphFlowSpec extends AkkaSpec { in2 } - tap1.connect(flow).connect(sink).run() + source1.connect(flow).connect(sink).run() validateProbe(probe, stdRequests, stdResult) } @@ -281,8 +281,8 @@ class GraphFlowSpec extends AkkaSpec { "used together" should { "materialize properly" in { val probe = StreamTestKit.SubscriberProbe[Int]() - val inSource = SubscriberTap[Int] - val outSink = PublisherDrain[Int] + val inSource = Source.subscriber[Int] + val outSink = Sink.publisher[Int] val flow = Flow(partialGraph) { implicit b ⇒ import FlowGraphImplicits._ @@ -312,9 +312,9 @@ class GraphFlowSpec extends AkkaSpec { source ~> Flow[String].map(_.toInt) ~> flow ~> Flow[Int].map(_.toString) ~> sink }.run() - val subscriber = mm.materializedTap(inSource) - val publisher = mm.materializedDrain(outSink) - tap1.runWith(PublisherDrain()).subscribe(subscriber) + val subscriber = mm.get(inSource) + val publisher = mm.get(outSink) + source1.runWith(Sink.publisher).subscribe(subscriber) publisher.subscribe(probe) validateProbe(probe, stdRequests, stdResult) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphMergeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphMergeSpec.scala index 565015a5f8..359b7288f5 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphMergeSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphMergeSpec.scala @@ -19,9 +19,9 @@ class GraphMergeSpec extends TwoStreamsSetup { "work in the happy case" in { // Different input sizes (4 and 6) - val tap1 = Source((0 to 3).iterator) - val tap2 = Source((4 to 9).iterator) - val tap3 = Source(List.empty[Int].iterator) + val source1 = Source((0 to 3).iterator) + val source2 = Source((4 to 9).iterator) + val source3 = Source(List.empty[Int].iterator) val probe = StreamTestKit.SubscriberProbe[Int]() FlowGraph { implicit b ⇒ @@ -29,9 +29,9 @@ class GraphMergeSpec extends TwoStreamsSetup { val m2 = Merge[Int]("m2") val m3 = Merge[Int]("m3") - tap1 ~> m1 ~> Flow[Int].map(_ * 2) ~> m2 ~> Flow[Int].map(_ / 2).map(_ + 1) ~> SubscriberDrain(probe) - tap2 ~> m1 - tap3 ~> m2 + source1 ~> m1 ~> Flow[Int].map(_ * 2) ~> m2 ~> Flow[Int].map(_ / 2).map(_ + 1) ~> Sink(probe) + source2 ~> m1 + source3 ~> m2 }.run() @@ -48,24 +48,24 @@ class GraphMergeSpec extends TwoStreamsSetup { } "work with n-way merge" in { - val tap1 = Source(List(1)) - val tap2 = Source(List(2)) - val tap3 = Source(List(3)) - val tap4 = Source(List(4)) - val tap5 = Source(List(5)) - val tap6 = Source(List.empty[Int]) + val source1 = Source(List(1)) + val source2 = Source(List(2)) + val source3 = Source(List(3)) + val source4 = Source(List(4)) + val source5 = Source(List(5)) + val source6 = Source(List.empty[Int]) val probe = StreamTestKit.SubscriberProbe[Int]() FlowGraph { implicit b ⇒ val merge = Merge[Int]("merge") - tap1 ~> merge ~> Flow[Int] ~> SubscriberDrain(probe) - tap2 ~> merge - tap3 ~> merge - tap4 ~> merge - tap5 ~> merge - tap6 ~> merge + source1 ~> merge ~> Flow[Int] ~> Sink(probe) + source2 ~> merge + source3 ~> merge + source4 ~> merge + source5 ~> merge + source6 ~> merge }.run() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphOpsIntegrationSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphOpsIntegrationSpec.scala index 283826c737..8668b93c33 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphOpsIntegrationSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphOpsIntegrationSpec.scala @@ -63,7 +63,7 @@ class GraphOpsIntegrationSpec extends AkkaSpec { "FlowGraphs" must { "support broadcast - merge layouts" in { - val resultFuture = FutureDrain[Seq[Int]] + val resultFuture = Sink.future[Seq[Int]] val g = FlowGraph { implicit b ⇒ val bcast = Broadcast[Int]("broadcast") @@ -75,14 +75,14 @@ class GraphOpsIntegrationSpec extends AkkaSpec { merge ~> Flow[Int].grouped(10) ~> resultFuture }.run() - Await.result(g.materializedDrain(resultFuture), 3.seconds).sorted should be(List(1, 2, 3, 4, 5, 6)) + Await.result(g.get(resultFuture), 3.seconds).sorted should be(List(1, 2, 3, 4, 5, 6)) } "support balance - merge (parallelization) layouts" in { val elements = 0 to 10 val in = Source(elements) val f = Flow[Int] - val out = FutureDrain[Seq[Int]] + val out = Sink.future[Seq[Int]] val g = FlowGraph { implicit b ⇒ val balance = Balance[Int] @@ -95,14 +95,14 @@ class GraphOpsIntegrationSpec extends AkkaSpec { balance ~> f ~> merge ~> Flow[Int].grouped(elements.size * 2) ~> out }.run() - Await.result(g.materializedDrain(out), 3.seconds).sorted should be(elements) + Await.result(g.get(out), 3.seconds).sorted should be(elements) } "support wikipedia Topological_sorting 2" in { // see https://en.wikipedia.org/wiki/Topological_sorting#mediaviewer/File:Directed_acyclic_graph.png - val resultFuture2 = FutureDrain[Seq[Int]] - val resultFuture9 = FutureDrain[Seq[Int]] - val resultFuture10 = FutureDrain[Seq[Int]] + val resultFuture2 = Sink.future[Seq[Int]] + val resultFuture9 = Sink.future[Seq[Int]] + val resultFuture10 = Sink.future[Seq[Int]] val g = FlowGraph { implicit b ⇒ val b3 = Broadcast[Int]("b3") @@ -112,9 +112,9 @@ class GraphOpsIntegrationSpec extends AkkaSpec { val m9 = Merge[Int]("m9") val m10 = Merge[Int]("m10") val m11 = Merge[Int]("m11") - val in3 = IterableTap(List(3)) - val in5 = IterableTap(List(5)) - val in7 = IterableTap(List(7)) + val in3 = Source(List(3)) + val in5 = Source(List(5)) + val in7 = Source(List(7)) // First layer in7 ~> b7 @@ -141,14 +141,14 @@ class GraphOpsIntegrationSpec extends AkkaSpec { }.run() - Await.result(g.materializedDrain(resultFuture2), 3.seconds).sorted should be(List(5, 7)) - Await.result(g.materializedDrain(resultFuture9), 3.seconds).sorted should be(List(3, 5, 7, 7)) - Await.result(g.materializedDrain(resultFuture10), 3.seconds).sorted should be(List(3, 5, 7)) + Await.result(g.get(resultFuture2), 3.seconds).sorted should be(List(5, 7)) + Await.result(g.get(resultFuture9), 3.seconds).sorted should be(List(3, 5, 7, 7)) + Await.result(g.get(resultFuture10), 3.seconds).sorted should be(List(3, 5, 7)) } "allow adding of flows to sources and sinks to flows" in { - val resultFuture = FutureDrain[Seq[Int]] + val resultFuture = Sink.future[Seq[Int]] val g = FlowGraph { implicit b ⇒ val bcast = Broadcast[Int]("broadcast") @@ -160,16 +160,16 @@ class GraphOpsIntegrationSpec extends AkkaSpec { merge ~> Flow[Int].grouped(10).connect(resultFuture) }.run() - Await.result(g.materializedDrain(resultFuture), 3.seconds) should contain theSameElementsAs (Seq(2, 4, 6, 5, 7, 9)) + Await.result(g.get(resultFuture), 3.seconds) should contain theSameElementsAs (Seq(2, 4, 6, 5, 7, 9)) } "be able to run plain flow" in { - val p = Source(List(1, 2, 3)).runWith(PublisherDrain()) + val p = Source(List(1, 2, 3)).runWith(Sink.publisher) val s = SubscriberProbe[Int] val flow = Flow[Int].map(_ * 2) FlowGraph { implicit builder ⇒ import FlowGraphImplicits._ - PublisherTap(p) ~> flow ~> SubscriberDrain(s) + Source(p) ~> flow ~> Sink(s) }.run() val sub = s.expectSubscription() sub.request(10) @@ -193,8 +193,8 @@ class GraphOpsIntegrationSpec extends AkkaSpec { val s2 = SubscriberProbe[String] FlowGraph(partial) { builder ⇒ builder.attachSource(input1, Source(List(0, 1, 2).map(_ + 1))) - builder.attachSink(output1, Flow[Int].filter(n ⇒ (n % 2) != 0).connect(SubscriberDrain(s1))) - builder.attachSink(output2, Flow[String].map(_.toUpperCase).connect(SubscriberDrain(s2))) + builder.attachSink(output1, Flow[Int].filter(n ⇒ (n % 2) != 0).connect(Sink(s1))) + builder.attachSink(output2, Flow[String].map(_.toUpperCase).connect(Sink(s2))) }.run() val sub1 = s1.expectSubscription() @@ -216,7 +216,7 @@ class GraphOpsIntegrationSpec extends AkkaSpec { val lego3 = lego1.connect(lego2, Flow[ByteString].map(_.utf8String)) val source = Source(List("green ", "blue", "red", "yellow", "black")) val s = SubscriberProbe[ByteString] - val sink = SubscriberDrain(s) + val sink = Sink(s) lego3.run(source, sink) val sub = s.expectSubscription() sub.request(100) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphPreferredMergeSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphPreferredMergeSpec.scala index 2855deacda..2073214ac9 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphPreferredMergeSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphPreferredMergeSpec.scala @@ -25,17 +25,17 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup { val preferred = Source(Stream.fill(numElements)(1)) val aux1, aux2, aux3 = Source(Stream.fill(numElements)(2)) - val drain = FutureDrain[Seq[Int]] + val sink = Sink.future[Seq[Int]] val g = FlowGraph { implicit b ⇒ val merge = MergePreferred[Int] - preferred ~> merge.preferred ~> Flow[Int].grouped(numElements * 2) ~> drain + preferred ~> merge.preferred ~> Flow[Int].grouped(numElements * 2) ~> sink aux1 ~> merge aux2 ~> merge aux3 ~> merge }.run() - Await.result(g.materializedDrain(drain), 3.seconds).filter(_ == 1).size should be(numElements) + Await.result(g.get(sink), 3.seconds).filter(_ == 1).size should be(numElements) } "disallow multiple preferred inputs" in { @@ -45,7 +45,7 @@ class GraphPreferredMergeSpec extends TwoStreamsSetup { val g = FlowGraph { implicit b ⇒ val merge = MergePreferred[Int] - s1 ~> merge.preferred ~> FutureDrain[Int] + s1 ~> merge.preferred ~> Sink.future[Int] s2 ~> merge.preferred s3 ~> merge } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphUnzipSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphUnzipSpec.scala index 5b732d8ada..8f0391ef20 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphUnzipSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphUnzipSpec.scala @@ -23,8 +23,8 @@ class GraphUnzipSpec extends AkkaSpec { FlowGraph { implicit b ⇒ val unzip = Unzip[Int, String]("unzip") Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in - unzip.right ~> Flow[String].buffer(16, OverflowStrategy.backpressure) ~> SubscriberDrain(c2) - unzip.left ~> Flow[Int].buffer(16, OverflowStrategy.backpressure).map(_ * 2) ~> SubscriberDrain(c1) + unzip.right ~> Flow[String].buffer(16, OverflowStrategy.backpressure) ~> Sink(c2) + unzip.left ~> Flow[Int].buffer(16, OverflowStrategy.backpressure).map(_ * 2) ~> Sink(c1) }.run() val sub1 = c1.expectSubscription() @@ -52,8 +52,8 @@ class GraphUnzipSpec extends AkkaSpec { FlowGraph { implicit b ⇒ val unzip = Unzip[Int, String]("unzip") Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in - unzip.left ~> SubscriberDrain(c1) - unzip.right ~> SubscriberDrain(c2) + unzip.left ~> Sink(c1) + unzip.right ~> Sink(c2) }.run() val sub1 = c1.expectSubscription() @@ -73,8 +73,8 @@ class GraphUnzipSpec extends AkkaSpec { FlowGraph { implicit b ⇒ val unzip = Unzip[Int, String]("unzip") Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in - unzip.left ~> SubscriberDrain(c1) - unzip.right ~> SubscriberDrain(c2) + unzip.left ~> Sink(c1) + unzip.right ~> Sink(c2) }.run() val sub1 = c1.expectSubscription() @@ -95,8 +95,8 @@ class GraphUnzipSpec extends AkkaSpec { FlowGraph { implicit b ⇒ val unzip = Unzip[Int, String]("unzip") Source(p1.getPublisher) ~> unzip.in - unzip.left ~> SubscriberDrain(c1) - unzip.right ~> SubscriberDrain(c2) + unzip.left ~> Sink(c1) + unzip.right ~> Sink(c2) }.run() val p1Sub = p1.expectSubscription() @@ -125,7 +125,7 @@ class GraphUnzipSpec extends AkkaSpec { Source(List(1 -> "a", 2 -> "b", 3 -> "c")) ~> unzip.in unzip.left ~> zip.left unzip.right ~> zip.right - zip.out ~> SubscriberDrain(c1) + zip.out ~> Sink(c1) }.run() val sub1 = c1.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphZipSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphZipSpec.scala index fecf3f4b0b..4e0cc8bd48 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphZipSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/GraphZipSpec.scala @@ -22,7 +22,7 @@ class GraphZipSpec extends TwoStreamsSetup { Source(1 to 4) ~> zip.left Source(List("A", "B", "C", "D", "E", "F")) ~> zip.right - zip.out ~> SubscriberDrain(probe) + zip.out ~> Sink(probe) }.run() val subscription = probe.expectSubscription() diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/ImplicitFlowMaterializerSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/ImplicitFlowMaterializerSpec.scala index c1014097ae..a71141528f 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/ImplicitFlowMaterializerSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/ImplicitFlowMaterializerSpec.scala @@ -22,7 +22,6 @@ object ImplicitFlowMaterializerSpec { case "run" ⇒ // run takes an implicit FlowMaterializer parameter, which is provided by ImplicitFlowMaterializer import context.dispatcher - val foldDrain = FoldDrain[String, String]("")(_ + _) flow.fold("")(_ + _) pipeTo sender() } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/SourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/SourceSpec.scala index 7a878e211a..63c844bee7 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/SourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/SourceSpec.scala @@ -14,7 +14,7 @@ class SourceSpec extends AkkaSpec { "Singleton Source" must { "produce element" in { - val p = Source.singleton(1).runWith(PublisherDrain()) + val p = Source.singleton(1).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) val sub = c.expectSubscription() @@ -24,7 +24,7 @@ class SourceSpec extends AkkaSpec { } "produce elements to later subscriber" in { - val p = Source.singleton(1).runWith(PublisherDrain()) + val p = Source.singleton(1).runWith(Sink.publisher) val c1 = StreamTestKit.SubscriberProbe[Int]() val c2 = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c1) @@ -44,7 +44,7 @@ class SourceSpec extends AkkaSpec { "Empty Source" must { "complete immediately" in { - val p = Source.empty.runWith(PublisherDrain()) + val p = Source.empty.runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) c.expectComplete() @@ -58,7 +58,7 @@ class SourceSpec extends AkkaSpec { "Failed Source" must { "emit error immediately" in { val ex = new RuntimeException with NoStackTrace - val p = Source.failed(ex).runWith(PublisherDrain()) + val p = Source.failed(ex).runWith(Sink.publisher) val c = StreamTestKit.SubscriberProbe[Int]() p.subscribe(c) c.expectError(ex) diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/TickPublisherSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/TickPublisherSpec.scala index 3d5bd3ce3a..9177332c52 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/TickPublisherSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl2/TickPublisherSpec.scala @@ -17,7 +17,7 @@ class TickPublisherSpec extends AkkaSpec { "produce ticks" in { val tickGen = Iterator from 1 val c = StreamTestKit.SubscriberProbe[String]() - Source(1.second, 500.millis, () ⇒ "tick-" + tickGen.next()).connect(SubscriberDrain(c)).run() + Source(1.second, 500.millis, () ⇒ "tick-" + tickGen.next()).connect(Sink(c)).run() val sub = c.expectSubscription() sub.request(3) c.expectNoMsg(600.millis) @@ -33,7 +33,7 @@ class TickPublisherSpec extends AkkaSpec { "drop ticks when not requested" in { val tickGen = Iterator from 1 val c = StreamTestKit.SubscriberProbe[String]() - Source(1.second, 1.second, () ⇒ "tick-" + tickGen.next()).connect(SubscriberDrain(c)).run() + Source(1.second, 1.second, () ⇒ "tick-" + tickGen.next()).connect(Sink(c)).run() val sub = c.expectSubscription() sub.request(2) c.expectNext("tick-1") @@ -50,7 +50,7 @@ class TickPublisherSpec extends AkkaSpec { "produce ticks with multiple subscribers" in { val tickGen = Iterator from 1 - val p = Source(1.second, 1.second, () ⇒ "tick-" + tickGen.next()).runWith(PublisherDrain()) + val p = Source(1.second, 1.second, () ⇒ "tick-" + tickGen.next()).runWith(Sink.publisher) val c1 = StreamTestKit.SubscriberProbe[String]() val c2 = StreamTestKit.SubscriberProbe[String]() p.subscribe(c1) @@ -74,7 +74,7 @@ class TickPublisherSpec extends AkkaSpec { "signal onError when tick closure throws" in { val c = StreamTestKit.SubscriberProbe[String]() - Source[String](1.second, 1.second, () ⇒ throw new RuntimeException("tick err") with NoStackTrace).connect(SubscriberDrain(c)).run() + Source[String](1.second, 1.second, () ⇒ throw new RuntimeException("tick err") with NoStackTrace).connect(Sink(c)).run() val sub = c.expectSubscription() sub.request(3) c.expectError.getMessage should be("tick err") @@ -83,8 +83,8 @@ class TickPublisherSpec extends AkkaSpec { // FIXME enable this test again when zip is back "be usable with zip for a simple form of rate limiting" ignore { // val c = StreamTestKit.SubscriberProbe[Int]() - // val rate = Source(1.second, 1.second, () ⇒ "tick").runWith(PublisherDrain()) - // Source(1 to 100).zip(rate).map { case (n, _) ⇒ n }.connect(SubscriberDrain(c)).run() + // val rate = Source(1.second, 1.second, () ⇒ "tick").runWith(Sink.publisher) + // Source(1 to 100).zip(rate).map { case (n, _) ⇒ n }.connect(Sink(c)).run() // val sub = c.expectSubscription() // sub.request(1000) // c.expectNext(1) diff --git a/akka-stream/src/main/scala/akka/stream/impl2/ActorBasedFlowMaterializer.scala b/akka-stream/src/main/scala/akka/stream/impl2/ActorBasedFlowMaterializer.scala index 71ed06853f..c6ebdde6d9 100644 --- a/akka-stream/src/main/scala/akka/stream/impl2/ActorBasedFlowMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/impl2/ActorBasedFlowMaterializer.scala @@ -136,57 +136,54 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting } // Ops come in reverse order - override def materialize[In, Out](tap: Tap[In], drain: Drain[Out], ops: List[Ast.AstNode]): MaterializedMap = { + override def materialize[In, Out](source: Source[In], sink: Sink[Out], ops: List[Ast.AstNode]): MaterializedMap = { val flowName = createFlowName() - def attachDrain(pub: Publisher[Out]) = drain match { - case s: SimpleDrain[Out] ⇒ s.attach(pub, this, flowName) - case s: DrainWithKey[Out] ⇒ s.attach(pub, this, flowName) - case _ ⇒ throw new MaterializationException("unknown Drain type " + drain.getClass) + def throwUnknownType(typeName: String, s: Any): Nothing = + throw new MaterializationException(s"unknown $typeName type " + s.getClass) + + def attachSink(pub: Publisher[Out]) = sink match { + case s: ActorFlowSink[Out] ⇒ s.attach(pub, this, flowName) + case s ⇒ throwUnknownType("Sink", s) } - def attachTap(sub: Subscriber[In]) = tap match { - case s: SimpleTap[In] ⇒ s.attach(sub, this, flowName) - case s: TapWithKey[In] ⇒ s.attach(sub, this, flowName) - case _ ⇒ throw new MaterializationException("unknown Tap type " + drain.getClass) + def attachSource(sub: Subscriber[In]) = source match { + case s: ActorFlowSource[In] ⇒ s.attach(sub, this, flowName) + case s ⇒ throwUnknownType("Source", s) } - def createDrain() = drain.asInstanceOf[Drain[In]] match { - case s: SimpleDrain[In] ⇒ s.create(this, flowName) -> (()) - case s: DrainWithKey[In] ⇒ s.create(this, flowName) - case _ ⇒ throw new MaterializationException("unknown Drain type " + drain.getClass) + def createSink() = sink match { + case s: ActorFlowSink[In] ⇒ s.create(this, flowName) + case s ⇒ throwUnknownType("Sink", s) } - def createTap() = tap.asInstanceOf[Tap[Out]] match { - case s: SimpleTap[Out] ⇒ s.create(this, flowName) -> (()) - case s: TapWithKey[Out] ⇒ s.create(this, flowName) - case _ ⇒ throw new MaterializationException("unknown Tap type " + drain.getClass) + def createSource() = source match { + case s: ActorFlowSource[Out] ⇒ s.create(this, flowName) + case s ⇒ throwUnknownType("Source", s) } def isActive(s: AnyRef) = s match { - case tap: SimpleTap[_] ⇒ tap.isActive - case tap: TapWithKey[_] ⇒ tap.isActive - case drain: SimpleDrain[_] ⇒ drain.isActive - case drain: DrainWithKey[_] ⇒ drain.isActive - case _: Tap[_] ⇒ throw new MaterializationException("unknown Tap type " + drain.getClass) - case _: Drain[_] ⇒ throw new MaterializationException("unknown Drain type " + drain.getClass) + case s: ActorFlowSource[_] ⇒ s.isActive + case s: ActorFlowSink[_] ⇒ s.isActive + case s: Source[_] ⇒ throwUnknownType("Source", s) + case s: Sink[_] ⇒ throwUnknownType("Sink", s) } - val (tapValue, drainValue) = + val (sourceValue, sinkValue) = if (ops.isEmpty) { - if (isActive(drain)) { - val (sub, value) = createDrain() - (attachTap(sub), value) - } else if (isActive(tap)) { - val (pub, value) = createTap() - (value, attachDrain(pub)) + if (isActive(sink)) { + val (sub, value) = createSink() + (attachSource(sub), value) + } else if (isActive(source)) { + val (pub, value) = createSource() + (value, attachSink(pub)) } else { val id: Processor[In, Out] = processorForNode(identityTransform, flowName, 1).asInstanceOf[Processor[In, Out]] - (attachTap(id), attachDrain(id)) + (attachSource(id), attachSink(id)) } } else { val opsSize = ops.size val last = processorForNode(ops.head, flowName, opsSize).asInstanceOf[Processor[Any, Out]] val first = processorChain(last, ops.tail, flowName, opsSize - 1).asInstanceOf[Processor[In, Any]] - (attachTap(first), attachDrain(last)) + (attachSource(first), attachSink(last)) } - new MaterializedPipe(tap, tapValue, drain, drainValue) + new MaterializedPipe(source, sourceValue, sink, sinkValue) } private val identityTransform = Ast.Transform("identity", () ⇒ diff --git a/akka-stream/src/main/scala/akka/stream/impl2/ConcatAllImpl.scala b/akka-stream/src/main/scala/akka/stream/impl2/ConcatAllImpl.scala index 471535b183..7f5cf43dbd 100644 --- a/akka-stream/src/main/scala/akka/stream/impl2/ConcatAllImpl.scala +++ b/akka-stream/src/main/scala/akka/stream/impl2/ConcatAllImpl.scala @@ -4,7 +4,7 @@ package akka.stream.impl2 import akka.stream.impl.{ Extract, MultiStreamInputProcessor, TransferPhase } -import akka.stream.scaladsl2.{ FlowMaterializer, PublisherDrain } +import akka.stream.scaladsl2.{ Sink, FlowMaterializer } /** * INTERNAL API @@ -16,7 +16,7 @@ private[akka] class ConcatAllImpl(materializer: FlowMaterializer) val takeNextSubstream = TransferPhase(primaryInputs.NeedsInput && primaryOutputs.NeedsDemand) { () ⇒ val Extract.Source(source) = primaryInputs.dequeueInputElement() - val publisher = source.runWith(PublisherDrain())(materializer) + val publisher = source.runWith(Sink.publisher)(materializer) // FIXME we can pass the flow to createSubstreamInput (but avoiding copy impl now) val inputs = createAndSubscribeSubstreamInput(publisher) nextPhase(streamSubstream(inputs)) diff --git a/akka-stream/src/main/scala/akka/stream/io2/TcpListenStreamActor.scala b/akka-stream/src/main/scala/akka/stream/io2/TcpListenStreamActor.scala index b90080cd0f..31dc7ed7bd 100644 --- a/akka-stream/src/main/scala/akka/stream/io2/TcpListenStreamActor.scala +++ b/akka-stream/src/main/scala/akka/stream/io2/TcpListenStreamActor.scala @@ -41,26 +41,26 @@ private[akka] object TcpListenStreamActor { } } - val sink = new SimpleDrain[ByteString] { + val sink = new SimpleActorFlowSink[ByteString] { override def attach(flowPublisher: Publisher[ByteString], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = { val p = processor(materializer, flowName) flowPublisher.subscribe(p) } - override def create(materializer: ActorBasedFlowMaterializer, flowName: String): Subscriber[ByteString] = - processor(materializer, flowName) + override def create(materializer: ActorBasedFlowMaterializer, flowName: String) = + (processor(materializer, flowName), ()) override def isActive: Boolean = true } - val source = new SimpleTap[ByteString] { + val source = new SimpleActorFlowSource[ByteString] { override def attach(flowSubscriber: Subscriber[ByteString], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = { val p = processor(materializer, flowName) p.subscribe(flowSubscriber) } - override def create(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[ByteString] = - processor(materializer, flowName) + override def create(materializer: ActorBasedFlowMaterializer, flowName: String) = + (processor(materializer, flowName), ()) override def isActive = true } diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/ActorFlowSink.scala b/akka-stream/src/main/scala/akka/stream/javadsl/ActorFlowSink.scala new file mode 100644 index 0000000000..dbcd61fdcf --- /dev/null +++ b/akka-stream/src/main/scala/akka/stream/javadsl/ActorFlowSink.scala @@ -0,0 +1,15 @@ +/** + * Copyright (C) 2014 Typesafe Inc. + */ +package akka.stream.javadsl + +import akka.stream.javadsl +import akka.stream.scaladsl2 + +final case class SimpleSink[-In](override val delegate: scaladsl2.Sink[In]) extends javadsl.SinkAdapter[In] { + override def asScala: scaladsl2.SimpleActorFlowSink[In] = super.asScala.asInstanceOf[scaladsl2.SimpleActorFlowSink[In]] +} + +final case class KeyedSink[-In, M](override val delegate: scaladsl2.Sink[In]) extends javadsl.SinkAdapter[In] { + override def asScala: scaladsl2.KeyedSink[In] = super.asScala.asInstanceOf[scaladsl2.KeyedSink[In]] +} diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/ActorFlowSource.scala b/akka-stream/src/main/scala/akka/stream/javadsl/ActorFlowSource.scala new file mode 100644 index 0000000000..99acddfc91 --- /dev/null +++ b/akka-stream/src/main/scala/akka/stream/javadsl/ActorFlowSource.scala @@ -0,0 +1,14 @@ +/** + * Copyright (C) 2014 Typesafe Inc. + */ +package akka.stream.javadsl + +import akka.stream.scaladsl2 + +final case class SimpleSource[+Out](override val delegate: scaladsl2.Source[Out]) extends SourceAdapter[Out] { + override def asScala: scaladsl2.SimpleActorFlowSource[Out] = super.asScala.asInstanceOf[scaladsl2.SimpleActorFlowSource[Out]] +} + +final case class KeyedSource[+Out, T](override val delegate: scaladsl2.Source[Out]) extends SourceAdapter[Out] { + override def asScala: scaladsl2.KeyedActorFlowSource[Out] = super.asScala.asInstanceOf[scaladsl2.KeyedActorFlowSource[Out]] +} diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Drain.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Drain.scala deleted file mode 100644 index 802e79464d..0000000000 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Drain.scala +++ /dev/null @@ -1,83 +0,0 @@ -/** - * Copyright (C) 2014 Typesafe Inc. - */ -package akka.stream.javadsl - -import akka.stream.scaladsl2.FlowMaterializer -import org.reactivestreams.{ Subscriber, Publisher } - -import akka.stream.javadsl -import akka.stream.scaladsl2 - -import scala.concurrent.Future - -abstract class Drain[-In] extends javadsl.SinkAdapter[In] { - protected def delegate: scaladsl2.Drain[In] - - override def runWith[T](tap: javadsl.TapWithKey[In, T], materializer: FlowMaterializer): T = { - val sTap = tap.asScala - sTap.connect(asScala).run()(materializer).materializedTap(sTap).asInstanceOf[T] - } - - override def runWith(tap: javadsl.SimpleTap[In], materializer: FlowMaterializer): Unit = { - tap.asScala.connect(asScala).run()(materializer) - } -} - -abstract class SimpleDrain[-In] extends javadsl.Drain[In] { - override def asScala: scaladsl2.SimpleDrain[In] = super.asScala.asInstanceOf[scaladsl2.SimpleDrain[In]] -} - -abstract class DrainWithKey[-In, M] extends javadsl.Drain[In] { - override def asScala: scaladsl2.DrainWithKey[In] = super.asScala.asInstanceOf[scaladsl2.DrainWithKey[In]] -} - -// adapters // - -object SubscriberDrain { - def create[In](subs: Subscriber[In]): SubscriberDrain[In] = - new SubscriberDrain(scaladsl2.SubscriberDrain[In](subs)) -} -final class SubscriberDrain[In](protected val delegate: scaladsl2.SubscriberDrain[In]) extends javadsl.DrainWithKey[In, Subscriber[In]] - -object PublisherDrain { - def create[In](): PublisherDrain[In] = - new PublisherDrain(scaladsl2.PublisherDrain[In]()) -} -final class PublisherDrain[In](protected val delegate: scaladsl2.PublisherDrain[In]) extends javadsl.DrainWithKey[In, Publisher[In]] - -object FanoutPublisherDrain { - def create[In](initialBufferSize: Int, maximumBufferSize: Int): FanoutPublisherDrain[In] = - new FanoutPublisherDrain(scaladsl2.PublisherDrain.withFanout[In](initialBufferSize, maximumBufferSize)) -} -final class FanoutPublisherDrain[In](protected val delegate: scaladsl2.FanoutPublisherDrain[In]) extends javadsl.DrainWithKey[In, Publisher[In]] - -object FutureDrain { - def create[In](): FutureDrain[In] = - new FutureDrain[In](scaladsl2.FutureDrain[In]()) -} -final class FutureDrain[In](protected val delegate: scaladsl2.FutureDrain[In]) extends javadsl.DrainWithKey[In, Future[In]] - -object BlackholeDrain { - def create[In](): BlackholeDrain[In] = - new BlackholeDrain[In](scaladsl2.BlackholeDrain) -} -final class BlackholeDrain[In](protected val delegate: scaladsl2.BlackholeDrain.type) extends javadsl.SimpleDrain[In] - -object OnCompleteDrain { - def create[In](onComplete: akka.dispatch.OnComplete[Unit]): OnCompleteDrain[In] = - new OnCompleteDrain[In](scaladsl2.OnCompleteDrain[In](x ⇒ onComplete.apply(x))) -} -final class OnCompleteDrain[In](protected val delegate: scaladsl2.OnCompleteDrain[In]) extends javadsl.SimpleDrain[In] - -object ForeachDrain { - def create[In](f: japi.Procedure[In]): ForeachDrain[In] = - new ForeachDrain[In](new scaladsl2.ForeachDrain[In](x ⇒ f(x))) -} -final class ForeachDrain[In](protected val delegate: scaladsl2.ForeachDrain[In]) extends javadsl.DrainWithKey[In, Future[Unit]] - -object FoldDrain { - def create[U, In](zero: U, f: japi.Function2[U, In, U]): FoldDrain[U, In] = - new FoldDrain[U, In](new scaladsl2.FoldDrain[U, In](zero)(f.apply)) -} -final class FoldDrain[U, In](protected val delegate: scaladsl2.FoldDrain[U, In]) extends javadsl.DrainWithKey[In, Future[U]] 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 98e53df5f6..6594a17845 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala @@ -66,19 +66,19 @@ private[akka] class FlowAdapter[-In, +Out](delegate: scaladsl2.Flow[In, Out]) ex // RUN WITH // - def runWith[T, D](tap: javadsl.TapWithKey[In, T], drain: javadsl.DrainWithKey[Out, D], materializer: scaladsl2.FlowMaterializer): akka.japi.Pair[T, D] = { - val p = delegate.runWith(tap.asScala, drain.asScala)(materializer) + def runWith[T, D](source: javadsl.KeyedSource[In, T], sink: javadsl.KeyedSink[Out, D], materializer: scaladsl2.FlowMaterializer): akka.japi.Pair[T, D] = { + val p = delegate.runWith(source.asScala, sink.asScala)(materializer) akka.japi.Pair(p._1.asInstanceOf[T], p._2.asInstanceOf[D]) } - def runWith[D](tap: javadsl.SimpleTap[In], drain: javadsl.DrainWithKey[Out, D], materializer: scaladsl2.FlowMaterializer): D = - delegate.runWith(tap.asScala, drain.asScala)(materializer).asInstanceOf[D] + def runWith[D](source: javadsl.SimpleSource[In], sink: javadsl.KeyedSink[Out, D], materializer: scaladsl2.FlowMaterializer): D = + delegate.runWith(source.asScala, sink.asScala)(materializer).asInstanceOf[D] - def runWith[T](tap: javadsl.TapWithKey[In, T], drain: javadsl.SimpleDrain[Out], materializer: scaladsl2.FlowMaterializer): T = - delegate.runWith(tap.asScala, drain.asScala)(materializer).asInstanceOf[T] + def runWith[T](source: javadsl.KeyedSource[In, T], sink: javadsl.SimpleSink[Out], materializer: scaladsl2.FlowMaterializer): T = + delegate.runWith(source.asScala, sink.asScala)(materializer).asInstanceOf[T] - def runWith(tap: javadsl.SimpleTap[In], drain: javadsl.SimpleDrain[Out], materializer: scaladsl2.FlowMaterializer): Unit = - delegate.runWith(tap.asScala, drain.asScala)(materializer) + def runWith(source: javadsl.SimpleSource[In], sink: javadsl.SimpleSink[Out], materializer: scaladsl2.FlowMaterializer): Unit = + delegate.runWith(source.asScala, sink.asScala)(materializer) // COMMON OPS // diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/FlowGraph.scala b/akka-stream/src/main/scala/akka/stream/javadsl/FlowGraph.scala index c220cb86ef..baadb27881 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/FlowGraph.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/FlowGraph.scala @@ -70,8 +70,8 @@ object Merge { * Merge several streams, taking elements as they arrive from input streams * (picking randomly when several have elements ready). * - * When building the [[FlowGraph]] you must connect one or more input pipes/taps - * and one output pipe/sink to the `Merge` vertex. + * When building the [[FlowGraph]] you must connect one or more input sources + * and one output sink to the `Merge` vertex. */ class Merge[T] private (delegate: scaladsl2.Merge[T]) extends javadsl.Junction[T] { override def asScala: scaladsl2.Merge[T] = delegate @@ -115,8 +115,8 @@ object MergePreferred { * Merge several streams, taking elements as they arrive from input streams * (picking from preferred when several have elements ready). * - * When building the [[FlowGraph]] you must connect one or more input pipes/taps - * and one output pipe/drain to the `Merge` vertex. + * When building the [[FlowGraph]] you must connect one or more input streams + * and one output sink to the `Merge` vertex. */ class MergePreferred[T](delegate: scaladsl2.MergePreferred[T]) extends javadsl.Junction[T] { override def asScala: scaladsl2.MergePreferred[T] = delegate @@ -249,8 +249,8 @@ object UndefinedSource { /** * It is possible to define a [[akka.stream.javadsl.PartialFlowGraph]] with input pipes that are not connected - * yet by using this placeholder instead of the real [[Tap]]. Later the placeholder can - * be replaced with [[akka.stream.javadsl.FlowGraphBuilder#attachTap]]. + * yet by using this placeholder instead of the real [[Source]]. Later the placeholder can + * be replaced with [[akka.stream.javadsl.FlowGraphBuilder#attachSource]]. */ final class UndefinedSource[+T](delegate: scaladsl2.UndefinedSource[T]) { def asScala: scaladsl2.UndefinedSource[T] = delegate @@ -292,8 +292,8 @@ object UndefinedSink { /** * It is possible to define a [[akka.stream.javadsl.PartialFlowGraph]] with input pipes that are not connected - * yet by using this placeholder instead of the real [[Tap]]. Later the placeholder can - * be replaced with [[akka.stream.javadsl.FlowGraphBuilder#attachTap]]. + * yet by using this placeholder instead of the real [[Sink]]. Later the placeholder can + * be replaced with [[akka.stream.javadsl.FlowGraphBuilder#attachSink]]. */ final class UndefinedSink[-T](delegate: scaladsl2.UndefinedSink[T]) { def asScala: scaladsl2.UndefinedSink[T] = delegate diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/MaterializedMap.scala b/akka-stream/src/main/scala/akka/stream/javadsl/MaterializedMap.scala index d3333eca1b..1a93284e3f 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/MaterializedMap.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/MaterializedMap.scala @@ -10,33 +10,27 @@ import akka.stream.scaladsl2 * Java API * * Returned by [[RunnableFlow#run]] and can be used as parameter to the - * accessor method to retrieve the materialized `Tap` or `Drain`, e.g. - * [[akka.stream.scaladsl2.SubscriberTap#subscriber]] or [[akka.stream.scaladsl2.PublisherDrain#publisher]]. + * accessor method to retrieve the materialized `Source` or `Sink`, e.g. + * [[akka.stream.scaladsl2.SubscriberSource#subscriber]] or [[akka.stream.scaladsl2.PublisherSink#publisher]]. */ -trait MaterializedMap extends javadsl.MaterializedTap with javadsl.MaterializedDrain - -/** Java API */ -trait MaterializedTap { +trait MaterializedMap { /** - * Retrieve a materialized `Tap`, e.g. the `Subscriber` of a [[akka.stream.scaladsl2.SubscriberTap]]. + * Retrieve a materialized `Source`, e.g. the `Subscriber` of a [[akka.stream.scaladsl2.SubscriberSource]]. */ - def materializedTap[T](key: javadsl.TapWithKey[_, T]): T -} + def get[T](key: javadsl.KeyedSource[_, T]): T -/** Java API */ -trait MaterializedDrain { /** - * Retrieve a materialized `Drain`, e.g. the `Publisher` of a [[akka.stream.scaladsl2.PublisherDrain]]. + * Retrieve a materialized `Sink`, e.g. the `Publisher` of a [[akka.stream.scaladsl2.PublisherSink]]. */ - def materializedDrain[D](key: javadsl.DrainWithKey[_, D]): D + def get[D](key: javadsl.KeyedSink[_, D]): D } /** INTERNAL API */ private[akka] class MaterializedMapAdapter(delegate: scaladsl2.MaterializedMap) extends MaterializedMap { - override def materializedTap[T](key: javadsl.TapWithKey[_, T]): T = - delegate.materializedTap(key.asScala).asInstanceOf[T] + override def get[T](key: javadsl.KeyedSource[_, T]): T = + delegate.get(key.asScala).asInstanceOf[T] - override def materializedDrain[D](key: javadsl.DrainWithKey[_, D]): D = - delegate.materializedDrain(key.asScala).asInstanceOf[D] + override def get[D](key: javadsl.KeyedSink[_, D]): D = + delegate.get(key.asScala).asInstanceOf[D] } diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala index 109e3b9449..65cddb881a 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Sink.scala @@ -5,6 +5,9 @@ package akka.stream.javadsl import akka.stream.javadsl import akka.stream.scaladsl2 +import org.reactivestreams.{ Publisher, Subscriber } + +import scala.concurrent.Future object Sink { /** @@ -13,12 +16,54 @@ object Sink { * Adapt [[scaladsl2.Sink]] for use within JavaDSL */ def adapt[O](sink: scaladsl2.Sink[O]): javadsl.Sink[O] = SinkAdapter(sink) + + /** + * A `Sink` that will invoke the given function for every received element, giving it its previous + * output (or the given `zero` value) and the element as input. + * The returned [[scala.concurrent.Future]] will be completed with value of the final + * function evaluation when the input stream ends, or completed with `Failure` + * if there is an error is signaled in the stream. + */ + def fold[U, In](zero: U, f: japi.Function2[U, In, U]): javadsl.KeyedSink[In, Future[U]] = KeyedSink(scaladsl2.Sink.fold[U, In](zero)(f.apply)) + + /** + * A `Sink` that will invoke the given procedure for each received element. The sink is materialized + * into a [[scala.concurrent.Future]] will be completed with `Success` when reaching the + * normal end of the stream, or completed with `Failure` if there is an error is signaled in + * the stream.. + */ + def foreach[In](f: japi.Procedure[In]): javadsl.KeyedSink[In, Future[Unit]] = KeyedSink(scaladsl2.Sink.foreach[In](x ⇒ f(x))) + + /** + * Helper to create [[Sink]] from `Subscriber`. + */ + def subscriber[In](subs: Subscriber[In]): KeyedSink[In, Subscriber[In]] = KeyedSink(scaladsl2.Sink(subs)) + + /** + * A `Sink` that materializes into a [[org.reactivestreams.Publisher]]. + * that can handle one [[org.reactivestreams.Subscriber]]. + */ + def publisher[In](): KeyedSink[In, Publisher[In]] = KeyedSink(scaladsl2.Sink.publisher) + + /** + * A `Sink` that when the flow is completed, either through an error or normal + * completion, apply the provided function with [[scala.util.Success]] + * or [[scala.util.Failure]]. + */ + def onComplete[In](onComplete: akka.dispatch.OnComplete[Unit]): SimpleSink[In] = + SimpleSink(scaladsl2.Sink.onComplete[In](x ⇒ onComplete.apply(x))) + + /** + * A `Sink` that materializes into a `Future` of the first value received. + */ + def future[In]: KeyedSink[In, Future[In]] = KeyedSink(scaladsl2.Sink.future[In]) } + /** * A `Sink` is a set of stream processing steps that has one open input and an attached output. * Can be used as a `Subscriber` */ -trait Sink[-In] extends javadsl.SinkOps[In] +abstract class Sink[-In] extends javadsl.SinkOps[In] /** INTERNAL API */ private[akka] object SinkAdapter { @@ -35,10 +80,10 @@ private[akka] abstract class SinkAdapter[-In] extends Sink[In] { // RUN WITH // - def runWith[T](tap: javadsl.TapWithKey[In, T], materializer: scaladsl2.FlowMaterializer): T = - delegate.runWith(tap.asScala)(materializer).asInstanceOf[T] + def runWith[T](source: javadsl.KeyedSource[In, T], materializer: scaladsl2.FlowMaterializer): T = + delegate.runWith(source.asScala)(materializer).asInstanceOf[T] - def runWith(tap: javadsl.SimpleTap[In], materializer: scaladsl2.FlowMaterializer): Unit = - delegate.runWith(tap.asScala)(materializer) + def runWith(source: javadsl.SimpleSource[In], materializer: scaladsl2.FlowMaterializer): Unit = + delegate.runWith(source.asScala)(materializer) -} \ No newline at end of file +} diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala index 0c60590624..880605a784 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala @@ -42,18 +42,18 @@ abstract class Source[+Out] extends javadsl.SourceOps[Out] { def connect(sink: javadsl.Sink[Out]): javadsl.RunnableFlow /** - * Connect this `Source` to a `Drain` and run it. The returned value is the materialized value - * of the `Drain`, e.g. the `Publisher` of a [[akka.stream.scaladsl2.PublisherDrain]]. + * Connect this `Source` to a `Sink` and run it. The returned value is the materialized value + * of the `Sink`, e.g. the `Publisher` of a [[akka.stream.scaladsl2.PublisherSink]]. * - * @tparam D materialized type of the given Drain + * @tparam S materialized type of the given Sink */ - def runWith[D](drain: DrainWithKey[Out, D], materializer: FlowMaterializer): D + def runWith[S](sink: KeyedSink[Out, S], materializer: FlowMaterializer): S /** - * Connect this `Source` to a `Drain` and run it. The returned value is the materialized value - * of the `Drain`, e.g. the `Publisher` of a [[akka.stream.scaladsl2.PublisherDrain]]. + * Connect this `Source` to a `Sink` and run it. The returned value is the materialized value + * of the `Sink`, e.g. the `Publisher` of a [[akka.stream.scaladsl2.PublisherSink]]. */ - def runWith(drain: SimpleDrain[Out], materializer: FlowMaterializer): Unit + def runWith(sink: SimpleSink[Out], materializer: FlowMaterializer): Unit /** * Shortcut for running this `Source` with a fold function. @@ -123,7 +123,7 @@ object Source { * steps. */ def from[O](iterator: java.util.Iterator[O]): javadsl.Source[O] = - SourceAdapter(scaladsl2.IteratorTap(iterator.asScala)) + SourceAdapter(scaladsl2.Source(iterator.asScala)) /** * Java API @@ -176,18 +176,8 @@ object Source { /** INTERNAL API */ private[akka] object SourceAdapter { - def apply[O](tap: scaladsl2.Tap[O]): javadsl.Source[O] = - new SourceAdapter[O] { def delegate = scaladsl2.Pipe.empty[O].withTap(tap) } - def apply[O](source: scaladsl2.Source[O]): javadsl.Source[O] = - source match { - case pipe: scaladsl2.SourcePipe[O] ⇒ apply(pipe) - case _ ⇒ apply(source.asInstanceOf[scaladsl2.Tap[O]]) - } - - def apply[O](pipe: scaladsl2.SourcePipe[O]): javadsl.Source[O] = - new SourceAdapter[O] { def delegate = pipe } - + new SourceAdapter[O] { def delegate = source } } /** INTERNAL API */ @@ -209,20 +199,20 @@ private[akka] abstract class SourceAdapter[+Out] extends Source[Out] { override def connect(sink: javadsl.Sink[Out]): javadsl.RunnableFlow = new RunnableFlowAdapter(delegate.connect(sink.asScala)) - override def runWith[D](drain: DrainWithKey[Out, D], materializer: FlowMaterializer): D = - asScala.runWith(drain.asScala)(materializer).asInstanceOf[D] + override def runWith[D](sink: KeyedSink[Out, D], materializer: FlowMaterializer): D = + asScala.runWith(sink.asScala)(materializer).asInstanceOf[D] - override def runWith(drain: SimpleDrain[Out], materializer: FlowMaterializer): Unit = - delegate.connect(drain.asScala).run()(materializer) + override def runWith(sink: SimpleSink[Out], materializer: FlowMaterializer): Unit = + delegate.connect(sink.asScala).run()(materializer) override def fold[U](zero: U, f: japi.Function2[U, Out, U], materializer: FlowMaterializer): Future[U] = - runWith(FoldDrain.create(zero, f), materializer) + runWith(Sink.fold(zero, f), materializer) override def concat[Out2 >: Out](second: javadsl.Source[Out2]): javadsl.Source[Out2] = delegate.concat(second.asScala).asJava override def foreach(f: japi.Procedure[Out], materializer: FlowMaterializer): Future[Unit] = - runWith(ForeachDrain.create(f), materializer) + runWith(Sink.foreach(f), materializer) // COMMON OPS // diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/StreamOps.scala b/akka-stream/src/main/scala/akka/stream/javadsl/StreamOps.scala index f4307c307c..9f30b87c92 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/StreamOps.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/StreamOps.scala @@ -17,40 +17,40 @@ trait FlowOps[-In, +Out] { // RUN WITH // /** - * Connect the `Tap` to this `Flow` and then connect it to the `Drain` and run it. + * Connect the `Source` to this `Flow` and then connect it to the `Sink` and run it. * - * The returned tuple contains the materialized values of the `Tap` and `Drain`, - * e.g. the `Subscriber` of a [[SubscriberTap]] and `Publisher` of a [[PublisherDrain]]. + * The returned tuple contains the materialized values of the `Source` and `Sink`, + * e.g. the `Subscriber` of a [[SubscriberSource]] and `Publisher` of a [[PublisherSink]]. * - * @tparam T materialized type of given Tap - * @tparam D materialized type of given Drain + * @tparam T materialized type of given Source + * @tparam U materialized type of given Sink */ - def runWith[T, D](tap: javadsl.TapWithKey[In, T], drain: javadsl.DrainWithKey[Out, D], materializer: FlowMaterializer): akka.japi.Pair[T, D] + def runWith[T, U](source: javadsl.KeyedSource[In, T], sink: javadsl.KeyedSink[Out, U], materializer: FlowMaterializer): akka.japi.Pair[T, U] /** - * Connect the `Tap` to this `Flow` and then connect it to the `Drain` and run it. + * Connect the `Source` to this `Flow and then connect it to the `Sink` and run it. * - * The returned value will contain the materialized value of the `DrainWithKey`, e.g. `Publisher` of a [[PublisherDrain]]. + * The returned value will contain the materialized value of the `KeyedSink`, e.g. `Publisher` of a [[PublisherSink]]. * - * @tparam D materialized type of given Drain + * @tparam T materialized type of given Sink */ - def runWith[D](tap: javadsl.SimpleTap[In], drain: javadsl.DrainWithKey[Out, D], materializer: FlowMaterializer): D + def runWith[T](source: javadsl.SimpleSource[In], sink: javadsl.KeyedSink[Out, T], materializer: FlowMaterializer): T /** - * Connect the `Tap` to this `Flow` and then connect it to the `Drain` and run it. + * Connect the `Source` to this `Flow` and then connect it to the `Sink` and run it. * - * The returned value will contain the materialized value of the `TapWithKey`, e.g. `Subscriber` of a [[SubscriberTap]]. + * The returned value will contain the materialized value of the `KeyedSource`, e.g. `Subscriber` of a [[SubscriberSource]]. * - * @tparam T materialized type of given Tap + * @tparam T materialized type of given Source */ - def runWith[T](tap: javadsl.TapWithKey[In, T], drain: javadsl.SimpleDrain[Out], materializer: FlowMaterializer): T + def runWith[T](source: javadsl.KeyedSource[In, T], sink: javadsl.SimpleSink[Out], materializer: FlowMaterializer): T /** - * Connect the `Tap` to this `Flow` and then connect it to the `Drain` and run it. + * Connect the `Source` to this `Flow` and then connect it to the `Sink` and run it. * - * As both `Tap` and `Drain` are "simple", no value is returned from this `runWith` overload. + * As both `Source` and `Sink` are "simple", no value is returned from this `runWith` overload. */ - def runWith(tap: javadsl.SimpleTap[In], drain: javadsl.SimpleDrain[Out], materializer: FlowMaterializer): Unit + def runWith(source: javadsl.SimpleSource[In], sink: javadsl.SimpleSink[Out], materializer: FlowMaterializer): Unit // COMMON OPS // @@ -291,19 +291,19 @@ trait SourceOps[+Out] { // RUN WITH // /** - * Connect the `Sink` to this `Source` and then connect it to the `Tap` and run it. + * Connect the `Sink` to this `Source` and then connect it to the `Source` and run it. * - * The returned value the materialized values of the `Tap` and `Drain`, e.g. the `Subscriber` of a - * [[akka.stream.scaladsl2.SubscriberTap]] and and `Publisher` of a [[akka.stream.scaladsl2.PublisherDrain]]. + * The returned value the materialized values of the `Source` and `Sink`, e.g. the `Subscriber` of a + * [[akka.stream.scaladsl2.SubscriberSource]] and and `Publisher` of a [[akka.stream.scaladsl2.PublisherSink]]. * - * @tparam M materialized type of given Tap + * @tparam M materialized type of given Source */ - def runWith[M](drain: javadsl.DrainWithKey[Out, M], materializer: FlowMaterializer): M + def runWith[M](sink: javadsl.KeyedSink[Out, M], materializer: FlowMaterializer): M /** - * Connect this `Source` to a `Tap` and run it. + * Connect this `Source` to a `Source` and run it. */ - def runWith(drain: javadsl.SimpleDrain[Out], materializer: FlowMaterializer): Unit + def runWith(sink: javadsl.SimpleSink[Out], materializer: FlowMaterializer): Unit // COMMON OPS // @@ -544,17 +544,17 @@ trait SinkOps[-In] { // RUN WITH // /** - * Connect the `Tap` to this `Flow` and then connect it to the `Tap` and run it. - * The returned tuple contains the materialized values of the `Tap` and `Drain`, e.g. the `Subscriber` of a - * [[akka.stream.scaladsl2.SubscriberTap]] and and `Publisher` of a [[akka.stream.scaladsl2.PublisherDrain]]. + * Connect the `Source` to this `Flow` and then connect it to the `Source` and run it. + * The returned tuple contains the materialized values of the `Source` and `Sink`, e.g. the `Subscriber` of a + * [[akka.stream.scaladsl2.SubscriberSource]] and and `Publisher` of a [[akka.stream.scaladsl2.PublisherSink]]. * - * @tparam T materialized type of given Tap + * @tparam T materialized type of given Source */ - def runWith[T](tap: javadsl.TapWithKey[In, T], materializer: FlowMaterializer): T + def runWith[T](source: javadsl.KeyedSource[In, T], materializer: FlowMaterializer): T /** - * Connect this `Source` to a `Tap` and run it. The returned value is the materialized value - * of the `Drain`, e.g. the `Publisher` of a [[akka.stream.scaladsl2.PublisherDrain]]. + * Connect this `Source` to a `Source` and run it. The returned value is the materialized value + * of the `Sink`, e.g. the `Publisher` of a [[akka.stream.scaladsl2.PublisherSink]]. */ - def runWith(tap: javadsl.SimpleTap[In], materializer: FlowMaterializer): Unit + def runWith(source: javadsl.SimpleSource[In], materializer: FlowMaterializer): Unit } \ No newline at end of file diff --git a/akka-stream/src/main/scala/akka/stream/javadsl/Tap.scala b/akka-stream/src/main/scala/akka/stream/javadsl/Tap.scala deleted file mode 100644 index 2d9a5e32e2..0000000000 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Tap.scala +++ /dev/null @@ -1,63 +0,0 @@ -/** - * Copyright (C) 2014 Typesafe Inc. - */ -package akka.stream.javadsl - -import akka.stream.javadsl -import akka.stream.scaladsl2 - -import org.reactivestreams.{ Publisher, Subscriber } - -import concurrent.duration.FiniteDuration -import scala.concurrent.Future - -abstract class Tap[+Out] extends javadsl.SourceAdapter[Out] { - def delegate: scaladsl2.Source[Out] -} - -abstract class SimpleTap[+Out] extends Tap[Out] { - override def asScala: scaladsl2.SimpleTap[Out] = super.asScala.asInstanceOf[scaladsl2.SimpleTap[Out]] -} - -abstract class TapWithKey[+Out, T] extends Tap[Out] { - override def asScala: scaladsl2.TapWithKey[Out] = super.asScala.asInstanceOf[scaladsl2.TapWithKey[Out]] -} - -// adapters // - -object SubscriberTap { - def create[O]() = new SubscriberTap(new scaladsl2.SubscriberTap[O]) -} -final case class SubscriberTap[O](delegate: scaladsl2.SubscriberTap[O]) extends javadsl.TapWithKey[O, Subscriber[O]] - -object PublisherTap { - def create[O](p: Publisher[O]) = new PublisherTap(new scaladsl2.PublisherTap[O](p)) -} -final case class PublisherTap[O](delegate: scaladsl2.PublisherTap[O]) extends javadsl.TapWithKey[O, Publisher[O]] - -object IteratorTap { - import collection.JavaConverters._ - def create[O](iterator: java.util.Iterator[O]) = new IteratorTap(new scaladsl2.IteratorTap[O](iterator.asScala)) -} -final case class IteratorTap[O](delegate: scaladsl2.IteratorTap[O]) extends javadsl.SimpleTap[O] - -object IterableTap { - def create[O](iterable: java.lang.Iterable[O]) = new IterableTap(new scaladsl2.IterableTap[O](akka.stream.javadsl.japi.Util.immutableIterable(iterable))) -} -final case class IterableTap[O](delegate: scaladsl2.IterableTap[O]) extends javadsl.SimpleTap[O] - -object ThunkTap { - def create[O](f: japi.Creator[akka.japi.Option[O]]) = new ThunkTap(new scaladsl2.ThunkTap[O](() ⇒ f.create())) -} -final case class ThunkTap[O](delegate: scaladsl2.ThunkTap[O]) extends javadsl.SimpleTap[O] - -object FutureTap { - def create[O](future: Future[O]) = new FutureTap(new scaladsl2.FutureTap[O](future)) -} -final case class FutureTap[O](delegate: scaladsl2.FutureTap[O]) extends javadsl.SimpleTap[O] - -object TickTap { - def create[O](initialDelay: FiniteDuration, interval: FiniteDuration, tick: japi.Creator[O]) = - new TickTap(new scaladsl2.TickTap[O](initialDelay, interval, () ⇒ tick.create())) -} -final case class TickTap[O](delegate: scaladsl2.TickTap[O]) extends javadsl.SimpleTap[O] diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl2/Drain.scala b/akka-stream/src/main/scala/akka/stream/scaladsl2/ActorFlowSink.scala similarity index 58% rename from akka-stream/src/main/scala/akka/stream/scaladsl2/Drain.scala rename to akka-stream/src/main/scala/akka/stream/scaladsl2/ActorFlowSink.scala index e3904fc47a..fa4462d450 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl2/Drain.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl2/ActorFlowSink.scala @@ -15,62 +15,12 @@ import akka.stream.impl.{ FanoutProcessorImpl, BlackholeSubscriber } import akka.stream.impl2.{ ActorProcessorFactory, ActorBasedFlowMaterializer } import java.util.concurrent.atomic.AtomicReference -/** - * This trait is a marker for a pluggable stream drain. Concrete instances should - * implement [[DrainWithKey]] or [[SimpleDrain]], otherwise a custom [[FlowMaterializer]] - * will have to be used to be able to attach them. - * - * All Drains defined in this package rely upon an [[akka.stream.impl2.ActorBasedFlowMaterializer]] being - * made available to them in order to use the attach method. Other - * FlowMaterializers can be used but must then implement the functionality of these - * Drain nodes themselves (or construct an ActorBasedFlowMaterializer). - */ -trait Drain[-In] extends Sink[In] - -/** - * A drain that does not need to create a user-accessible object during materialization. - */ -trait SimpleDrain[-In] extends Drain[In] { - /** - * Attach this drain to the given [[org.reactivestreams.Publisher]]. Using the given - * [[FlowMaterializer]] is completely optional, especially if this drain belongs to - * a different Reactive Streams implementation. It is the responsibility of the - * caller to provide a suitable FlowMaterializer that can be used for running - * Flows if necessary. - * - * @param flowPublisher the Publisher to consume elements from - * @param materializer a FlowMaterializer that may be used for creating flows - * @param flowName the name of the current flow, which should be used in log statements or error messages - */ - def attach(flowPublisher: Publisher[In @uncheckedVariance], materializer: ActorBasedFlowMaterializer, flowName: String): Unit - - /** - * This method is only used for Drains that return true from [[#isActive]], which then must - * implement it. - */ - def create(materializer: ActorBasedFlowMaterializer, flowName: String): Subscriber[In] @uncheckedVariance = - throw new UnsupportedOperationException(s"forgot to implement create() for $getClass that says isActive==true") - - /** - * This method indicates whether this Drain can create a Subscriber instead of being - * attached to a Publisher. This is only used if the Flow does not contain any - * operations. - */ - def isActive: Boolean = false -} - -/** - * A drain that will create an object during materialization that the user will need - * to retrieve in order to access aspects of this drain (could be a completion Future - * or a cancellation handle, etc.) - */ -trait DrainWithKey[-In] extends Drain[In] { - +sealed trait ActorFlowSink[-In] extends Sink[In] { type MaterializedType /** - * Attach this drain to the given [[org.reactivestreams.Publisher]]. Using the given - * [[FlowMaterializer]] is completely optional, especially if this drain belongs to + * Attach this sink to the given [[org.reactivestreams.Publisher]]. Using the given + * [[FlowMaterializer]] is completely optional, especially if this sink belongs to * a different Reactive Streams implementation. It is the responsibility of the * caller to provide a suitable FlowMaterializer that can be used for running * Flows if necessary. @@ -82,14 +32,14 @@ trait DrainWithKey[-In] extends Drain[In] { def attach(flowPublisher: Publisher[In @uncheckedVariance], materializer: ActorBasedFlowMaterializer, flowName: String): MaterializedType /** - * This method is only used for Drains that return true from [[#isActive]], which then must + * This method is only used for Sinks that return true from [[#isActive]], which then must * implement it. */ def create(materializer: ActorBasedFlowMaterializer, flowName: String): (Subscriber[In] @uncheckedVariance, MaterializedType) = throw new UnsupportedOperationException(s"forgot to implement create() for $getClass that says isActive==true") /** - * This method indicates whether this Drain can create a Subscriber instead of being + * This method indicates whether this Sink can create a Subscriber instead of being * attached to a Publisher. This is only used if the Flow does not contain any * operations. */ @@ -100,31 +50,45 @@ trait DrainWithKey[-In] extends Drain[In] { final override def hashCode: Int = super.hashCode } +/** + * A sink that does not need to create a user-accessible object during materialization. + */ +trait SimpleActorFlowSink[-In] extends ActorFlowSink[In] { + override type MaterializedType = Unit +} + +/** + * A sink that will create an object during materialization that the user will need + * to retrieve in order to access aspects of this sink (could be a completion Future + * or a cancellation handle, etc.) + */ +trait KeyedActorFlowSink[-In] extends ActorFlowSink[In] with KeyedSink[In] + /** * Holds the downstream-most [[org.reactivestreams.Publisher]] interface of the materialized flow. * 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 subscriber connects and creates demand for elements to be emitted. */ -object PublisherDrain { - private val instance = new PublisherDrain[Nothing] - def apply[T](): PublisherDrain[T] = instance.asInstanceOf[PublisherDrain[T]] - def withFanout[T](initialBufferSize: Int, maximumBufferSize: Int): FanoutPublisherDrain[T] = - new FanoutPublisherDrain[T](initialBufferSize, maximumBufferSize) +private[scaladsl2] object PublisherSink { + private val instance = new PublisherSink[Nothing] + def apply[T](): PublisherSink[T] = instance.asInstanceOf[PublisherSink[T]] + def withFanout[T](initialBufferSize: Int, maximumBufferSize: Int): FanoutPublisherSink[T] = + new FanoutPublisherSink[T](initialBufferSize, maximumBufferSize) } -class PublisherDrain[In] extends DrainWithKey[In] { +private[scaladsl2] class PublisherSink[In] extends KeyedActorFlowSink[In] { type MaterializedType = Publisher[In] - def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[In] = flowPublisher + override def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String) = flowPublisher - override def toString: String = "PublisherDrain" + override def toString: String = "PublisherSink" } -final case class FanoutPublisherDrain[In](initialBufferSize: Int, maximumBufferSize: Int) extends DrainWithKey[In] { +private[scaladsl2] final case class FanoutPublisherSink[In](initialBufferSize: Int, maximumBufferSize: Int) extends KeyedActorFlowSink[In] { type MaterializedType = Publisher[In] - override def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[In] = { + override def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String) = { val fanoutActor = materializer.actorOf( Props(new FanoutProcessorImpl(materializer.settings, initialBufferSize, maximumBufferSize)), s"$flowName-fanoutPublisher") val fanoutProcessor = ActorProcessorFactory[In, In](fanoutActor) @@ -133,8 +97,8 @@ final case class FanoutPublisherDrain[In](initialBufferSize: Int, maximumBufferS } } -object FutureDrain { - def apply[T](): FutureDrain[T] = new FutureDrain[T] +private[scaladsl2] object FutureSink { + def apply[T](): FutureSink[T] = new FutureSink[T] } /** @@ -144,17 +108,17 @@ object FutureDrain { * the Future into the corresponding failed state) or the end-of-stream * (failing the Future with a NoSuchElementException). */ -class FutureDrain[In] extends DrainWithKey[In] { +private[scaladsl2] class FutureSink[In] extends KeyedActorFlowSink[In] { type MaterializedType = Future[In] - def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String): Future[In] = { + def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String) = { val (sub, f) = create(materializer, flowName) flowPublisher.subscribe(sub) f } override def isActive = true - override def create(materializer: ActorBasedFlowMaterializer, flowName: String): (Subscriber[In], Future[In]) = { + override def create(materializer: ActorBasedFlowMaterializer, flowName: String) = { val p = Promise[In]() val sub = new Subscriber[In] { // TODO #15804 verify this using the RS TCK private val sub = new AtomicReference[Subscription] @@ -168,32 +132,32 @@ class FutureDrain[In] extends DrainWithKey[In] { (sub, p.future) } - override def toString: String = "FutureDrain" + override def toString: String = "FutureSink" } /** * Attaches a subscriber to this stream which will just discard all received * elements. */ -final case object BlackholeDrain extends SimpleDrain[Any] { +private[scaladsl2] final case object BlackholeSink extends SimpleActorFlowSink[Any] { override def attach(flowPublisher: Publisher[Any], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = - flowPublisher.subscribe(create(materializer, flowName)) + flowPublisher.subscribe(create(materializer, flowName)._1) override def isActive: Boolean = true - override def create(materializer: ActorBasedFlowMaterializer, flowName: String): Subscriber[Any] = - new BlackholeSubscriber[Any](materializer.settings.maxInputBufferSize) + override def create(materializer: ActorBasedFlowMaterializer, flowName: String) = + (new BlackholeSubscriber[Any](materializer.settings.maxInputBufferSize), ()) } /** * Attaches a subscriber to this stream. */ -final case class SubscriberDrain[In](subscriber: Subscriber[In]) extends SimpleDrain[In] { - override def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = +private[scaladsl2] final case class SubscriberSink[In](subscriber: Subscriber[In]) extends SimpleActorFlowSink[In] { + override def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String) = flowPublisher.subscribe(subscriber) override def isActive: Boolean = true - override def create(materializer: ActorBasedFlowMaterializer, flowName: String): Subscriber[In] = subscriber + override def create(materializer: ActorBasedFlowMaterializer, flowName: String) = (subscriber, ()) } -object OnCompleteDrain { +private[scaladsl2] object OnCompleteSink { private val SuccessUnit = Success[Unit](()) } @@ -202,32 +166,32 @@ object OnCompleteDrain { * completion, apply the provided function with [[scala.util.Success]] * or [[scala.util.Failure]]. */ -final case class OnCompleteDrain[In](callback: Try[Unit] ⇒ Unit) extends SimpleDrain[In] { +private[scaladsl2] final case class OnCompleteSink[In](callback: Try[Unit] ⇒ Unit) extends SimpleActorFlowSink[In] { - override def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = - Source(flowPublisher).transform("onCompleteDrain", () ⇒ new Transformer[In, Unit] { + override def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String) = + Source(flowPublisher).transform("onCompleteSink", () ⇒ new Transformer[In, Unit] { override def onNext(in: In) = Nil override def onError(e: Throwable) = () override def onTermination(e: Option[Throwable]) = { e match { - case None ⇒ callback(OnCompleteDrain.SuccessUnit) + case None ⇒ callback(OnCompleteSink.SuccessUnit) case Some(e) ⇒ callback(Failure(e)) } Nil } - }).connect(BlackholeDrain).run()(materializer.withNamePrefix(flowName)) + }).connect(BlackholeSink).run()(materializer.withNamePrefix(flowName)) } /** - * Invoke the given procedure for each received element. The drain holds a [[scala.concurrent.Future]] + * Invoke the given procedure for each received element. The sink holds a [[scala.concurrent.Future]] * that will be completed with `Success` when reaching the normal end of the stream, or completed * with `Failure` if there is an error is signaled in the stream. */ -final case class ForeachDrain[In](f: In ⇒ Unit) extends DrainWithKey[In] { +private[scaladsl2] final case class ForeachSink[In](f: In ⇒ Unit) extends KeyedActorFlowSink[In] { - type MaterializedType = Future[Unit] + override type MaterializedType = Future[Unit] - override def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String): Future[Unit] = { + override def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String) = { val promise = Promise[Unit]() Source(flowPublisher).transform("foreach", () ⇒ new Transformer[In, Unit] { override def onNext(in: In) = { f(in); Nil } @@ -239,23 +203,23 @@ final case class ForeachDrain[In](f: In ⇒ Unit) extends DrainWithKey[In] { } Nil } - }).connect(BlackholeDrain).run()(materializer.withNamePrefix(flowName)) + }).connect(BlackholeSink).run()(materializer.withNamePrefix(flowName)) promise.future } } /** * Invoke the given function for every received element, giving it its previous - * output (or the given `zero` value) and the element as input. The drain holds a + * output (or the given `zero` value) and the element as input. The sink holds a * [[scala.concurrent.Future]] that will be completed with value of the final * function evaluation when the input stream ends, or completed with `Failure` * if there is an error is signaled in the stream. */ -final case class FoldDrain[U, In](zero: U)(f: (U, In) ⇒ U) extends DrainWithKey[In] { +private[scaladsl2] final case class FoldSink[U, In](zero: U)(f: (U, In) ⇒ U) extends KeyedActorFlowSink[In] { type MaterializedType = Future[U] - override def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String): Future[U] = { + override def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String) = { val promise = Promise[U]() Source(flowPublisher).transform("fold", () ⇒ new Transformer[In, U] { @@ -269,16 +233,16 @@ final case class FoldDrain[U, In](zero: U)(f: (U, In) ⇒ U) extends DrainWithKe } Nil } - }).connect(BlackholeDrain).run()(materializer.withNamePrefix(flowName)) + }).connect(BlackholeSink).run()(materializer.withNamePrefix(flowName)) promise.future } } /** - * A drain that immediately cancels its upstream upon materialization. + * A sink that immediately cancels its upstream upon materialization. */ -final case object CancelDrain extends SimpleDrain[Any] { +private[scaladsl2] final case object CancelSink extends SimpleActorFlowSink[Any] { override def attach(flowPublisher: Publisher[Any], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = { flowPublisher.subscribe(new Subscriber[Any] { @@ -289,4 +253,3 @@ final case object CancelDrain extends SimpleDrain[Any] { }) } } - diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl2/Tap.scala b/akka-stream/src/main/scala/akka/stream/scaladsl2/ActorFlowSource.scala similarity index 51% rename from akka-stream/src/main/scala/akka/stream/scaladsl2/Tap.scala rename to akka-stream/src/main/scala/akka/stream/scaladsl2/ActorFlowSource.scala index ea9e1f45f7..5b54a9416e 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl2/Tap.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl2/ActorFlowSource.scala @@ -16,77 +16,12 @@ import scala.concurrent.duration.FiniteDuration import scala.util.Failure import scala.util.Success -/** - * This trait is a marker for a pluggable stream tap. Concrete instances should - * implement [[TapWithKey]] or [[SimpleTap]], otherwise a custom [[FlowMaterializer]] - * will have to be used to be able to attach them. - * - * All Taps defined in this package rely upon an ActorBasedFlowMaterializer being - * made available to them in order to use the attach method. Other - * FlowMaterializers can be used but must then implement the functionality of these - * Tap nodes themselves (or construct an ActorBasedFlowMaterializer). - */ -trait Tap[+Out] extends Source[Out] { - override type Repr[+O] = SourcePipe[O] - - private def sourcePipe = Pipe.empty[Out].withTap(this) - - override def connect[T](flow: Flow[Out, T]): Source[T] = sourcePipe.connect(flow) - - override def connect(sink: Sink[Out]): RunnableFlow = sourcePipe.connect(sink) - - /** INTERNAL API */ - override private[scaladsl2] def andThen[U](op: AstNode) = SourcePipe(this, List(op)) -} - -/** - * A tap that does not need to create a user-accessible object during materialization. - */ -trait SimpleTap[+Out] extends Tap[Out] { - /** - * Attach this tap to the given [[org.reactivestreams.Subscriber]]. Using the given - * [[FlowMaterializer]] is completely optional, especially if this tap belongs to - * a different Reactive Streams implementation. It is the responsibility of the - * caller to provide a suitable FlowMaterializer that can be used for running - * Flows if necessary. - * - * @param flowSubscriber the Subscriber to produce elements to - * @param materializer a FlowMaterializer that may be used for creating flows - * @param flowName the name of the current flow, which should be used in log statements or error messages - */ - def attach(flowSubscriber: Subscriber[Out] @uncheckedVariance, materializer: ActorBasedFlowMaterializer, flowName: String): Unit - - /** - * This method is only used for Taps that return true from [[#isActive]], which then must - * implement it. - */ - def create(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[Out] @uncheckedVariance = - throw new UnsupportedOperationException(s"forgot to implement create() for $getClass that says isActive==true") - - /** - * This method indicates whether this Tap can create a Publisher instead of being - * attached to a Subscriber. This is only used if the Flow does not contain any - * operations. - */ - def isActive: Boolean = false - - // these are unique keys, case class equality would break them - final override def equals(other: Any): Boolean = super.equals(other) - final override def hashCode: Int = super.hashCode -} - -/** - * A tap that will create an object during materialization that the user will need - * to retrieve in order to access aspects of this tap (could be a Subscriber, a - * Future/Promise, etc.). - */ -trait TapWithKey[+Out] extends Tap[Out] { - +sealed trait ActorFlowSource[+Out] extends Source[Out] { type MaterializedType /** - * Attach this tap to the given [[org.reactivestreams.Subscriber]]. Using the given - * [[FlowMaterializer]] is completely optional, especially if this tap belongs to + * Attach this source to the given [[org.reactivestreams.Subscriber]]. Using the given + * [[FlowMaterializer]] is completely optional, especially if this source belongs to * a different Reactive Streams implementation. It is the responsibility of the * caller to provide a suitable FlowMaterializer that can be used for running * Flows if necessary. @@ -98,14 +33,14 @@ trait TapWithKey[+Out] extends Tap[Out] { def attach(flowSubscriber: Subscriber[Out] @uncheckedVariance, materializer: ActorBasedFlowMaterializer, flowName: String): MaterializedType /** - * This method is only used for Taps that return true from [[#isActive]], which then must + * This method is only used for Sources that return true from [[#isActive]], which then must * implement it. */ def create(materializer: ActorBasedFlowMaterializer, flowName: String): (Publisher[Out] @uncheckedVariance, MaterializedType) = throw new UnsupportedOperationException(s"forgot to implement create() for $getClass that says isActive==true") /** - * This method indicates whether this Tap can create a Publisher instead of being + * This method indicates whether this Source can create a Publisher instead of being * attached to a Subscriber. This is only used if the Flow does not contain any * operations. */ @@ -114,18 +49,42 @@ trait TapWithKey[+Out] extends Tap[Out] { // these are unique keys, case class equality would break them final override def equals(other: Any): Boolean = super.equals(other) final override def hashCode: Int = super.hashCode + + override type Repr[+O] = SourcePipe[O] + + private def sourcePipe = Pipe.empty[Out].withSource(this) + + override def connect[T](flow: Flow[Out, T]): Source[T] = sourcePipe.connect(flow) + + override def connect(sink: Sink[Out]): RunnableFlow = sourcePipe.connect(sink) + + /** INTERNAL API */ + override private[scaladsl2] def andThen[U](op: AstNode) = SourcePipe(this, List(op)) } +/** + * A source that does not need to create a user-accessible object during materialization. + */ +trait SimpleActorFlowSource[+Out] extends ActorFlowSource[Out] { + override type MaterializedType = Unit +} + +/** + * A source that will create an object during materialization that the user will need + * to retrieve in order to access aspects of this source (could be a Subscriber, a + * Future/Promise, etc.). + */ +trait KeyedActorFlowSource[+Out] extends ActorFlowSource[Out] with KeyedSource[Out] + /** * Holds a `Subscriber` representing the input side of the flow. * The `Subscriber` can later be connected to an upstream `Publisher`. */ -final case class SubscriberTap[Out]() extends TapWithKey[Out] { - type MaterializedType = Subscriber[Out] +private[scaladsl2] final case class SubscriberSource[Out]() extends KeyedActorFlowSource[Out] { + override type MaterializedType = Subscriber[Out] override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String): Subscriber[Out] = flowSubscriber - } /** @@ -134,11 +93,11 @@ final case class SubscriberTap[Out]() extends TapWithKey[Out] { * that mediate the flow of elements downstream and the propagation of * back-pressure upstream. */ -final case class PublisherTap[Out](p: Publisher[Out]) extends SimpleTap[Out] { - override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = +private[scaladsl2] final case class PublisherSource[Out](p: Publisher[Out]) extends SimpleActorFlowSource[Out] { + override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String) = p.subscribe(flowSubscriber) override def isActive: Boolean = true - override def create(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[Out] = p + override def create(materializer: ActorBasedFlowMaterializer, flowName: String) = (p, ()) } /** @@ -148,14 +107,14 @@ final case class PublisherTap[Out](p: Publisher[Out]) extends SimpleTap[Out] { * in accordance with the demand coming from the downstream transformation * steps. */ -final case class IteratorTap[Out](iterator: Iterator[Out]) extends SimpleTap[Out] { - override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = - create(materializer, flowName).subscribe(flowSubscriber) +private[scaladsl2] final case class IteratorSource[Out](iterator: Iterator[Out]) extends SimpleActorFlowSource[Out] { + override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String) = + create(materializer, flowName)._1.subscribe(flowSubscriber) override def isActive: Boolean = true - override def create(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[Out] = - if (iterator.isEmpty) EmptyPublisher[Out] - else ActorPublisher[Out](materializer.actorOf(IteratorPublisher.props(iterator, materializer.settings), - name = s"$flowName-0-iterator")) + override def create(materializer: ActorBasedFlowMaterializer, flowName: String) = + if (iterator.isEmpty) (EmptyPublisher[Out], ()) + else (ActorPublisher[Out](materializer.actorOf(IteratorPublisher.props(iterator, materializer.settings), + name = s"$flowName-0-iterator")), ()) } /** @@ -164,14 +123,14 @@ final case class IteratorTap[Out](iterator: Iterator[Out]) extends SimpleTap[Out * stream will see an individual flow of elements (always starting from the * beginning) regardless of when they subscribed. */ -final case class IterableTap[Out](iterable: immutable.Iterable[Out]) extends SimpleTap[Out] { - override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = - create(materializer, flowName).subscribe(flowSubscriber) +private[scaladsl2] final case class IterableSource[Out](iterable: immutable.Iterable[Out]) extends SimpleActorFlowSource[Out] { + override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String) = + create(materializer, flowName)._1.subscribe(flowSubscriber) override def isActive: Boolean = true - override def create(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[Out] = - if (iterable.isEmpty) EmptyPublisher[Out] - else ActorPublisher[Out](materializer.actorOf(IterablePublisher.props(iterable, materializer.settings), - name = s"$flowName-0-iterable"), Some(iterable)) + override def create(materializer: ActorBasedFlowMaterializer, flowName: String) = + if (iterable.isEmpty) (EmptyPublisher[Out], ()) + else (ActorPublisher[Out](materializer.actorOf(IterablePublisher.props(iterable, materializer.settings), + name = s"$flowName-0-iterable"), Some(iterable)), ()) } /** @@ -179,16 +138,16 @@ final case class IterableTap[Out](iterable: immutable.Iterable[Out]) extends Sim * The stream ends normally when evaluation of the closure returns a `None`. * The stream ends exceptionally when an exception is thrown from the closure. */ -final case class ThunkTap[Out](f: () ⇒ Option[Out]) extends SimpleTap[Out] { - override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = - create(materializer, flowName).subscribe(flowSubscriber) +private[scaladsl2] final case class ThunkSource[Out](f: () ⇒ Option[Out]) extends SimpleActorFlowSource[Out] { + override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String) = + create(materializer, flowName)._1.subscribe(flowSubscriber) override def isActive: Boolean = true - override def create(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[Out] = - ActorPublisher[Out](materializer.actorOf(SimpleCallbackPublisher.props(materializer.settings, + override def create(materializer: ActorBasedFlowMaterializer, flowName: String) = + (ActorPublisher[Out](materializer.actorOf(SimpleCallbackPublisher.props(materializer.settings, () ⇒ f() match { case Some(out) ⇒ out case _ ⇒ throw Stop - }), name = s"$flowName-0-thunk")) + }), name = s"$flowName-0-thunk")), ()) } /** @@ -197,20 +156,20 @@ final case class ThunkTap[Out](f: () ⇒ Option[Out]) extends SimpleTap[Out] { * may happen before or after materializing the `Flow`. * The stream terminates with an error if the `Future` is completed with a failure. */ -final case class FutureTap[Out](future: Future[Out]) extends SimpleTap[Out] { - override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = - create(materializer, flowName).subscribe(flowSubscriber) +private[scaladsl2] final case class FutureSource[Out](future: Future[Out]) extends SimpleActorFlowSource[Out] { + override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String) = + create(materializer, flowName)._1.subscribe(flowSubscriber) override def isActive: Boolean = true - override def create(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[Out] = + override def create(materializer: ActorBasedFlowMaterializer, flowName: String) = future.value match { case Some(Success(element)) ⇒ - ActorPublisher[Out](materializer.actorOf(IterablePublisher.props(List(element), materializer.settings), - name = s"$flowName-0-future"), Some(future)) + (ActorPublisher[Out](materializer.actorOf(IterablePublisher.props(List(element), materializer.settings), + name = s"$flowName-0-future"), Some(future)), ()) case Some(Failure(t)) ⇒ - ErrorPublisher(t).asInstanceOf[Publisher[Out]] + (ErrorPublisher(t).asInstanceOf[Publisher[Out]], ()) case None ⇒ - ActorPublisher[Out](materializer.actorOf(FuturePublisher.props(future, materializer.settings), - name = s"$flowName-0-future"), Some(future)) + (ActorPublisher[Out](materializer.actorOf(FuturePublisher.props(future, materializer.settings), + name = s"$flowName-0-future"), Some(future)), ()) } } @@ -221,29 +180,29 @@ final case class FutureTap[Out](future: Future[Out]) extends SimpleTap[Out] { * 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. */ -final case class TickTap[Out](initialDelay: FiniteDuration, interval: FiniteDuration, tick: () ⇒ Out) extends SimpleTap[Out] { - override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = - create(materializer, flowName).subscribe(flowSubscriber) +private[scaladsl2] final case class TickSource[Out](initialDelay: FiniteDuration, interval: FiniteDuration, tick: () ⇒ Out) extends SimpleActorFlowSource[Out] { + override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String) = + create(materializer, flowName)._1.subscribe(flowSubscriber) override def isActive: Boolean = true - override def create(materializer: ActorBasedFlowMaterializer, flowName: String): Publisher[Out] = - ActorPublisher[Out](materializer.actorOf(TickPublisher.props(initialDelay, interval, tick, materializer.settings), - name = s"$flowName-0-tick")) + override def create(materializer: ActorBasedFlowMaterializer, flowName: String) = + (ActorPublisher[Out](materializer.actorOf(TickPublisher.props(initialDelay, interval, tick, materializer.settings), + name = s"$flowName-0-tick")), ()) } /** - * This tap takes two Sources and concatenates them together by draining the elements coming from the first Source + * This Source takes two Sources and concatenates them together by draining the elements coming from the first Source * completely, then draining the elements arriving from the second Source. If the first Source is infinite then the * second Source will be never drained. */ -final case class ConcatTap[Out](source1: Source[Out], source2: Source[Out]) extends SimpleTap[Out] { +private[scaladsl2] final case class ConcatSource[Out](source1: Source[Out], source2: Source[Out]) extends SimpleActorFlowSource[Out] { - override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = { + override def attach(flowSubscriber: Subscriber[Out], materializer: ActorBasedFlowMaterializer, flowName: String) = { val concatter = Concat[Out] val concatGraph = FlowGraph { builder ⇒ builder .addEdge(source1, Pipe.empty[Out], concatter.first) .addEdge(source2, Pipe.empty[Out], concatter.second) - .addEdge(concatter.out, SubscriberDrain(flowSubscriber)) + .addEdge(concatter.out, Sink(flowSubscriber)) }.run()(materializer) } diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl2/Flow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl2/Flow.scala index e8f2853af2..7a4402b764 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl2/Flow.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl2/Flow.scala @@ -30,38 +30,38 @@ trait Flow[-In, +Out] extends FlowOps[Out] { /** * - * Connect the `Tap` to this `Flow` and then connect it to the `Drain` and run it. The returned tuple contains - * the materialized values of the `Tap` and `Drain`, e.g. the `Subscriber` of a [[SubscriberTap]] and - * and `Publisher` of a [[PublisherDrain]]. + * Connect the `Source` to this `Flow` and then connect it to the `Sink` and run it. The returned tuple contains + * the materialized values of the `Source` and `Sink`, e.g. the `Subscriber` of a [[SubscriberSource]] and + * and `Publisher` of a [[PublisherSink]]. */ - def runWith(tap: TapWithKey[In], drain: DrainWithKey[Out])(implicit materializer: FlowMaterializer): (tap.MaterializedType, drain.MaterializedType) = { - val m = tap.connect(this).connect(drain).run() - (m.materializedTap(tap), m.materializedDrain(drain)) + def runWith(source: KeyedSource[In], sink: KeyedSink[Out])(implicit materializer: FlowMaterializer): (source.MaterializedType, sink.MaterializedType) = { + val m = source.connect(this).connect(sink).run() + (m.get(source), m.get(sink)) } /** - * Connect the `Tap` to this `Flow` and then connect it to the `Drain` and run it. + * Connect the `Source` to this `Flow` and then connect it to the `Sink` and run it. * - * The returned value will contain the materialized value of the `DrainWithKey`, e.g. `Publisher` of a [[PublisherDrain]]. + * The returned value will contain the materialized value of the `KeyedSink`, e.g. `Publisher` of a [[PublisherSink]]. */ - def runWith(tap: SimpleTap[In], drain: DrainWithKey[Out])(implicit materializer: FlowMaterializer): drain.MaterializedType = - tap.connect(this).runWith(drain) + def runWith(source: Source[In], sink: KeyedSink[Out])(implicit materializer: FlowMaterializer): sink.MaterializedType = + source.connect(this).runWith(sink) /** - * Connect the `Tap` to this `Flow` and then connect it to the `Drain` and run it. + * Connect the `Source` to this `Flow` and then connect it to the `Sink` and run it. * - * The returned value will contain the materialized value of the `TapWithKey`, e.g. `Subscriber` of a [[SubscriberTap]]. + * The returned value will contain the materialized value of the `SourceWithKey`, e.g. `Subscriber` of a [[SubscriberSource]]. */ - def runWith(tap: TapWithKey[In], drain: SimpleDrain[Out])(implicit materializer: FlowMaterializer): tap.MaterializedType = - tap.connect(this).connect(drain).run().materializedTap(tap) + def runWith(source: KeyedSource[In], sink: Sink[Out])(implicit materializer: FlowMaterializer): source.MaterializedType = + source.connect(this).connect(sink).run().get(source) /** - * Connect the `Tap` to this `Flow` and then connect it to the `Drain` and run it. + * Connect the `Source` to this `Flow` and then connect it to the `Sink` and run it. * - * As both `Tap` and `Drain` are "simple", no value is returned from this `runWith` overload. + * As both `Source` and `Sink` are "simple", no value is returned from this `runWith` overload. */ - def runWith(tap: SimpleTap[In], drain: SimpleDrain[Out])(implicit materializer: FlowMaterializer): Unit = - tap.connect(this).connect(drain).run() + def runWith(source: Source[In], sink: Sink[Out])(implicit materializer: FlowMaterializer): Unit = + source.connect(this).connect(sink).run() } object Flow { diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl2/FlowGraph.scala b/akka-stream/src/main/scala/akka/stream/scaladsl2/FlowGraph.scala index e2f5af5ff2..008c47c727 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl2/FlowGraph.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl2/FlowGraph.scala @@ -73,8 +73,8 @@ object Merge { * Merge several streams, taking elements as they arrive from input streams * (picking randomly when several have elements ready). * - * When building the [[FlowGraph]] you must connect one or more input pipes/taps - * and one output pipe/sink to the `Merge` vertex. + * When building the [[FlowGraph]] you must connect one or more input sources + * and one output sink to the `Merge` vertex. */ final class Merge[T](override val name: Option[String]) extends FlowGraphInternal.InternalVertex with Junction[T] { override private[akka] val vertex = this @@ -119,8 +119,8 @@ object MergePreferred { * Merge several streams, taking elements as they arrive from input streams * (picking from preferred when several have elements ready). * - * When building the [[FlowGraph]] you must connect one or more input pipes/taps - * and one output pipe/drain to the `Merge` vertex. + * When building the [[FlowGraph]] you must connect one or more input sources + * and one output sink to the `Merge` vertex. */ final class MergePreferred[T](override val name: Option[String]) extends FlowGraphInternal.InternalVertex with Junction[T] { @@ -371,8 +371,8 @@ object UndefinedSink { } /** * It is possible to define a [[PartialFlowGraph]] with output pipes that are not connected - * yet by using this placeholder instead of the real [[Drain]]. Later the placeholder can - * be replaced with [[FlowGraphBuilder#attachDrain]]. + * yet by using this placeholder instead of the real [[Sink]]. Later the placeholder can + * be replaced with [[FlowGraphBuilder#attachSink]]. */ final class UndefinedSink[-T](override val name: Option[String]) extends FlowGraphInternal.InternalVertex { @@ -404,8 +404,8 @@ object UndefinedSource { } /** * It is possible to define a [[PartialFlowGraph]] with input pipes that are not connected - * yet by using this placeholder instead of the real [[Tap]]. Later the placeholder can - * be replaced with [[FlowGraphBuilder#attachTap]]. + * yet by using this placeholder instead of the real [[Source]]. Later the placeholder can + * be replaced with [[FlowGraphBuilder#attachSource]]. */ final class UndefinedSource[+T](override val name: Option[String]) extends FlowGraphInternal.InternalVertex { override def minimumInputCount: Int = 0 @@ -432,8 +432,8 @@ private[akka] object FlowGraphInternal { private[scaladsl2] def newInstance(): Vertex } - case class TapVertex(tap: Tap[_]) extends Vertex { - override def toString = tap.toString + case class SourceVertex(source: Source[_]) extends Vertex { + override def toString = source.toString // these are unique keys, case class equality would break them final override def equals(other: Any): Boolean = super.equals(other) final override def hashCode: Int = super.hashCode @@ -441,8 +441,8 @@ private[akka] object FlowGraphInternal { final override private[scaladsl2] def newInstance() = this.copy() } - case class DrainVertex(drain: Drain[_]) extends Vertex { - override def toString = drain.toString + case class SinkVertex(sink: Sink[_]) extends Vertex { + override def toString = sink.toString // these are unique keys, case class equality would break them final override def equals(other: Any): Boolean = super.equals(other) final override def hashCode: Int = super.hashCode @@ -520,17 +520,17 @@ class FlowGraphBuilder private (graph: Graph[FlowGraphInternal.Vertex, FlowGraph private var cyclesAllowed = false - private def addTapPipeEdge[In, Out](tap: Tap[In], pipe: Pipe[In, Out], junctionIn: JunctionInPort[Out]): this.type = { - val tapVertex = TapVertex(tap) + private def addSourceToPipeEdge[In, Out](source: Source[In], pipe: Pipe[In, Out], junctionIn: JunctionInPort[Out]): this.type = { + val sourceVertex = SourceVertex(source) checkJunctionInPortPrecondition(junctionIn) - addGraphEdge(tapVertex, junctionIn.vertex, pipe, inputPort = junctionIn.port, outputPort = UnlabeledPort) + addGraphEdge(sourceVertex, junctionIn.vertex, pipe, inputPort = junctionIn.port, outputPort = UnlabeledPort) this } - private def addPipeDrainEdge[In, Out](junctionOut: JunctionOutPort[In], pipe: Pipe[In, Out], drain: Drain[Out]): this.type = { - val drainVertex = DrainVertex(drain) + private def addPipeToSinkEdge[In, Out](junctionOut: JunctionOutPort[In], pipe: Pipe[In, Out], sink: Sink[Out]): this.type = { + val sinkVertex = SinkVertex(sink) checkJunctionOutPortPrecondition(junctionOut) - addGraphEdge(junctionOut.vertex, drainVertex, pipe, inputPort = UnlabeledPort, outputPort = junctionOut.port) + addGraphEdge(junctionOut.vertex, sinkVertex, pipe, inputPort = UnlabeledPort, outputPort = junctionOut.port) this } @@ -595,17 +595,16 @@ class FlowGraphBuilder private (graph: Graph[FlowGraphInternal.Vertex, FlowGraph def addEdge[In, Out](source: Source[In], flow: Flow[In, Out], junctionIn: JunctionInPort[Out]): this.type = { (source, flow) match { - case (tap: Tap[In], pipe: Pipe[In, Out]) ⇒ - addTapPipeEdge(tap, pipe, junctionIn) case (spipe: SourcePipe[In], pipe: Pipe[In, Out]) ⇒ - addTapPipeEdge(spipe.input, Pipe(spipe.ops).appendPipe(pipe), junctionIn) + addSourceToPipeEdge(spipe.input, Pipe(spipe.ops).appendPipe(pipe), junctionIn) case (gsource: GraphSource[_, In], _) ⇒ val tOut = UndefinedSink[In] val tIn = UndefinedSource[Out] addEdge(gsource, tOut) addEdge(tIn, junctionIn) connect(tOut, flow, tIn) - case x ⇒ throwUnsupportedValue(x) + case (source: Source[In], pipe: Pipe[In, Out]) ⇒ + addSourceToPipeEdge(source, pipe, junctionIn) } this } @@ -615,16 +614,16 @@ class FlowGraphBuilder private (graph: Graph[FlowGraphInternal.Vertex, FlowGraph def addEdge[In, Out](junctionOut: JunctionOutPort[In], flow: Flow[In, Out], sink: Sink[Out]): this.type = { (flow, sink) match { - case (pipe: Pipe[In, Out], drain: Drain[Out]) ⇒ - addPipeDrainEdge(junctionOut, pipe, drain) case (pipe: Pipe[In, Out], spipe: SinkPipe[Out]) ⇒ - addPipeDrainEdge(junctionOut, pipe.appendPipe(Pipe(spipe.ops)), spipe.output) + addPipeToSinkEdge(junctionOut, pipe.appendPipe(Pipe(spipe.ops)), spipe.output) case (_, gsink: GraphSink[Out, _]) ⇒ val tOut = UndefinedSink[In] val tIn = UndefinedSource[Out] addEdge(tIn, gsink) addEdge(junctionOut, tOut) connect(tOut, flow, tIn) + case (pipe: Pipe[In, Out], sink: Sink[Out]) ⇒ + addPipeToSinkEdge(junctionOut, pipe, sink) case x ⇒ throwUnsupportedValue(x) } this @@ -634,27 +633,27 @@ class FlowGraphBuilder private (graph: Graph[FlowGraphInternal.Vertex, FlowGraph def addEdge[In, Out](source: Source[In], flow: Flow[In, Out], sink: Sink[Out]): this.type = { (source, flow, sink) match { - case (tap: Tap[In], pipe: Pipe[In, Out], drain: Drain[Out]) ⇒ - addGraphEdge(TapVertex(tap), DrainVertex(drain), pipe, inputPort = UnlabeledPort, outputPort = UnlabeledPort) case (sourcePipe: SourcePipe[In], pipe: Pipe[In, Out], sinkPipe: SinkPipe[Out]) ⇒ - val tap = sourcePipe.input + val src = sourcePipe.input val newPipe = Pipe(sourcePipe.ops).connect(pipe).connect(Pipe(sinkPipe.ops)) - val drain = sinkPipe.output - addEdge(tap, newPipe, drain) // recursive, but now it is a Tap-Pipe-Drain - case (tap: Tap[In], pipe: Pipe[In, Out], sinkPipe: SinkPipe[Out]) ⇒ - val newPipe = pipe.connect(Pipe(sinkPipe.ops)) - val drain = sinkPipe.output - addEdge(tap, newPipe, drain) // recursive, but now it is a Tap-Pipe-Drain - case (sourcePipe: SourcePipe[In], pipe: Pipe[In, Out], drain: Drain[Out]) ⇒ - val tap = sourcePipe.input + val snk = sinkPipe.output + addEdge(src, newPipe, snk) // recursive, but now it is a Source-Pipe-Sink + case (sourcePipe: SourcePipe[In], pipe: Pipe[In, Out], sink: Sink[Out]) ⇒ + val src = sourcePipe.input val newPipe = Pipe(sourcePipe.ops).connect(pipe) - addEdge(tap, newPipe, drain) // recursive, but now it is a Tap-Pipe-Drain + addEdge(src, newPipe, sink) // recursive, but now it is a Source-Pipe-Sink + case (source: Source[In], pipe: Pipe[In, Out], sinkPipe: SinkPipe[Out]) ⇒ + val newPipe = pipe.connect(Pipe(sinkPipe.ops)) + val snk = sinkPipe.output + addEdge(source, newPipe, snk) // recursive, but now it is a Source-Pipe-Sink case (_, gflow: GraphFlow[In, _, _, Out], _) ⇒ val tOut = UndefinedSink[In] val tIn = UndefinedSource[Out] addEdge(source, tOut) addEdge(tIn, sink) connect(tOut, gflow, tIn) + case (source: Source[In], pipe: Pipe[In, Out], sink: Sink[Out]) ⇒ + addGraphEdge(SourceVertex(source), SinkVertex(sink), pipe, inputPort = UnlabeledPort, outputPort = UnlabeledPort) case x ⇒ throwUnsupportedValue(x) } @@ -682,10 +681,8 @@ class FlowGraphBuilder private (graph: Graph[FlowGraphInternal.Vertex, FlowGraph def addEdge[In, Out](source: UndefinedSource[In], flow: Flow[In, Out], sink: Sink[Out]): this.type = { (flow, sink) match { - case (pipe: Pipe[In, Out], drain: Drain[Out]) ⇒ - addGraphEdge(source, DrainVertex(drain), pipe, inputPort = UnlabeledPort, outputPort = UnlabeledPort) case (pipe: Pipe[In, Out], spipe: SinkPipe[Out]) ⇒ - addGraphEdge(source, DrainVertex(spipe.output), pipe.appendPipe(Pipe(spipe.ops)), inputPort = UnlabeledPort, outputPort = UnlabeledPort) + addGraphEdge(source, SinkVertex(spipe.output), pipe.appendPipe(Pipe(spipe.ops)), inputPort = UnlabeledPort, outputPort = UnlabeledPort) case (gflow: GraphFlow[In, _, _, Out], _) ⇒ val tOut = UndefinedSink[In] val tIn = UndefinedSource[Out] @@ -696,6 +693,8 @@ class FlowGraphBuilder private (graph: Graph[FlowGraphInternal.Vertex, FlowGraph val oOut = UndefinedSink[Out] addEdge(source, flow, oOut) gSink.importAndConnect(this, oOut) + case (pipe: Pipe[In, Out], sink: Sink[Out]) ⇒ + addGraphEdge(source, SinkVertex(sink), pipe, inputPort = UnlabeledPort, outputPort = UnlabeledPort) case x ⇒ throwUnsupportedValue(x) } this @@ -705,10 +704,8 @@ class FlowGraphBuilder private (graph: Graph[FlowGraphInternal.Vertex, FlowGraph def addEdge[In, Out](source: Source[In], flow: Flow[In, Out], sink: UndefinedSink[Out]): this.type = { (flow, source) match { - case (pipe: Pipe[In, Out], tap: Tap[In]) ⇒ - addGraphEdge(TapVertex(tap), sink, pipe, inputPort = UnlabeledPort, outputPort = UnlabeledPort) case (pipe: Pipe[In, Out], spipe: SourcePipe[Out]) ⇒ - addGraphEdge(TapVertex(spipe.input), sink, Pipe(spipe.ops).appendPipe(pipe), inputPort = UnlabeledPort, outputPort = UnlabeledPort) + addGraphEdge(SourceVertex(spipe.input), sink, Pipe(spipe.ops).appendPipe(pipe), inputPort = UnlabeledPort, outputPort = UnlabeledPort) case (_, gsource: GraphSource[_, In]) ⇒ val tOut1 = UndefinedSource[In] val tOut2 = UndefinedSink[In] @@ -717,6 +714,8 @@ class FlowGraphBuilder private (graph: Graph[FlowGraphInternal.Vertex, FlowGraph gsource.importAndConnect(this, tOut1) addEdge(tIn, sink) connect(tOut2, flow, tIn) + case (pipe: Pipe[In, Out], source: Source[In]) ⇒ + addGraphEdge(SourceVertex(source), sink, pipe, inputPort = UnlabeledPort, outputPort = UnlabeledPort) case x ⇒ throwUnsupportedValue(x) } this @@ -730,14 +729,14 @@ class FlowGraphBuilder private (graph: Graph[FlowGraphInternal.Vertex, FlowGraph } private def addGraphEdge[In, Out](from: Vertex, to: Vertex, pipe: Pipe[In, Out], inputPort: Int, outputPort: Int): Unit = { - checkAddTapDrainPrecondition(from) - checkAddTapDrainPrecondition(to) + checkAddSourceSinkPrecondition(from) + checkAddSourceSinkPrecondition(to) uncheckedAddGraphEdge(from, to, pipe, inputPort, outputPort) } private def addOrReplaceGraphEdge[In, Out](from: Vertex, to: Vertex, pipe: Pipe[In, Out], inputPort: Int, outputPort: Int): Unit = { - checkAddOrReplaceTapDrainPrecondition(from) - checkAddOrReplaceTapDrainPrecondition(to) + checkAddOrReplaceSourceSinkPrecondition(from) + checkAddOrReplaceSourceSinkPrecondition(to) uncheckedAddGraphEdge(from, to, pipe, inputPort, outputPort) } @@ -747,14 +746,13 @@ class FlowGraphBuilder private (graph: Graph[FlowGraphInternal.Vertex, FlowGraph val edge = existing.incoming.head graph.remove(existing) sink match { - case drain: Drain[Out] ⇒ - addGraphEdge(edge.from.value, DrainVertex(drain), edge.label.pipe, edge.label.inputPort, edge.label.outputPort) case spipe: SinkPipe[Out] ⇒ val pipe = edge.label.pipe.appendPipe(Pipe(spipe.ops)) - addGraphEdge(edge.from.value, DrainVertex(spipe.output), pipe, edge.label.inputPort, edge.label.outputPort) + addGraphEdge(edge.from.value, SinkVertex(spipe.output), pipe, edge.label.inputPort, edge.label.outputPort) case gsink: GraphSink[Out, _] ⇒ gsink.importAndConnect(this, token) - case x ⇒ throwUnsupportedValue(x) + case sink: Sink[Out] ⇒ + addGraphEdge(edge.from.value, SinkVertex(sink), edge.label.pipe, edge.label.inputPort, edge.label.outputPort) } case None ⇒ throw new IllegalArgumentException(s"No matching UndefinedSink [${token}]") @@ -768,13 +766,13 @@ class FlowGraphBuilder private (graph: Graph[FlowGraphInternal.Vertex, FlowGraph val edge = existing.outgoing.head graph.remove(existing) source match { - case tap: Tap[In] ⇒ - addGraphEdge(TapVertex(tap), edge.to.value, edge.label.pipe, edge.label.inputPort, edge.label.outputPort) case spipe: SourcePipe[In] ⇒ val pipe = Pipe(spipe.ops).appendPipe(edge.label.pipe) - addGraphEdge(TapVertex(spipe.input), edge.to.value, pipe, edge.label.inputPort, edge.label.outputPort) + addGraphEdge(SourceVertex(spipe.input), edge.to.value, pipe, edge.label.inputPort, edge.label.outputPort) case gsource: GraphSource[_, In] ⇒ gsource.importAndConnect(this, token) + case source: Source[In] ⇒ + addGraphEdge(SourceVertex(source), edge.to.value, edge.label.pipe, edge.label.inputPort, edge.label.outputPort) case x ⇒ throwUnsupportedValue(x) } @@ -853,7 +851,7 @@ class FlowGraphBuilder private (graph: Graph[FlowGraphInternal.Vertex, FlowGraph cyclesAllowed = true } - private def checkAddTapDrainPrecondition(vertex: Vertex): Unit = { + private def checkAddSourceSinkPrecondition(vertex: Vertex): Unit = { vertex match { case node @ (_: UndefinedSource[_] | _: UndefinedSink[_]) ⇒ require(!graph.contains(node), s"[$node] instance is already used in this flow graph") @@ -861,7 +859,7 @@ class FlowGraphBuilder private (graph: Graph[FlowGraphInternal.Vertex, FlowGraph } } - private def checkAddOrReplaceTapDrainPrecondition(vertex: Vertex): Unit = { + private def checkAddOrReplaceSourceSinkPrecondition(vertex: Vertex): Unit = { vertex match { // it is ok to add or replace edges with new or existing undefined sources or sinks case node @ (_: UndefinedSource[_] | _: UndefinedSink[_]) ⇒ @@ -928,14 +926,14 @@ class FlowGraphBuilder private (graph: Graph[FlowGraphInternal.Vertex, FlowGraph } private def checkBuildPreconditions(): Unit = { - val undefinedSourcesDrains = graph.nodes.filter { + val undefinedSourcesSinks = graph.nodes.filter { _.value match { case _: UndefinedSource[_] | _: UndefinedSink[_] ⇒ true case x ⇒ false } } - if (undefinedSourcesDrains.nonEmpty) { - val formatted = undefinedSourcesDrains.map(n ⇒ n.value match { + if (undefinedSourcesSinks.nonEmpty) { + val formatted = undefinedSourcesSinks.map(n ⇒ n.value match { case u: UndefinedSource[_] ⇒ s"$u -> ${n.outgoing.head.label} -> ${n.outgoing.head.to}" case u: UndefinedSink[_] ⇒ s"${n.incoming.head.from} -> ${n.incoming.head.label} -> $u" }) @@ -995,7 +993,7 @@ object FlowGraph { /** * Continue building a [[FlowGraph]] from an existing `PartialFlowGraph`. * For example you can attach undefined sources and sinks with - * [[FlowGraphBuilder#attachTap]] and [[FlowGraphBuilder#attachDrain]] + * [[FlowGraphBuilder#attachSource]] and [[FlowGraphBuilder#attachSink]] */ def apply(partialFlowGraph: PartialFlowGraph)(block: FlowGraphBuilder ⇒ Unit): FlowGraph = apply(partialFlowGraph.graph)(block) @@ -1031,9 +1029,9 @@ class FlowGraph private[akka] (private[akka] val graph: ImmutableGraph[FlowGraph if (edges.size == 1) { val edge = edges.head (edge.from.value, edge.to.value) match { - case (tapVertex: TapVertex, drainVertex: DrainVertex) ⇒ + case (sourceVertex: SourceVertex, sinkVertex: SinkVertex) ⇒ val pipe = edge.label.pipe - runSimple(tapVertex, drainVertex, pipe) + runSimple(sourceVertex, sinkVertex, pipe) case _ ⇒ runGraph() } @@ -1044,15 +1042,15 @@ class FlowGraph private[akka] (private[akka] val graph: ImmutableGraph[FlowGraph /** * Run FlowGraph that only contains one edge from a `Source` to a `Sink`. */ - private def runSimple(tapVertex: TapVertex, drainVertex: DrainVertex, pipe: Pipe[Any, Nothing])(implicit materializer: FlowMaterializer): MaterializedMap = { - val mf = pipe.withTap(tapVertex.tap).withDrain(drainVertex.drain).run() - val materializedSources: Map[TapWithKey[_], Any] = tapVertex match { - case TapVertex(tap: TapWithKey[_]) ⇒ Map(tap -> mf.materializedTap(tap)) - case _ ⇒ Map.empty + private def runSimple(sourceVertex: SourceVertex, sinkVertex: SinkVertex, pipe: Pipe[Any, Nothing])(implicit materializer: FlowMaterializer): MaterializedMap = { + val mf = pipe.withSource(sourceVertex.source).withSink(sinkVertex.sink).run() + val materializedSources: Map[KeyedSource[_], Any] = sourceVertex match { + case SourceVertex(source: KeyedSource[_]) ⇒ Map(source -> mf.get(source)) + case _ ⇒ Map.empty } - val materializedSinks: Map[DrainWithKey[_], Any] = drainVertex match { - case DrainVertex(drain: DrainWithKey[_]) ⇒ Map(drain -> mf.materializedDrain(drain)) - case _ ⇒ Map.empty + val materializedSinks: Map[KeyedSink[_], Any] = sinkVertex match { + case SinkVertex(sink: KeyedSink[_]) ⇒ Map(sink -> mf.get(sink)) + case _ ⇒ Map.empty } new MaterializedFlowGraph(materializedSources, materializedSinks) } @@ -1060,14 +1058,14 @@ class FlowGraph private[akka] (private[akka] val graph: ImmutableGraph[FlowGraph private def runGraph()(implicit materializer: FlowMaterializer): MaterializedMap = { import scalax.collection.GraphTraversal._ - // start with drains + // start with sinks val startingNodes = graph.nodes.filter(n ⇒ n.isLeaf && n.diSuccessors.isEmpty) case class Memo(visited: Set[graph.EdgeT] = Set.empty, downstreamSubscriber: Map[graph.EdgeT, Subscriber[Any]] = Map.empty, upstreamPublishers: Map[graph.EdgeT, Publisher[Any]] = Map.empty, - taps: Map[TapVertex, SinkPipe[Any]] = Map.empty, - materializedDrains: Map[DrainWithKey[_], Any] = Map.empty) + sources: Map[SourceVertex, SinkPipe[Any]] = Map.empty, + materializedSinks: Map[KeyedSink[_], Any] = Map.empty) val result = startingNodes.foldLeft(Memo()) { case (memo, start) ⇒ @@ -1081,36 +1079,36 @@ class FlowGraph private[akka] (private[akka] val graph: ImmutableGraph[FlowGraph } else { val pipe = edge.label.pipe - // returns the materialized drain, if any - def connectToDownstream(publisher: Publisher[Any]): Option[(DrainWithKey[_], Any)] = { - val f = pipe.withTap(PublisherTap(publisher)) + // returns the materialized sink, if any + def connectToDownstream(publisher: Publisher[Any]): Option[(KeyedSink[_], Any)] = { + val f = pipe.withSource(PublisherSource(publisher)) edge.to.value match { - case DrainVertex(drain: DrainWithKey[_]) ⇒ - val mf = f.withDrain(drain).run() - Some(drain -> mf.materializedDrain(drain)) - case DrainVertex(drain) ⇒ - f.withDrain(drain).run() + case SinkVertex(sink: KeyedSink[_]) ⇒ + val mf = f.withSink(sink).run() + Some(sink -> mf.get(sink)) + case SinkVertex(sink) ⇒ + f.withSink(sink).run() None case _ ⇒ - f.withDrain(SubscriberDrain(memo.downstreamSubscriber(edge))).run() + f.withSink(SubscriberSink(memo.downstreamSubscriber(edge))).run() None } } edge.from.value match { - case tap: TapVertex ⇒ - val f = pipe.withDrain(SubscriberDrain(memo.downstreamSubscriber(edge))) - // connect the tap with the pipe later + case source: SourceVertex ⇒ + val f = pipe.withSink(SubscriberSink(memo.downstreamSubscriber(edge))) + // connect the source with the pipe later memo.copy(visited = memo.visited + edge, - taps = memo.taps.updated(tap, f)) + sources = memo.sources.updated(source, f)) case v: InternalVertex ⇒ if (memo.upstreamPublishers.contains(edge)) { // vertex already materialized - val materializedDrain = connectToDownstream(memo.upstreamPublishers(edge)) + val materializedSink = connectToDownstream(memo.upstreamPublishers(edge)) memo.copy( visited = memo.visited + edge, - materializedDrains = memo.materializedDrains ++ materializedDrain) + materializedSinks = memo.materializedSinks ++ materializedSink) } else { val op = v.astNode @@ -1122,12 +1120,12 @@ class FlowGraph private[akka] (private[akka] val graph: ImmutableGraph[FlowGraph val edgePublishers = edge.from.outgoing.toSeq.sortBy(_.label.outputPort).zip(publishers).toMap val publisher = edgePublishers(edge) - val materializedDrain = connectToDownstream(publisher) + val materializedSink = connectToDownstream(publisher) memo.copy( visited = memo.visited + edge, downstreamSubscriber = memo.downstreamSubscriber ++ edgeSubscribers, upstreamPublishers = memo.upstreamPublishers ++ edgePublishers, - materializedDrains = memo.materializedDrains ++ materializedDrain) + materializedSinks = memo.materializedSinks ++ materializedSink) } } @@ -1137,17 +1135,17 @@ class FlowGraph private[akka] (private[akka] val graph: ImmutableGraph[FlowGraph } - // connect all input taps as the last thing - val materializedTaps = result.taps.foldLeft(Map.empty[TapWithKey[_], Any]) { - case (acc, (TapVertex(tap), pipe)) ⇒ - val mf = pipe.withTap(tap).run() - tap match { - case tapKey: TapWithKey[_] ⇒ acc.updated(tapKey, mf.materializedTap(tapKey)) - case _ ⇒ acc + // connect all input sources as the last thing + val materializedSources = result.sources.foldLeft(Map.empty[KeyedSource[_], Any]) { + case (acc, (SourceVertex(source), pipe)) ⇒ + val mf = pipe.withSource(source).run() + source match { + case sourceKey: KeyedSource[_] ⇒ acc.updated(sourceKey, mf.get(sourceKey)) + case _ ⇒ acc } } - new MaterializedFlowGraph(materializedTaps, result.materializedDrains) + new MaterializedFlowGraph(materializedSources, result.materializedSinks) } } @@ -1250,23 +1248,23 @@ class PartialFlowGraph private[akka] (private[akka] val graph: ImmutableGraph[Fl /** * Returned by [[FlowGraph#run]] and can be used to retrieve the materialized - * `Tap` inputs or `Drain` outputs. + * `Source` inputs or `Sink` outputs. */ -private[scaladsl2] class MaterializedFlowGraph(materializedTaps: Map[TapWithKey[_], Any], materializedDrains: Map[DrainWithKey[_], Any]) +private[scaladsl2] class MaterializedFlowGraph(materializedSources: Map[KeyedSource[_], Any], materializedSinks: Map[KeyedSink[_], Any]) extends MaterializedMap { - override def materializedTap(key: TapWithKey[_]): key.MaterializedType = - materializedTaps.get(key) match { - case Some(matTap) ⇒ matTap.asInstanceOf[key.MaterializedType] + override def get(key: KeyedSource[_]): key.MaterializedType = + materializedSources.get(key) match { + case Some(matSource) ⇒ matSource.asInstanceOf[key.MaterializedType] case None ⇒ - throw new IllegalArgumentException(s"Tap key [$key] doesn't exist in this flow graph") + throw new IllegalArgumentException(s"Source key [$key] doesn't exist in this flow graph") } - def materializedDrain(key: DrainWithKey[_]): key.MaterializedType = - materializedDrains.get(key) match { - case Some(matDrain) ⇒ matDrain.asInstanceOf[key.MaterializedType] + def get(key: KeyedSink[_]): key.MaterializedType = + materializedSinks.get(key) match { + case Some(matSink) ⇒ matSink.asInstanceOf[key.MaterializedType] case None ⇒ - throw new IllegalArgumentException(s"Drain key [$key] doesn't exist in this flow graph") + throw new IllegalArgumentException(s"Sink key [$key] doesn't exist in this flow graph") } } diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl2/FlowMaterializer.scala b/akka-stream/src/main/scala/akka/stream/scaladsl2/FlowMaterializer.scala index d7e798d288..da98883bf2 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl2/FlowMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl2/FlowMaterializer.scala @@ -139,7 +139,7 @@ abstract class FlowMaterializer(val settings: MaterializerSettings) { * stream. The result can be highly implementation specific, ranging from * local actor chains to remote-deployed processing networks. */ - def materialize[In, Out](tap: Tap[In], drain: Drain[Out], ops: List[Ast.AstNode]): MaterializedMap + def materialize[In, Out](source: Source[In], sink: Sink[Out], ops: List[Ast.AstNode]): MaterializedMap /** * Create publishers and subscribers for fan-in and fan-out operations. diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl2/GraphFlow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl2/GraphFlow.scala index fae11c7e09..f090b3be83 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl2/GraphFlow.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl2/GraphFlow.scala @@ -44,7 +44,6 @@ private[scaladsl2] case class GraphFlow[-In, CIn, COut, +Out](inPipe: Pipe[In, C } override def connect(sink: Sink[Out]) = sink match { - case drain: Drain[Out] ⇒ connect(Pipe.empty.withDrain(drain)) // recursive, but now it is a SinkPipe case sinkPipe: SinkPipe[Out] ⇒ val newGraph = PartialFlowGraph(this.graph) { builder ⇒ builder.attachSink(out, outPipe.connect(sinkPipe)) @@ -56,6 +55,7 @@ private[scaladsl2] case class GraphFlow[-In, CIn, COut, +Out](inPipe: Pipe[In, C b.connect(out, outPipe.connect(gSink.inPipe), oIn) } GraphSink(inPipe, in, newGraph) + case sink: Sink[Out] ⇒ connect(Pipe.empty.withSink(sink)) // recursive, but now it is a SinkPipe } override private[scaladsl2] def andThen[T](op: AstNode): Repr[T] = copy(outPipe = outPipe.andThen(op)) @@ -87,8 +87,6 @@ private[scaladsl2] case class GraphSource[COut, +Out](graph: PartialFlowGraph, o } override def connect(sink: Sink[Out]): RunnableFlow = sink match { - case drain: Drain[Out] ⇒ - connect(Pipe.empty.withDrain(drain)) // recursive, but now it is a SinkPipe case sinkPipe: SinkPipe[Out] ⇒ FlowGraph(this.graph) { implicit builder ⇒ builder.attachSink(out, outPipe.connect(sinkPipe)) @@ -98,6 +96,8 @@ private[scaladsl2] case class GraphSource[COut, +Out](graph: PartialFlowGraph, o val oIn = gSink.remap(b) b.connect(out, outPipe.connect(gSink.inPipe), oIn) } + case sink: Sink[Out] ⇒ + connect(Pipe.empty.withSink(sink)) // recursive, but now it is a SinkPipe } override private[scaladsl2] def andThen[T](op: AstNode): Repr[T] = copy(outPipe = outPipe.andThen(op)) diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl2/MaterializedMap.scala b/akka-stream/src/main/scala/akka/stream/scaladsl2/MaterializedMap.scala index ca2fe2a638..352c6ea0fa 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl2/MaterializedMap.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl2/MaterializedMap.scala @@ -5,18 +5,17 @@ package akka.stream.scaladsl2 /** * Returned by [[RunnableFlow#run]] and [[FlowGraph#run]] and can be used to retrieve the materialized - * `Tap` inputs or `Drain` outputs, e.g. [[SubscriberTap]] or [[PublisherDrain]]. + * `Source` inputs or `Sink` outputs, e.g. [[SubscriberSource]] or [[PublisherSink]]. */ trait MaterializedMap { /** - * Retrieve a materialized `Tap`, e.g. the `Subscriber` of a [[SubscriberTap]]. + * Retrieve a materialized `Source`, e.g. the `Subscriber` of a [[SubscriberSource]]. */ - def materializedTap(key: TapWithKey[_]): key.MaterializedType + def get(key: KeyedSource[_]): key.MaterializedType /** - * Retrieve a materialized `Drain`, e.g. the `Publisher` of a [[PublisherDrain]]. + * Retrieve a materialized `Sink`, e.g. the `Publisher` of a [[PublisherSink]]. */ - def materializedDrain(key: DrainWithKey[_]): key.MaterializedType - -} \ No newline at end of file + def get(key: KeyedSink[_]): key.MaterializedType +} diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl2/Pipe.scala b/akka-stream/src/main/scala/akka/stream/scaladsl2/Pipe.scala index 4885068f6c..2874211b28 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl2/Pipe.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl2/Pipe.scala @@ -21,9 +21,9 @@ private[stream] final case class Pipe[-In, +Out](ops: List[AstNode]) extends Flo override private[scaladsl2] def andThen[U](op: AstNode): Repr[U] = this.copy(ops = op :: ops) - private[stream] def withDrain(out: Drain[Out]): SinkPipe[In] = SinkPipe(out, ops) + private[stream] def withSink(out: Sink[Out]): SinkPipe[In] = SinkPipe(out, ops) - private[stream] def withTap(in: Tap[In]): SourcePipe[Out] = SourcePipe(in, ops) + private[stream] def withSource(in: Source[In]): SourcePipe[Out] = SourcePipe(in, ops) override def connect[T](flow: Flow[Out, T]): Flow[In, T] = flow match { case p: Pipe[T, In] ⇒ Pipe(p.ops ++: ops) @@ -32,10 +32,9 @@ private[stream] final case class Pipe[-In, +Out](ops: List[AstNode]) extends Flo } override def connect(sink: Sink[Out]): Sink[In] = sink match { - case d: Drain[Out] ⇒ this.withDrain(d) case sp: SinkPipe[Out] ⇒ sp.prependPipe(this) case gs: GraphSink[Out, _] ⇒ gs.prepend(this) - case x ⇒ FlowGraphInternal.throwUnsupportedValue(x) + case d: Sink[Out] ⇒ this.withSink(d) } private[stream] def appendPipe[T](pipe: Pipe[Out, T]): Pipe[In, T] = Pipe(pipe.ops ++: ops) @@ -44,25 +43,25 @@ private[stream] final case class Pipe[-In, +Out](ops: List[AstNode]) extends Flo /** * Pipe with open input and attached output. Can be used as a `Subscriber`. */ -private[stream] final case class SinkPipe[-In](output: Drain[_], ops: List[AstNode]) extends Sink[In] { +private[stream] final case class SinkPipe[-In](output: Sink[_], ops: List[AstNode]) extends Sink[In] { - private[stream] def withTap(in: Tap[In]): RunnablePipe = RunnablePipe(in, output, ops) + private[stream] def withSource(in: Source[In]): RunnablePipe = RunnablePipe(in, output, ops) private[stream] def prependPipe[T](pipe: Pipe[T, In]): SinkPipe[T] = SinkPipe(output, ops ::: pipe.ops) - override def runWith(tap: SimpleTap[In])(implicit materializer: FlowMaterializer): Unit = - tap.connect(this).run() + override def runWith(source: Source[In])(implicit materializer: FlowMaterializer): Unit = + source.connect(this).run() } /** * Pipe with open output and attached input. Can be used as a `Publisher`. */ -private[stream] final case class SourcePipe[+Out](input: Tap[_], ops: List[AstNode]) extends Source[Out] { +private[stream] final case class SourcePipe[+Out](input: Source[_], ops: List[AstNode]) extends Source[Out] { override type Repr[+O] = SourcePipe[O] override private[scaladsl2] def andThen[U](op: AstNode): Repr[U] = SourcePipe(input, op :: ops) - private[stream] def withDrain(out: Drain[Out]): RunnablePipe = RunnablePipe(input, out, ops) + private[stream] def withSink(out: Sink[Out]): RunnablePipe = RunnablePipe(input, out, ops) private[stream] def appendPipe[T](pipe: Pipe[Out, T]): SourcePipe[T] = SourcePipe(input, pipe.ops ++: ops) @@ -74,30 +73,29 @@ private[stream] final case class SourcePipe[+Out](input: Tap[_], ops: List[AstNo override def connect(sink: Sink[Out]): RunnableFlow = sink match { case sp: SinkPipe[Out] ⇒ RunnablePipe(input, sp.output, sp.ops ++: ops) - case d: Drain[Out] ⇒ this.withDrain(d) case g: GraphSink[Out, _] ⇒ g.prepend(this) - case x ⇒ FlowGraphInternal.throwUnsupportedValue(x) + case d: Sink[Out] ⇒ this.withSink(d) } } /** * Pipe with attached input and output, can be executed. */ -private[scaladsl2] final case class RunnablePipe(input: Tap[_], output: Drain[_], ops: List[AstNode]) extends RunnableFlow { +private[scaladsl2] final case class RunnablePipe(input: Source[_], output: Sink[_], ops: List[AstNode]) extends RunnableFlow { def run()(implicit materializer: FlowMaterializer): MaterializedMap = materializer.materialize(input, output, ops) } /** * Returned by [[RunnablePipe#run]] and can be used as parameter to retrieve the materialized - * `Tap` input or `Drain` output. + * `Source` input or `Sink` output. */ -private[stream] class MaterializedPipe(tapKey: AnyRef, matTap: Any, drainKey: AnyRef, matDrain: Any) extends MaterializedMap { - override def materializedTap(key: TapWithKey[_]): key.MaterializedType = - if (key == tapKey) matTap.asInstanceOf[key.MaterializedType] - else throw new IllegalArgumentException(s"Tap key [$key] doesn't match the tap [$tapKey] of this flow") +private[stream] class MaterializedPipe(sourceKey: AnyRef, matSource: Any, sinkKey: AnyRef, matSink: Any) extends MaterializedMap { + override def get(key: KeyedSource[_]): key.MaterializedType = + if (key == sourceKey) matSource.asInstanceOf[key.MaterializedType] + else throw new IllegalArgumentException(s"Source key [$key] doesn't match the source [$sourceKey] of this flow") - override def materializedDrain(key: DrainWithKey[_]): key.MaterializedType = - if (key == drainKey) matDrain.asInstanceOf[key.MaterializedType] - else throw new IllegalArgumentException(s"Drain key [$key] doesn't match the drain [$drainKey] of this flow") + override def get(key: KeyedSink[_]): key.MaterializedType = + if (key == sinkKey) matSink.asInstanceOf[key.MaterializedType] + else throw new IllegalArgumentException(s"Sink key [$key] doesn't match the sink [$sinkKey] of this flow") } diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl2/Sink.scala b/akka-stream/src/main/scala/akka/stream/scaladsl2/Sink.scala index 57cb4613aa..492645e381 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl2/Sink.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl2/Sink.scala @@ -4,10 +4,7 @@ package akka.stream.scaladsl2 import org.reactivestreams.Subscriber - -import scala.concurrent.Future -import scala.language.implicitConversions -import scala.annotation.unchecked.uncheckedVariance +import scala.util.Try /** * A `Sink` is a set of stream processing steps that has one open input and an attached output. @@ -15,25 +12,25 @@ import scala.annotation.unchecked.uncheckedVariance */ trait Sink[-In] { /** - * Connect this `Sink` to a `Tap` and run it. The returned value is the materialized value - * of the `Tap`, e.g. the `Subscriber` of a [[SubscriberTap]]. + * Connect this `Sink` to a `Source` and run it. The returned value is the materialized value + * of the `Source`, e.g. the `Subscriber` of a [[SubscriberSource]]. */ - def runWith(tap: TapWithKey[In])(implicit materializer: FlowMaterializer): tap.MaterializedType = - tap.connect(this).run().materializedTap(tap) + def runWith(source: KeyedSource[In])(implicit materializer: FlowMaterializer): source.MaterializedType = + source.connect(this).run().get(source) /** - * Connect this `Sink` to a `Tap` and run it. The returned value is the materialized value - * of the `Tap`, e.g. the `Subscriber` of a [[SubscriberTap]]. + * Connect this `Sink` to a `Source` and run it. The returned value is the materialized value + * of the `Source`, e.g. the `Subscriber` of a [[SubscriberSource]]. */ - def runWith(tap: SimpleTap[In])(implicit materializer: FlowMaterializer): Unit = - tap.connect(this).run() + def runWith(source: Source[In])(implicit materializer: FlowMaterializer): Unit = + source.connect(this).run() } object Sink { /** * Helper to create [[Sink]] from `Subscriber`. */ - def apply[T](subscriber: Subscriber[T]): Drain[T] = SubscriberDrain(subscriber) + def apply[T](subscriber: Subscriber[T]): Sink[T] = SubscriberSink(subscriber) /** * Creates a `Sink` by using an empty [[FlowGraphBuilder]] on a block that expects a [[FlowGraphBuilder]] and @@ -49,13 +46,69 @@ object Sink { def apply[T](graph: PartialFlowGraph)(block: FlowGraphBuilder ⇒ UndefinedSource[T]): Sink[T] = createSinkFromBuilder(new FlowGraphBuilder(graph.graph), block) - /** - * A `Sink` that immediately cancels its upstream after materialization. - */ - def cancelled[T]: Drain[T] = CancelDrain - private def createSinkFromBuilder[T](builder: FlowGraphBuilder, block: FlowGraphBuilder ⇒ UndefinedSource[T]): Sink[T] = { val in = block(builder) builder.partialBuild().toSink(in) } + + /** + * A `Sink` that immediately cancels its upstream after materialization. + */ + def cancelled[T]: Sink[T] = CancelSink + + /** + * A `Sink` that materializes into a `Future` of the first value received. + */ + def future[T]: FutureSink[T] = FutureSink[T] + + /** + * A `Sink` that materializes into a [[org.reactivestreams.Publisher]]. + * that can handle one [[org.reactivestreams.Subscriber]]. + */ + def publisher[T]: PublisherSink[T] = PublisherSink[T] + + /** + * A `Sink` that materializes into a [[org.reactivestreams.Publisher]] + * that can handle more than one [[org.reactivestreams.Subscriber]]. + */ + def fanoutPublisher[T](initialBufferSize: Int, maximumBufferSize: Int): FanoutPublisherSink[T] = + FanoutPublisherSink[T](initialBufferSize, maximumBufferSize) + + /** + * A `Sink` that will consume the stream and discard the elements. + */ + def ignore: Sink[Any] = BlackholeSink + + /** + * A `Sink` that will invoke the given procedure for each received element. The sink is materialized + * into a [[scala.concurrent.Future]] will be completed with `Success` when reaching the + * normal end of the stream, or completed with `Failure` if there is an error is signaled in + * the stream.. + */ + def foreach[T](f: T ⇒ Unit): ForeachSink[T] = ForeachSink(f) + + /** + * A `Sink` that will invoke the given function for every received element, giving it its previous + * output (or the given `zero` value) and the element as input. + * The returned [[scala.concurrent.Future]] will be completed with value of the final + * function evaluation when the input stream ends, or completed with `Failure` + * if there is an error is signaled in the stream. + */ + def fold[U, T](zero: U)(f: (U, T) ⇒ U): FoldSink[U, T] = FoldSink(zero)(f) + + /** + * A `Sink` that when the flow is completed, either through an error or normal + * completion, apply the provided function with [[scala.util.Success]] + * or [[scala.util.Failure]]. + */ + def onComplete[T](callback: Try[Unit] ⇒ Unit): Sink[T] = OnCompleteSink[T](callback) +} + +/** + * A `Sink` that will create an object during materialization that the user will need + * to retrieve in order to access aspects of this sink (could be a completion Future + * or a cancellation handle, etc.) + */ +trait KeyedSink[-In] extends Sink[In] { + type MaterializedType } diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl2/Source.scala b/akka-stream/src/main/scala/akka/stream/scaladsl2/Source.scala index a2c3b572ba..6cf39e623f 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl2/Source.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl2/Source.scala @@ -3,16 +3,13 @@ */ package akka.stream.scaladsl2 -import akka.stream.impl.EmptyPublisher -import akka.stream.impl.ErrorPublisher -import akka.stream.impl.SynchronousPublisherFromIterable +import akka.stream.impl.{ EmptyPublisher, ErrorPublisher, SynchronousPublisherFromIterable } import org.reactivestreams.Publisher - import scala.collection.immutable -import scala.concurrent.Future import scala.concurrent.duration.FiniteDuration +import scala.concurrent.Future + import scala.language.higherKinds -import scala.language.implicitConversions /** * A `Source` is a set of stream processing steps that has one open output and an attached input. @@ -32,11 +29,11 @@ trait Source[+Out] extends FlowOps[Out] { def connect(sink: Sink[Out]): RunnableFlow /** - * Connect this `Source` to a `Drain` and run it. The returned value is the materialized value - * of the `Drain`, e.g. the `Publisher` of a [[PublisherDrain]]. + * Connect this `Source` to a `Sink` and run it. The returned value is the materialized value + * of the `Sink`, e.g. the `Publisher` of a [[Sink.fanoutPublisher]]. */ - def runWith(drain: DrainWithKey[Out])(implicit materializer: FlowMaterializer): drain.MaterializedType = - connect(drain).run().materializedDrain(drain) + def runWith(sink: KeyedSink[Out])(implicit materializer: FlowMaterializer): sink.MaterializedType = + connect(sink).run().get(sink) /** * Shortcut for running this `Source` with a fold function. @@ -47,7 +44,7 @@ trait Source[+Out] extends FlowOps[Out] { * if there is an error is signaled in the stream. */ def fold[U](zero: U)(f: (U, Out) ⇒ U)(implicit materializer: FlowMaterializer): Future[U] = - runWith(FoldDrain(zero)(f)) + runWith(FoldSink(zero)(f)) /** * Shortcut for running this `Source` with a foreach procedure. The given procedure is invoked @@ -57,7 +54,7 @@ trait Source[+Out] extends FlowOps[Out] { * the stream. */ def foreach(f: Out ⇒ Unit)(implicit materializer: FlowMaterializer): Future[Unit] = - runWith(ForeachDrain(f)) + runWith(ForeachSink(f)) /** * Concatenates a second source so that the first element @@ -86,7 +83,7 @@ object Source { * that mediate the flow of elements downstream and the propagation of * back-pressure upstream. */ - def apply[T](publisher: Publisher[T]): Source[T] = PublisherTap(publisher) + def apply[T](publisher: Publisher[T]): Source[T] = PublisherSource(publisher) /** * Helper to create [[Source]] from `Iterator`. @@ -98,7 +95,7 @@ object Source { * in accordance with the demand coming from the downstream transformation * steps. */ - def apply[T](iterator: Iterator[T]): Source[T] = IteratorTap(iterator) + def apply[T](iterator: Iterator[T]): Source[T] = IteratorSource(iterator) /** * Helper to create [[Source]] from `Iterable`. @@ -109,14 +106,14 @@ object Source { * 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]): Source[T] = IterableTap(iterable) + def apply[T](iterable: immutable.Iterable[T]): Source[T] = IterableSource(iterable) /** * Define the sequence of elements to be produced by the given closure. * The stream ends normally when evaluation of the closure returns a `None`. * The stream ends exceptionally when an exception is thrown from the closure. */ - def apply[T](f: () ⇒ Option[T]): Source[T] = ThunkTap(f) + def apply[T](f: () ⇒ Option[T]): Source[T] = ThunkSource(f) /** * Start a new `Source` from the given `Future`. The stream will consist of @@ -124,7 +121,7 @@ object Source { * 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]): Source[T] = FutureTap(future) + def apply[T](future: Future[T]): Source[T] = FutureSource(future) /** * Elements are produced from the tick closure periodically with the specified interval. @@ -134,7 +131,7 @@ object Source { * receive new tick elements as soon as it has requested more elements. */ def apply[T](initialDelay: FiniteDuration, interval: FiniteDuration, tick: () ⇒ T): Source[T] = - TickTap(initialDelay, interval, tick) + TickSource(initialDelay, interval, tick) /** * Create a `Source` with one element. @@ -171,10 +168,25 @@ object Source { val out = block(builder) builder.partialBuild().toSource(out) } + /** * Concatenates two sources so that the first element * emitted by the second source is emitted after the last element of the first * source. */ - def concat[T](source1: Source[T], source2: Source[T]): Source[T] = ConcatTap(source1, source2) + def concat[T](source1: Source[T], source2: Source[T]): Source[T] = ConcatSource(source1, source2) + + /** + * Creates a `Source` that is materialized as a [[org.reactivestreams.Subscriber]] + */ + def subscriber[T]: SubscriberSource[T] = SubscriberSource[T] +} + +/** + * A `Source` that will create an object during materialization that the user will need + * to retrieve in order to access aspects of this source (could be a Subscriber, a + * Future/Promise, etc.). + */ +trait KeyedSource[+Out] extends Source[Out] { + type MaterializedType } diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl2/package.scala b/akka-stream/src/main/scala/akka/stream/scaladsl2/package.scala index a9c37956e6..b8654a43d3 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl2/package.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl2/package.scala @@ -5,10 +5,9 @@ package akka.stream /** * Scala API: The flow DSL allows the formulation of stream transformations based on some - * input. The starting point is called [[Tap]] and can be a collection, an iterator, + * input. The starting point is called [[Source]] and can be a collection, an iterator, * a block of code which is evaluated repeatedly or a [[org.reactivestreams.Publisher]]. - * A flow with an attached input and open output, including `Tap` is a [[Source]], and is - * constructed with the `apply` methods in [[Source]]. + * A flow with an attached input and open output, is also a [[Source]]. * * A flow may also be defined without an attached input or output and that is then * a [[Flow]]. The `Flow` can be connected to the `Source` later by using [[Source#connect]] with @@ -18,9 +17,9 @@ package akka.stream * defined in [[FlowOps]]. Each DSL element produces a new flow that can be further transformed, * building up a description of the complete transformation pipeline. * - * The termination point of a flow is called [[Drain]] and can for example be a `Future` or - * [[org.reactivestreams.Subscriber]]. A flow with an attached output and open input, - * including `Drain`, is a [[Sink]]. + * The termination point of a flow is called [[Sink]] and can for example be a `Future` or + * [[org.reactivestreams.Subscriber]]. A flow with an attached output and open input + * is also a [[Sink]]. * * If a flow has both an attached input and an attached output it becomes a [[RunnableFlow]]. * In order to execute this pipeline the flow must be materialized by calling [[RunnableFlow#run]] on it.