diff --git a/akka-docs-dev/rst/scala/code/docs/http/HttpServerExampleSpec.scala b/akka-docs-dev/rst/scala/code/docs/http/HttpServerExampleSpec.scala index 471698934d..dbf5e6fa9d 100644 --- a/akka-docs-dev/rst/scala/code/docs/http/HttpServerExampleSpec.scala +++ b/akka-docs-dev/rst/scala/code/docs/http/HttpServerExampleSpec.scala @@ -7,9 +7,6 @@ package docs.http import akka.actor.ActorSystem import akka.http.model._ import akka.stream.testkit.AkkaSpec -import akka.util.Timeout - -import scala.concurrent.duration._ class HttpServerExampleSpec extends AkkaSpec("akka.actor.default-mailbox.mailbox-type = akka.dispatch.UnboundedMailbox") { @@ -18,45 +15,33 @@ class HttpServerExampleSpec "binding example" in { //#bind-example import akka.http.Http - import akka.io.IO - import akka.pattern.ask import akka.stream.FlowMaterializer - import akka.stream.scaladsl.Source implicit val system = ActorSystem() - import system.dispatcher implicit val materializer = FlowMaterializer() - implicit val askTimeout: Timeout = 500.millis - val bindingFuture = IO(Http) ? Http.Bind(interface = "localhost", port = 8080) - bindingFuture foreach { - case Http.ServerBinding(localAddress, connectionStream) ⇒ - Source(connectionStream).foreach({ - case Http.IncomingConnection(remoteAddress, requestProducer, responseConsumer) ⇒ - println("Accepted new connection from " + remoteAddress) + val Http.ServerSource(source, serverBindingKey) = Http(system).bind(interface = "localhost", port = 8080) + source.foreach { + case Http.IncomingConnection(remoteAddress, flow) ⇒ + println("Accepted new connection from " + remoteAddress) - // handle connection here - }) + // handle connection here } //#bind-example } + "full-server-example" in { import akka.http.Http - import akka.io.IO - import akka.pattern.ask import akka.stream.FlowMaterializer - import akka.stream.scaladsl.Source - import akka.stream.scaladsl.Sink implicit val system = ActorSystem() - import system.dispatcher implicit val materializer = FlowMaterializer() - implicit val askTimeout: Timeout = 500.millis - val bindingFuture = IO(Http) ? Http.Bind(interface = "localhost", port = 8080) + val Http.ServerSource(source, serverBindingKey) = Http(system).bind(interface = "localhost", port = 8080) //#full-server-example import akka.http.model.HttpMethods._ + import akka.stream.scaladsl.Flow val requestHandler: HttpRequest ⇒ HttpResponse = { case HttpRequest(GET, Uri.Path("/"), _, _, _) ⇒ @@ -70,14 +55,12 @@ class HttpServerExampleSpec } // ... - bindingFuture foreach { - case Http.ServerBinding(localAddress, connectionStream) ⇒ - Source(connectionStream).foreach({ - case Http.IncomingConnection(remoteAddress, requestProducer, responseConsumer) ⇒ - println("Accepted new connection from " + remoteAddress) - Source(requestProducer).map(requestHandler).to(Sink(responseConsumer)).run() - }) + source.foreach { + case Http.IncomingConnection(remoteAddress, flow) ⇒ + println("Accepted new connection from " + remoteAddress) + + flow.join(Flow[HttpRequest].map(requestHandler)).run() } //#full-server-example } diff --git a/akka-http-core/src/main/java/akka/http/model/japi/Http.java b/akka-http-core/src/main/java/akka/http/model/japi/Http.java index 3f4b327f47..e7ed550547 100644 --- a/akka-http-core/src/main/java/akka/http/model/japi/Http.java +++ b/akka-http-core/src/main/java/akka/http/model/japi/Http.java @@ -15,12 +15,4 @@ public final class Http { public static HttpExt get(ActorSystem system) { return (HttpExt) akka.http.Http.get(system); } - /** Create a Bind message to send to the Http Manager */ - public static Object bind(String host, int port) { - return Accessors$.MODULE$.Bind(host, port); - } - /** Create a Bind message to send to the Http Manager */ - public static Object bind(String host, int port, MaterializerSettings materializerSettings) { - return Accessors$.MODULE$.Bind(host, port, materializerSettings); - } } diff --git a/akka-http-core/src/main/scala/akka/http/Http.scala b/akka-http-core/src/main/scala/akka/http/Http.scala index 66d1be6f27..a4457fb069 100644 --- a/akka-http-core/src/main/scala/akka/http/Http.scala +++ b/akka-http-core/src/main/scala/akka/http/Http.scala @@ -6,150 +6,107 @@ package akka.http import java.io.Closeable import java.net.InetSocketAddress -import com.typesafe.config.Config -import org.reactivestreams.{ Publisher, Subscriber } +import akka.stream.io.StreamTcp +import akka.stream.scaladsl._ import scala.collection.immutable import akka.io.Inet -import akka.stream.MaterializerSettings -import akka.http.engine.client.ClientConnectionSettings -import akka.http.engine.server.ServerSettings -import akka.http.model.{ ErrorInfo, HttpResponse, HttpRequest, japi } -import akka.http.util._ +import akka.http.engine.client.{ HttpClientPipeline, ClientConnectionSettings } +import akka.http.engine.server.{ HttpServerPipeline, ServerSettings } +import akka.http.model.{ ErrorInfo, HttpResponse, HttpRequest } import akka.actor._ -object Http extends ExtensionKey[HttpExt] { +import scala.concurrent.Future + +object Http extends ExtensionKey[HttpExt] with ExtensionIdProvider { /** - * Command that can be sent to `IO(Http)` to trigger the setup of an HTTP client facility at - * a certain API level (connection, host or request). - * The HTTP layer will respond with an `Http.OutgoingChannel` reply (or `Status.Failure`). - * The sender `ActorRef`of this response can then be sent `HttpRequest` instances to which - * it will respond with `HttpResponse` instances (or `Status.Failure`). + * A flow representing an outgoing HTTP connection, and the key used to get information about + * the materialized connection. The flow takes pairs of a ``HttpRequest`` and a user definable + * context that will be correlated with the corresponding ``HttpResponse``. */ - sealed trait SetupOutgoingChannel - - final case class Connect(remoteAddress: InetSocketAddress, - localAddress: Option[InetSocketAddress], - options: immutable.Traversable[Inet.SocketOption], - settings: Option[ClientConnectionSettings], - materializerSettings: Option[MaterializerSettings]) extends SetupOutgoingChannel - object Connect { - def apply(host: String, port: Int = 80, - localAddress: Option[InetSocketAddress] = None, - options: immutable.Traversable[Inet.SocketOption] = Nil, - settings: Option[ClientConnectionSettings] = None, - materializerSettings: Option[MaterializerSettings] = None): Connect = - apply(new InetSocketAddress(host, port), localAddress, options, settings, materializerSettings) - } - - // PREVIEW OF COMING API HERE: - // - // case class SetupHostConnector(host: String, port: Int = 80, - // options: immutable.Traversable[Inet.SocketOption] = Nil, - // settings: Option[HostConnectorSettings] = None, - // connectionType: ClientConnectionType = ClientConnectionType.AutoProxied, - // defaultHeaders: immutable.Seq[HttpHeader] = Nil) extends SetupOutgoingChannel { - // private[http] def normalized(implicit refFactory: ActorRefFactory) = - // if (settings.isDefined) this - // else copy(settings = Some(HostConnectorSettings(actorSystem))) - // } - // object SetupHostConnector { - // def apply(host: String, port: Int, sslEncryption: Boolean)(implicit refFactory: ActorRefFactory): SetupHostConnector = - // apply(host, port, sslEncryption).normalized - // } - // sealed trait ClientConnectionType - // object ClientConnectionType { - // object Direct extends ClientConnectionType - // object AutoProxied extends ClientConnectionType - // final case class Proxied(proxyHost: String, proxyPort: Int) extends ClientConnectionType - // } - // - // case object SetupRequestChannel extends SetupOutgoingChannel + final case class OutgoingFlow(flow: Flow[(HttpRequest, Any), (HttpResponse, Any)], + key: Key { type MaterializedType = Future[Http.OutgoingConnection] }) /** - * An `OutgoingHttpChannel` with a single outgoing HTTP connection as the underlying transport. + * The materialized result of an outgoing HTTP connection stream with a single connection as the underlying transport. */ - // FIXME: hook up with new style IO final case class OutgoingConnection(remoteAddress: InetSocketAddress, - localAddress: InetSocketAddress, - responsePublisher: Publisher[(HttpResponse, Any)], - requestSubscriber: Subscriber[(HttpRequest, Any)]) { - } + localAddress: InetSocketAddress) - // PREVIEW OF COMING API HERE: - // - // /** - // * An `OutgoingHttpChannel` with a connection pool to a specific host/port as the underlying transport. - // */ - // final case class HostChannel(host: String, port: Int, - // untypedProcessor: HttpClientProcessor[Any]) extends OutgoingChannel { - // def processor[T] = untypedProcessor.asInstanceOf[HttpClientProcessor[T]] - // } - // - // /** - // * A general `OutgoingHttpChannel` with connection pools to all possible host/port combinations - // * as the underlying transport. - // */ - // final case class RequestChannel(untypedProcessor: HttpClientProcessor[Any]) extends OutgoingChannel { - // def processor[T] = untypedProcessor.asInstanceOf[HttpClientProcessor[T]] - // } + /** + * A source representing an bound HTTP server socket, and the key to get information about + * the materialized bound socket. + */ + final case class ServerSource(source: Source[IncomingConnection], + key: Key { type MaterializedType = Future[ServerBinding] }) - final case class Bind(endpoint: InetSocketAddress, - backlog: Int, - options: immutable.Traversable[Inet.SocketOption], - serverSettings: Option[ServerSettings], - materializerSettings: Option[MaterializerSettings]) - object Bind { - def apply(interface: String, port: Int = 80, backlog: Int = 100, - options: immutable.Traversable[Inet.SocketOption] = Nil, - serverSettings: Option[ServerSettings] = None, - materializerSettings: Option[MaterializerSettings] = None): Bind = - apply(new InetSocketAddress(interface, port), backlog, options, serverSettings, materializerSettings) - } + /** + * An incoming HTTP connection. + */ + final case class IncomingConnection(remoteAddress: InetSocketAddress, stream: Flow[HttpResponse, HttpRequest]) - sealed abstract case class ServerBinding(localAddress: InetSocketAddress, - connectionStream: Publisher[IncomingConnection]) extends model.japi.ServerBinding with Closeable { - /** Java API */ - def getConnectionStream: Publisher[japi.IncomingConnection] = connectionStream.asInstanceOf[Publisher[japi.IncomingConnection]] - } + class StreamException(val info: ErrorInfo) extends RuntimeException(info.summary) - object ServerBinding { - def apply(localAddress: InetSocketAddress, connectionStream: Publisher[IncomingConnection]): ServerBinding = - new ServerBinding(localAddress, connectionStream) { - override def close() = () - } + /** + * The materialized result of a bound HTTP server socket. + */ + private[akka] sealed abstract case class ServerBinding(localAddress: InetSocketAddress) extends Closeable - def apply(localAddress: InetSocketAddress, connectionStream: Publisher[IncomingConnection], closeable: Closeable): ServerBinding = - new ServerBinding(localAddress, connectionStream) { + /** + * INTERNAL API + */ + private[akka] object ServerBinding { + def apply(localAddress: InetSocketAddress, closeable: Closeable): ServerBinding = + new ServerBinding(localAddress) { override def close() = closeable.close() } } - - final case class IncomingConnection(remoteAddress: InetSocketAddress, - requestPublisher: Publisher[HttpRequest], - responseSubscriber: Subscriber[HttpResponse]) extends model.japi.IncomingConnection { - /** Java API */ - def getRequestPublisher: Publisher[japi.HttpRequest] = requestPublisher.asInstanceOf[Publisher[japi.HttpRequest]] - /** Java API */ - def getResponseSubscriber: Subscriber[japi.HttpResponse] = responseSubscriber.asInstanceOf[Subscriber[japi.HttpResponse]] - } - - case object BindFailedException extends SingletonException - - class ConnectionException(message: String) extends RuntimeException(message) - - class ConnectionAttemptFailedException(val endpoint: InetSocketAddress) extends ConnectionException(s"Connection attempt to $endpoint failed") - - class RequestTimeoutException(val request: HttpRequest, message: String) extends ConnectionException(message) - - class StreamException(val info: ErrorInfo) extends RuntimeException(info.summary) } -class HttpExt(system: ExtendedActorSystem) extends akka.io.IO.Extension { - val Settings = new Settings(system.settings.config getConfig "akka.http") - class Settings private[HttpExt] (config: Config) { - val ManagerDispatcher = config getString "manager-dispatcher" +class HttpExt(system: ExtendedActorSystem) extends Extension { + @volatile private[this] var clientPipelines = Map.empty[ClientConnectionSettings, HttpClientPipeline] + + def connect(remoteAddress: InetSocketAddress, + localAddress: Option[InetSocketAddress], + options: immutable.Traversable[Inet.SocketOption], + settings: Option[ClientConnectionSettings]): Http.OutgoingFlow = { + // FIXME #16378 Where to do logging? log.debug("Attempting connection to {}", remoteAddress) + val effectiveSettings = ClientConnectionSettings(settings)(system) + + val tcpFlow = StreamTcp(system).connect(remoteAddress, localAddress, options, effectiveSettings.connectingTimeout) + val pipeline = clientPipelines.getOrElse(effectiveSettings, { + val pl = new HttpClientPipeline(effectiveSettings, system.log)(system.dispatcher) + clientPipelines = clientPipelines.updated(effectiveSettings, pl) + pl + }) + pipeline(tcpFlow, remoteAddress) } - val manager = system.actorOf(props = HttpManager.props(Settings), name = "IO-HTTP") -} \ No newline at end of file + def connect(host: String, port: Int = 80, + localAddress: Option[InetSocketAddress] = None, + options: immutable.Traversable[Inet.SocketOption] = Nil, + settings: Option[ClientConnectionSettings] = None): Http.OutgoingFlow = + connect(new InetSocketAddress(host, port), localAddress, options, settings) + + def bind(endpoint: InetSocketAddress, + backlog: Int, + options: immutable.Traversable[Inet.SocketOption], + serverSettings: Option[ServerSettings]): Http.ServerSource = { + import system.dispatcher + + // FIXME IdleTimeout? + val src = StreamTcp(system).bind(endpoint, backlog, options) + val key = new Key { + override type MaterializedType = Future[Http.ServerBinding] + override def materialize(map: MaterializedMap) = map.get(src).map(s ⇒ Http.ServerBinding(s.localAddress, s)) + } + val log = system.log + val effectiveSettings = ServerSettings(serverSettings)(system) + Http.ServerSource(src.withKey(key).map(new HttpServerPipeline(effectiveSettings, log)), key) + } + + def bind(interface: String, port: Int = 80, backlog: Int = 100, + options: immutable.Traversable[Inet.SocketOption] = Nil, + serverSettings: Option[ServerSettings] = None): Http.ServerSource = + bind(new InetSocketAddress(interface, port), backlog, options, serverSettings) +} diff --git a/akka-http-core/src/main/scala/akka/http/HttpManager.scala b/akka-http-core/src/main/scala/akka/http/HttpManager.scala deleted file mode 100644 index f741bdbe5f..0000000000 --- a/akka-http-core/src/main/scala/akka/http/HttpManager.scala +++ /dev/null @@ -1,84 +0,0 @@ -/** - * Copyright (C) 2009-2014 Typesafe Inc. - */ - -package akka.http - -import scala.util.{ Failure, Success } -import scala.concurrent.duration._ -import akka.actor._ -import akka.http.engine.client._ -import akka.http.engine.server.{ HttpServerPipeline, ServerSettings } -import akka.io.IO -import akka.pattern.ask -import akka.stream.FlowMaterializer -import akka.stream.scaladsl.{ Sink, Source } -import akka.stream.io.StreamTcp -import akka.util.Timeout - -/** - * INTERNAL API - * - * The gateway actor into the low-level HTTP layer. - */ -private[http] class HttpManager(httpSettings: HttpExt#Settings) extends Actor with ActorLogging { - import context.dispatcher - - private[this] var clientPipelines = Map.empty[ClientConnectionSettings, HttpClientPipeline] - - def receive = { - case connect @ Http.Connect(remoteAddress, localAddress, options, clientConnectionSettings, materializerSettings) ⇒ - log.debug("Attempting connection to {}", remoteAddress) - val commander = sender() - val effectiveSettings = ClientConnectionSettings(clientConnectionSettings) - - val tcpConnect = StreamTcp.Connect(remoteAddress, localAddress, materializerSettings, options, - effectiveSettings.connectingTimeout, effectiveSettings.idleTimeout) - val askTimeout = Timeout(effectiveSettings.connectingTimeout + 5.seconds) // FIXME: how can we improve this? - val tcpConnectionFuture = IO(StreamTcp)(context.system).ask(tcpConnect)(askTimeout) - tcpConnectionFuture onComplete { - case Success(tcpConn: StreamTcp.OutgoingTcpConnection) ⇒ - val pipeline = clientPipelines.getOrElse(effectiveSettings, { - val pl = new HttpClientPipeline(effectiveSettings, log)(FlowMaterializer(materializerSettings)) - clientPipelines = clientPipelines.updated(effectiveSettings, pl) - pl - }) - commander ! pipeline(tcpConn) - - case Failure(error) ⇒ - log.debug("Could not connect to {} due to {}", remoteAddress, error) - commander ! Status.Failure(new Http.ConnectionAttemptFailedException(remoteAddress)) - - case x ⇒ throw new IllegalStateException("Unexpected response to `Connect` from StreamTcp: " + x) - } - - case Http.Bind(endpoint, backlog, options, serverSettings, materializerSettings) ⇒ - log.debug("Binding to {}", endpoint) - val commander = sender() - val effectiveSettings = ServerSettings(serverSettings) - val tcpBind = StreamTcp.Bind(endpoint, materializerSettings, backlog, options) - val askTimeout = Timeout(effectiveSettings.bindTimeout + 5.seconds) // FIXME: how can we improve this? - val tcpServerBindingFuture = IO(StreamTcp)(context.system).ask(tcpBind)(askTimeout) - tcpServerBindingFuture onComplete { - case Success(tcpServerBinding @ StreamTcp.TcpServerBinding(localAddress, connectionStream)) ⇒ - log.info("Bound to {}", endpoint) - implicit val materializer = FlowMaterializer() - val httpServerPipeline = new HttpServerPipeline(effectiveSettings, log) - val httpConnectionStream = Source(connectionStream) - .map(httpServerPipeline) - .runWith(Sink.publisher) - commander ! Http.ServerBinding(localAddress, httpConnectionStream, tcpServerBinding) - - case Failure(error) ⇒ - log.warning("Bind to {} failed due to {}", endpoint, error) - commander ! Status.Failure(Http.BindFailedException) - - case x ⇒ throw new IllegalStateException("Unexpected response to `Bind` from StreamTcp: " + x) - } - } -} - -private[http] object HttpManager { - def props(httpSettings: HttpExt#Settings) = - Props(classOf[HttpManager], httpSettings) withDispatcher httpSettings.ManagerDispatcher -} \ No newline at end of file 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 d36f4c07ee..f0ca8e73e3 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 @@ -8,7 +8,6 @@ import java.net.InetSocketAddress import scala.collection.immutable.Queue import akka.stream.scaladsl._ import akka.event.LoggingAdapter -import akka.stream.FlowMaterializer import akka.stream.FlattenStrategy import akka.stream.io.StreamTcp import akka.util.ByteString @@ -19,12 +18,14 @@ import akka.http.engine.parsing.{ HttpRequestParser, HttpHeaderParser, HttpRespo import akka.http.engine.parsing.ParserOutput._ import akka.http.util._ +import scala.concurrent.{ ExecutionContext, Future } + /** * INTERNAL API */ private[http] class HttpClientPipeline(effectiveSettings: ClientConnectionSettings, - log: LoggingAdapter)(implicit fm: FlowMaterializer) - extends (StreamTcp.OutgoingTcpConnection ⇒ Http.OutgoingConnection) { + log: LoggingAdapter)(implicit ec: ExecutionContext) + extends ((StreamTcp.OutgoingTcpFlow, InetSocketAddress) ⇒ Http.OutgoingFlow) { import effectiveSettings._ @@ -38,23 +39,24 @@ private[http] class HttpClientPipeline(effectiveSettings: ClientConnectionSettin val requestRendererFactory = new HttpRequestRendererFactory(userAgentHeader, requestHeaderSizeHint, log) - def apply(tcpConn: StreamTcp.OutgoingTcpConnection): Http.OutgoingConnection = { + def apply(tcpFlow: StreamTcp.OutgoingTcpFlow, remoteAddress: InetSocketAddress): Http.OutgoingFlow = { import FlowGraphImplicits._ - val requestMethodByPass = new RequestMethodByPass(tcpConn.remoteAddress) + val httpKey = new HttpKey(tcpFlow.key) - val userIn = Source.subscriber[(HttpRequest, Any)] - val userOut = Sink.publisher[(HttpResponse, Any)] + val flowWithHttpKey = tcpFlow.flow.withKey(httpKey) - val netOut = Sink(tcpConn.outputStream) - val netIn = Source(tcpConn.inputStream) + val requestMethodByPass = new RequestMethodByPass(remoteAddress) - val pipeline = FlowGraph { implicit b ⇒ - val bypassFanout = Broadcast[(HttpRequest, Any)]("bypassFanout") + val pipeline = Flow() { implicit b ⇒ + val userIn = UndefinedSource[(HttpRequest, Any)] + val userOut = UndefinedSink[(HttpResponse, Any)] + + val bypassFanout = Unzip[HttpRequest, Any]("bypassFanout") val bypassFanin = Zip[HttpResponse, Any]("bypassFanin") val requestPipeline = - Flow[(HttpRequest, Any)] + Flow[HttpRequest] .map(requestMethodByPass) .transform("renderer", () ⇒ requestRendererFactory.newRenderer) .flatten(FlattenStrategy.concat) @@ -74,24 +76,21 @@ private[http] class HttpClientPipeline(effectiveSettings: ClientConnectionSettin } //FIXME: the graph is unnecessary after fixing #15957 - userIn ~> bypassFanout ~> requestPipeline ~> netOut - bypassFanout ~> Flow[(HttpRequest, Any)].map(_._2) ~> bypassFanin.right - netIn ~> responsePipeline ~> bypassFanin.left + userIn ~> bypassFanout.in + bypassFanout.left ~> requestPipeline ~> flowWithHttpKey ~> responsePipeline ~> bypassFanin.left + bypassFanout.right ~> bypassFanin.right bypassFanin.out ~> userOut - }.run() - Http.OutgoingConnection( - tcpConn.remoteAddress, - tcpConn.localAddress, - pipeline.get(userOut), - pipeline.get(userIn)) + userIn -> userOut + } + + Http.OutgoingFlow(pipeline, httpKey) } class RequestMethodByPass(serverAddress: InetSocketAddress) - extends (((HttpRequest, Any)) ⇒ RequestRenderingContext) with (() ⇒ HttpMethod) { + extends ((HttpRequest) ⇒ RequestRenderingContext) with (() ⇒ HttpMethod) { private[this] var requestMethods = Queue.empty[HttpMethod] - def apply(tuple: (HttpRequest, Any)) = { - val request = tuple._1 + def apply(request: HttpRequest) = { requestMethods = requestMethods.enqueue(request.method) RequestRenderingContext(request, serverAddress) } @@ -102,4 +101,11 @@ private[http] class HttpClientPipeline(effectiveSettings: ClientConnectionSettin method } else HttpResponseParser.NoMethod } + + class HttpKey(tcpKey: Key { type MaterializedType = Future[StreamTcp.OutgoingTcpConnection] }) extends Key { + type MaterializedType = Future[Http.OutgoingConnection] + + override def materialize(map: MaterializedMap): MaterializedType = + map.get(tcpKey).map(tcp ⇒ Http.OutgoingConnection(tcp.remoteAddress, tcp.localAddress)) + } } 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 b4c61292b0..af94c742f5 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 @@ -24,8 +24,7 @@ import scala.util.control.NonFatal /** * INTERNAL API */ -private[http] class HttpServerPipeline(settings: ServerSettings, - log: LoggingAdapter)(implicit fm: FlowMaterializer) +private[http] class HttpServerPipeline(settings: ServerSettings, log: LoggingAdapter) extends (StreamTcp.IncomingTcpConnection ⇒ Http.IncomingConnection) { import settings.parserSettings @@ -40,55 +39,62 @@ private[http] class HttpServerPipeline(settings: ServerSettings, val responseRendererFactory = new HttpResponseRendererFactory(settings.serverHeader, settings.responseHeaderSizeHint, log) + val bypassFanout = Broadcast[RequestOutput]("bypassFanout") + + val bypassMerge = new BypassMerge + + val requestPreparation = + Flow[RequestOutput] + .splitWhen(x ⇒ x.isInstanceOf[MessageStart] || x == MessageEnd) + .headAndTail + .collect { + case (RequestStart(method, uri, protocol, headers, createEntity, _, _), entityParts) ⇒ + val effectiveUri = HttpRequest.effectiveUri(uri, headers, securedConnection = false, settings.defaultHostHeader) + val effectiveMethod = if (method == HttpMethods.HEAD && settings.transparentHeadRequests) HttpMethods.GET else method + HttpRequest(effectiveMethod, effectiveUri, headers, createEntity(entityParts), protocol) + } + + val rendererPipeline = + Flow[ResponseRenderingContext] + .transform("recover", () ⇒ new ErrorsTo500ResponseRecovery(log)) // FIXME: simplify after #16394 is closed + .transform("renderer", () ⇒ responseRendererFactory.newRenderer) + .flatten(FlattenStrategy.concat) + .transform("errorLogger", () ⇒ errorLogger(log, "Outgoing response stream error")) + def apply(tcpConn: StreamTcp.IncomingTcpConnection): Http.IncomingConnection = { import FlowGraphImplicits._ - val networkIn = Source(tcpConn.inputStream) - val networkOut = Sink(tcpConn.outputStream) + val userIn = UndefinedSink[HttpRequest] + val userOut = UndefinedSource[HttpResponse] - val userIn = Sink.publisher[HttpRequest] - val userOut = Source.subscriber[HttpResponse] - - val oneHundredContinueSource = Source[OneHundredContinue.type](Props[OneHundredContinueSourceActor]) @volatile var oneHundredContinueRef: Option[ActorRef] = None // FIXME: unnecessary after fixing #16168 + val oneHundredContinueSource = Source[OneHundredContinue.type] { + Props { + val actor = new OneHundredContinueSourceActor + oneHundredContinueRef = Some(actor.context.self) + actor + } + } - val pipeline = FlowGraph { implicit b ⇒ - val bypassFanout = Broadcast[RequestOutput]("bypassFanout") - val bypassMerge = new BypassMerge + // FIXME The whole pipeline can maybe be created up front when #16168 is fixed + val pipeline = Flow() { implicit b ⇒ val requestParsing = Flow[ByteString].transform("rootParser", () ⇒ // each connection uses a single (private) request parser instance for all its requests // which builds a cache of all header instances seen on that connection rootParser.createShallowCopy(() ⇒ oneHundredContinueRef)) - val requestPreparation = - Flow[RequestOutput] - .splitWhen(x ⇒ x.isInstanceOf[MessageStart] || x == MessageEnd) - .headAndTail - .collect { - case (RequestStart(method, uri, protocol, headers, createEntity, _, _), entityParts) ⇒ - val effectiveUri = HttpRequest.effectiveUri(uri, headers, securedConnection = false, settings.defaultHostHeader) - val effectiveMethod = if (method == HttpMethods.HEAD && settings.transparentHeadRequests) HttpMethods.GET else method - HttpRequest(effectiveMethod, effectiveUri, headers, createEntity(entityParts), protocol) - } - - val rendererPipeline = - Flow[ResponseRenderingContext] - .transform("recover", () ⇒ new ErrorsTo500ResponseRecovery(log)) // FIXME: simplify after #16394 is closed - .transform("renderer", () ⇒ responseRendererFactory.newRenderer) - .flatten(FlattenStrategy.concat) - .transform("errorLogger", () ⇒ errorLogger(log, "Outgoing response stream error")) - //FIXME: the graph is unnecessary after fixing #15957 - networkIn ~> requestParsing ~> bypassFanout ~> requestPreparation ~> userIn + userOut ~> bypassMerge.applicationInput ~> rendererPipeline ~> tcpConn.stream ~> requestParsing ~> bypassFanout ~> requestPreparation ~> userIn bypassFanout ~> bypassMerge.bypassInput - userOut ~> bypassMerge.applicationInput ~> rendererPipeline ~> networkOut oneHundredContinueSource ~> bypassMerge.oneHundredContinueInput - }.run() - oneHundredContinueRef = Some(pipeline.get(oneHundredContinueSource)) + b.allowCycles() - Http.IncomingConnection(tcpConn.remoteAddress, pipeline.get(userIn), pipeline.get(userOut)) + userOut -> userIn + } + + Http.IncomingConnection(tcpConn.remoteAddress, pipeline) } class BypassMerge extends FlexiMerge[ResponseRenderingContext]("BypassMerge") { diff --git a/akka-http-core/src/main/scala/akka/http/model/japi/Accessors.scala b/akka-http-core/src/main/scala/akka/http/model/japi/Accessors.scala index d9cf25fd7b..80b155305d 100644 --- a/akka-http-core/src/main/scala/akka/http/model/japi/Accessors.scala +++ b/akka-http-core/src/main/scala/akka/http/model/japi/Accessors.scala @@ -20,11 +20,4 @@ private[http] object Accessors { /** INTERNAL API */ private[http] def Uri(uri: model.Uri): Uri = JavaUri(uri) - /** INTERNAL API */ - private[http] def Bind(host: String, port: Int): AnyRef = - akka.http.Http.Bind(host, port, materializerSettings = None) - - /** INTERNAL API */ - private[http] def Bind(host: String, port: Int, materializerSettings: MaterializerSettings): AnyRef = - akka.http.Http.Bind(host, port, materializerSettings = Some(materializerSettings)) } diff --git a/akka-http-core/src/main/scala/akka/http/util/StreamUtils.scala b/akka-http-core/src/main/scala/akka/http/util/StreamUtils.scala index 276b855b70..d2f2c24288 100644 --- a/akka-http-core/src/main/scala/akka/http/util/StreamUtils.scala +++ b/akka-http-core/src/main/scala/akka/http/util/StreamUtils.scala @@ -181,7 +181,8 @@ private[http] object StreamUtils { def runStrict(sourceData: ByteString, transformer: Flow[ByteString, ByteString], maxByteSize: Long, maxElements: Int): Try[Option[ByteString]] = Try { transformer match { - case Pipe(ops) ⇒ + // FIXME #16382 right now the flow can't use keys, should that be allowed? + case Pipe(ops, keys) if keys.isEmpty ⇒ if (ops.isEmpty) Some(sourceData) else { 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 7a24c95d5b..c98161c75f 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 @@ -19,39 +19,41 @@ import java.io.IOException; import java.io.InputStreamReader; public abstract class JavaTestServer { - public static void main(String[] args) throws IOException, InterruptedException { - ActorSystem system = ActorSystem.create(); - final FlowMaterializer materializer = FlowMaterializer.create(system); - - ActorRef httpManager = Http.get(system).manager(); - Future binding = ask(httpManager, Http.bind("localhost", 8080), 1000); - binding.foreach(new Foreach() { - @Override - public void each(Object result) throws Throwable { - ServerBinding binding = (ServerBinding) result; - System.out.println("Bound to " + binding.localAddress()); - - Source.from(binding.getConnectionStream()).foreach(new akka.stream.javadsl.japi.Procedure() { - @Override - public void apply(IncomingConnection conn) throws Exception { - System.out.println("New incoming connection from " + conn.remoteAddress()); - - Source.from(conn.getRequestPublisher()).map(new akka.stream.javadsl.japi.Function() { - @Override - public HttpResponse apply(HttpRequest request) throws Exception { - System.out.println("Handling request to " + request.getUri()); - return JavaApiTestCases.handleRequest(request); - } - }).runWith(Sink.create(conn.getResponseSubscriber()), materializer); - } - }, materializer); - } - }, system.dispatcher()); - - System.out.println("Press ENTER to stop."); - new BufferedReader(new InputStreamReader(System.in)).readLine(); - - system.shutdown(); - } +// FIXME Java Http API +// public static void main(String[] args) throws IOException, InterruptedException { +// ActorSystem system = ActorSystem.create(); +// +// final FlowMaterializer materializer = FlowMaterializer.create(system); +// +// ActorRef httpManager = Http.get(system).manager(); +// Future binding = ask(httpManager, Http.bind("localhost", 8080), 1000); +// binding.foreach(new Foreach() { +// @Override +// public void each(Object result) throws Throwable { +// ServerBinding binding = (ServerBinding) result; +// System.out.println("Bound to " + binding.localAddress()); +// +// Source.from(binding.getConnectionStream()).foreach(new akka.stream.javadsl.japi.Procedure() { +// @Override +// public void apply(IncomingConnection conn) throws Exception { +// System.out.println("New incoming connection from " + conn.remoteAddress()); +// +// Source.from(conn.getRequestPublisher()).map(new akka.stream.javadsl.japi.Function() { +// @Override +// public HttpResponse apply(HttpRequest request) throws Exception { +// System.out.println("Handling request to " + request.getUri()); +// return JavaApiTestCases.handleRequest(request); +// } +// }).runWith(Sink.create(conn.getResponseSubscriber()), materializer); +// } +// }, materializer); +// } +// }, system.dispatcher()); +// +// System.out.println("Press ENTER to stop."); +// new BufferedReader(new InputStreamReader(System.in)).readLine(); +// +// system.shutdown(); +// } } 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 5f63675b35..da98db5163 100644 --- a/akka-http-core/src/test/scala/akka/http/ClientServerSpec.scala +++ b/akka-http-core/src/test/scala/akka/http/ClientServerSpec.scala @@ -7,16 +7,12 @@ package akka.http import java.io.{ BufferedReader, BufferedWriter, InputStreamReader, OutputStreamWriter } import java.net.Socket import com.typesafe.config.{ Config, ConfigFactory } -import org.reactivestreams.Publisher import scala.annotation.tailrec import scala.concurrent.Await import scala.concurrent.duration._ import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec } -import akka.actor.{ Status, ActorSystem } -import akka.io.IO -import akka.testkit.TestProbe +import akka.actor.ActorSystem import akka.stream.FlowMaterializer -import akka.stream.impl.SynchronousIterablePublisher import akka.stream.testkit.StreamTestKit import akka.stream.testkit.StreamTestKit.{ PublisherProbe, SubscriberProbe } import akka.stream.scaladsl._ @@ -42,33 +38,34 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll { "properly bind and unbind a server" in { val (hostname, port) = temporaryServerHostnameAndPort() - val commander = TestProbe() - commander.send(IO(Http), Http.Bind(hostname, port)) - - val Http.ServerBinding(localAddress, connectionStream) = commander.expectMsgType[Http.ServerBinding] + val Http.ServerSource(source, key) = Http(system).bind(hostname, port) + val c = StreamTestKit.SubscriberProbe[Http.IncomingConnection]() + val mm = source.to(Sink(c)).run() + val Http.ServerBinding(localAddress) = Await.result(mm.get(key), 3.seconds) + val sub = c.expectSubscription() localAddress.getHostName shouldEqual hostname localAddress.getPort shouldEqual port - val c = StreamTestKit.SubscriberProbe[Http.IncomingConnection]() - connectionStream.subscribe(c) - val sub = c.expectSubscription() - sub.cancel() + // TODO: verify unbinding effect } "report failure if bind fails" in { val (hostname, port) = temporaryServerHostnameAndPort() - val commander = TestProbe() - commander.send(IO(Http), Http.Bind(hostname, port)) - val binding = commander.expectMsgType[Http.ServerBinding] - commander.send(IO(Http), Http.Bind(hostname, port)) - commander.expectMsgType[Status.Failure] - - // Clean up - val c = StreamTestKit.SubscriberProbe[Http.IncomingConnection]() - binding.connectionStream.subscribe(c) - val sub = c.expectSubscription() + val Http.ServerSource(source, key) = Http(system).bind(hostname, port) + val c1 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]() + val mm1 = source.to(Sink(c1)).run() + val sub = c1.expectSubscription() + val Http.ServerBinding(localAddress) = Await.result(mm1.get(key), 3.seconds) + localAddress.getHostName shouldEqual hostname + localAddress.getPort shouldEqual port + val c2 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]() + val mm2 = source.to(Sink(c2)).run() + val failure = intercept[akka.stream.io.StreamTcp.IncomingTcpException] { + val serverBinding = Await.result(mm2.get(key), 3.seconds) + } + failure.getMessage should be("Bind failed") sub.cancel() } @@ -126,40 +123,38 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll { class TestSetup { val (hostname, port) = temporaryServerHostnameAndPort() - val bindHandler = TestProbe() def configOverrides = "" // automatically bind a server val connectionStream: SubscriberProbe[Http.IncomingConnection] = { - val commander = TestProbe() val settings = configOverrides.toOption.map(ServerSettings.apply) - commander.send(IO(Http), Http.Bind(hostname, port, serverSettings = settings)) + val Http.ServerSource(source, key) = Http(system).bind(hostname, port, serverSettings = settings) val probe = StreamTestKit.SubscriberProbe[Http.IncomingConnection] - commander.expectMsgType[Http.ServerBinding].connectionStream.subscribe(probe) + source.to(Sink(probe)).run() probe } val connectionStreamSub = connectionStream.expectSubscription() def openNewClientConnection[T](settings: Option[ClientConnectionSettings] = None): (PublisherProbe[(HttpRequest, T)], SubscriberProbe[(HttpResponse, T)]) = { - val commander = TestProbe() - commander.send(IO(Http), Http.Connect(hostname, port, settings = settings)) - val connection = commander.expectMsgType[Http.OutgoingConnection] - connection.remoteAddress.getPort shouldEqual port - connection.remoteAddress.getHostName shouldEqual hostname + val outgoingFlow = Http(system).connect(hostname, port, settings = settings) val requestPublisherProbe = StreamTestKit.PublisherProbe[(HttpRequest, T)]() val responseSubscriberProbe = StreamTestKit.SubscriberProbe[(HttpResponse, T)]() - requestPublisherProbe.subscribe(connection.requestSubscriber) - connection.responsePublisher.asInstanceOf[Publisher[(HttpResponse, T)]].subscribe(responseSubscriberProbe) + val tflow = outgoingFlow.flow.asInstanceOf[Flow[((HttpRequest, T)), ((HttpResponse, T))]] + val mm = Flow(Sink(responseSubscriberProbe), Source(requestPublisherProbe)).join(tflow).run() + val connection = Await.result(mm.get(outgoingFlow.key), 3.seconds) + connection.remoteAddress.getPort shouldEqual port + connection.remoteAddress.getHostName shouldEqual hostname + requestPublisherProbe -> responseSubscriberProbe } def acceptConnection(): (SubscriberProbe[HttpRequest], PublisherProbe[HttpResponse]) = { connectionStreamSub.request(1) - val Http.IncomingConnection(_, requestPublisher, responseSubscriber) = connectionStream.expectNext() + val Http.IncomingConnection(remoteAddress, flow) = connectionStream.expectNext() val requestSubscriberProbe = StreamTestKit.SubscriberProbe[HttpRequest]() val responsePublisherProbe = StreamTestKit.PublisherProbe[HttpResponse]() - requestPublisher.subscribe(requestSubscriberProbe) - responsePublisherProbe.subscribe(responseSubscriber) + Flow(Sink(requestSubscriberProbe), Source(responsePublisherProbe)).join(flow).run() + requestSubscriberProbe -> responsePublisherProbe } 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 d5c88bb917..4aca8607d6 100644 --- a/akka-http-core/src/test/scala/akka/http/TestClient.scala +++ b/akka-http-core/src/test/scala/akka/http/TestClient.scala @@ -5,17 +5,10 @@ package akka.http import com.typesafe.config.{ Config, ConfigFactory } -import org.reactivestreams.Subscriber -import scala.concurrent.Future -import scala.concurrent.duration._ import scala.util.{ Failure, Success } import akka.actor.ActorSystem -import akka.pattern.ask -import akka.util.Timeout import akka.stream.FlowMaterializer import akka.stream.scaladsl.{ Sink, Source } -import akka.io.IO -import akka.http.model.HttpMethods._ import akka.http.model._ object TestClient extends App { @@ -27,26 +20,16 @@ object TestClient extends App { import akka.http.TestClient.system.dispatcher implicit val materializer = FlowMaterializer() - implicit val askTimeout: Timeout = 500.millis val host = "spray.io" println(s"Fetching HTTP server version of host `$host` ...") - val result = for { - connection ← IO(Http).ask(Http.Connect(host)).mapTo[Http.OutgoingConnection] - response ← sendRequest(HttpRequest(GET, uri = "/"), connection) - } yield response.header[headers.Server] + val outgoingFlow = Http(system).connect(host) + val result = Source.singleton(HttpRequest() -> 'NoContext).via(outgoingFlow.flow).map(_._1).runWith(Sink.head) - def sendRequest(request: HttpRequest, connection: Http.OutgoingConnection): Future[HttpResponse] = { - Source.singleton(HttpRequest() -> 'NoContext) - .to(Sink(connection.requestSubscriber)) - .run() - Source(connection.responsePublisher).map(_._1).runWith(Sink.head) - } - - result onComplete { + result.map(_.header[headers.Server]) onComplete { case Success(res) ⇒ println(s"$host is running ${res mkString ", "}") case Failure(error) ⇒ println(s"Error: $error") } result onComplete { _ ⇒ system.shutdown() } -} +} \ No newline at end of file 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 29ed192a07..b5ea8e0d3e 100644 --- a/akka-http-core/src/test/scala/akka/http/TestServer.scala +++ b/akka-http-core/src/test/scala/akka/http/TestServer.scala @@ -4,16 +4,14 @@ package akka.http -import com.typesafe.config.{ ConfigFactory, Config } -import scala.concurrent.duration._ -import akka.stream.FlowMaterializer -import akka.stream.scaladsl.{ Sink, Source } -import akka.io.IO -import akka.util.Timeout import akka.actor.ActorSystem -import akka.pattern.ask import akka.http.model._ +import akka.stream.FlowMaterializer +import akka.stream.scaladsl.{ Flow, Sink } +import com.typesafe.config.{ ConfigFactory, Config } import HttpMethods._ +import scala.concurrent.Await +import scala.concurrent.duration._ object TestServer extends App { val testConf: Config = ConfigFactory.parseString(""" @@ -21,7 +19,6 @@ object TestServer extends App { akka.log-dead-letters = off """) implicit val system = ActorSystem("ServerTest", testConf) - import system.dispatcher val requestHandler: HttpRequest ⇒ HttpResponse = { case HttpRequest(GET, Uri.Path("/"), _, _, _) ⇒ index @@ -32,20 +29,20 @@ object TestServer extends App { implicit val materializer = FlowMaterializer() - implicit val askTimeout: Timeout = 500.millis - val bindingFuture = IO(Http) ? Http.Bind(interface = "localhost", port = 8080) - bindingFuture foreach { - case Http.ServerBinding(localAddress, connectionStream) ⇒ - Source(connectionStream).foreach { - case Http.IncomingConnection(remoteAddress, requestPublisher, responseSubscriber) ⇒ - println("Accepted new connection from " + remoteAddress) - Source(requestPublisher).map(requestHandler).to(Sink(responseSubscriber)).run() - } - } + val Http.ServerSource(source, key) = Http(system).bind(interface = "localhost", port = 8080) + val materializedMap = source.to(Sink.foreach { + case Http.IncomingConnection(remoteAddress, flow) ⇒ + println("Accepted new connection from " + remoteAddress) + flow.join(Flow[HttpRequest].map(requestHandler)).run() + }).run() - println(s"Server online at http://localhost:8080/\nPress RETURN to stop...") + val serverBinding = Await.result(materializedMap.get(key), 3 seconds) + println(s"Server online at http://${serverBinding.localAddress.getHostName}:${serverBinding.localAddress.getPort}") + println("Press RETURN to stop...") Console.readLine() + + serverBinding.close() system.shutdown() ////////////// helpers ////////////// 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 43f2fa82ed..446aa0567a 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 @@ -614,12 +614,12 @@ class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterA class TestSetup { val netIn = StreamTestKit.PublisherProbe[ByteString] val netOut = StreamTestKit.SubscriberProbe[ByteString] - val tcpConnection = StreamTcp.IncomingTcpConnection(null, netIn, netOut) + val tcpConnection = StreamTcp.IncomingTcpConnection(null, Flow(Sink(netOut), Source(netIn))) def settings = ServerSettings(system).copy(serverHeader = Some(Server(List(ProductVersion("akka-http", "test"))))) val pipeline = new HttpServerPipeline(settings, NoLogging) - val Http.IncomingConnection(_, requestsIn, responsesOut) = pipeline(tcpConnection) + val Http.IncomingConnection(_, httpPipelineFlow) = pipeline(tcpConnection) def wipeDate(string: String) = string.fastSplit('\n').map { @@ -627,14 +627,13 @@ class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterA case s ⇒ s }.mkString("\n") - val netInSub = netIn.expectSubscription() - val netOutSub = netOut.expectSubscription() - val requests = StreamTestKit.SubscriberProbe[HttpRequest] val responses = StreamTestKit.PublisherProbe[HttpResponse] - requestsIn.subscribe(requests) + Flow(Sink(requests), Source(responses)).join(httpPipelineFlow).run() + + val netInSub = netIn.expectSubscription() + val netOutSub = netOut.expectSubscription() val requestsSub = requests.expectSubscription() - responses.subscribe(responsesOut) val responsesSub = responses.expectSubscription() def expectRequest: HttpRequest = { diff --git a/akka-http-tests/src/test/scala/akka/http/server/TestServer.scala b/akka-http-tests/src/test/scala/akka/http/server/TestServer.scala index c3ec4475fa..2ff749c93c 100644 --- a/akka-http-tests/src/test/scala/akka/http/server/TestServer.scala +++ b/akka-http-tests/src/test/scala/akka/http/server/TestServer.scala @@ -9,10 +9,8 @@ import akka.http.server.directives.AuthenticationDirectives._ import com.typesafe.config.{ ConfigFactory, Config } import scala.concurrent.duration._ import akka.actor.ActorSystem -import akka.io.IO import akka.stream.FlowMaterializer import akka.util.Timeout -import akka.pattern.ask import akka.http.Http import akka.http.model._ @@ -26,7 +24,7 @@ object TestServer extends App { implicit val materializer = FlowMaterializer() implicit val askTimeout: Timeout = 500.millis - val bindingFuture = (IO(Http) ? Http.Bind(interface = "localhost", port = 8080)).mapTo[Http.ServerBinding] + val serverSource = Http(system).bind(interface = "localhost", port = 8080) import ScalaRoutingDSL._ @@ -40,7 +38,7 @@ object TestServer extends App { // of #16190 implicit val html = ScalaXmlSupport.nodeSeqMarshaller(MediaTypes.`text/html`) - handleConnections(bindingFuture) withRoute { + handleConnections(serverSource) withRoute { get { path("") { complete(index) 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 d02d05901e..ebfdb5555a 100644 --- a/akka-http/src/main/scala/akka/http/server/ScalaRoutingDSL.scala +++ b/akka-http/src/main/scala/akka/http/server/ScalaRoutingDSL.scala @@ -25,19 +25,8 @@ trait ScalaRoutingDSL extends Directives { def withAsyncHandler(handler: HttpRequest ⇒ Future[HttpResponse]): R } - def handleConnections(bindingFuture: Future[Http.ServerBinding])(implicit ec: ExecutionContext, fm: FlowMaterializer, - setupProvider: RoutingSetupProvider): Applicator[Future[Unit]] = - new Applicator[Future[Unit]] { - def withRoute(route: Route) = afterBinding(_ withRoute route) - def withSyncHandler(handler: HttpRequest ⇒ HttpResponse) = afterBinding(_ withSyncHandler handler) - def withAsyncHandler(handler: HttpRequest ⇒ Future[HttpResponse]) = afterBinding(_ withAsyncHandler handler) - - def afterBinding(f: Applicator[Unit] ⇒ Unit): Future[Unit] = - bindingFuture.map(binding ⇒ f(handleConnections(binding))) - } - - def handleConnections(binding: Http.ServerBinding)(implicit fm: FlowMaterializer, - setupProvider: RoutingSetupProvider): Applicator[Unit] = { + def handleConnections(serverSource: Http.ServerSource)(implicit fm: FlowMaterializer, + setupProvider: RoutingSetupProvider): Applicator[Unit] = { new Applicator[Unit] { def withRoute(route: Route): Unit = run(routeRunner(route, _)) @@ -49,14 +38,12 @@ trait ScalaRoutingDSL extends Directives { run(_ ⇒ handler) private def run(f: RoutingSetup ⇒ HttpRequest ⇒ Future[HttpResponse]): Unit = - Source(binding.connectionStream).foreach { - case connection @ Http.IncomingConnection(remoteAddress, requestProducer, responseConsumer) ⇒ + serverSource.source.foreach { + case connection @ Http.IncomingConnection(remoteAddress, flow) ⇒ val setup = setupProvider(connection) setup.routingLog.log.debug("Accepted new connection from " + remoteAddress) val runner = f(setup) - Source(requestProducer) - .mapAsync(request ⇒ runner(request)) - .to(Sink(responseConsumer)).run()(fm) + flow.join(Flow[HttpRequest].mapAsync(request ⇒ runner(request))).run()(fm) } } } diff --git a/akka-stream-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala b/akka-stream-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala index ffccf1074b..906ad5efb0 100644 --- a/akka-stream-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala +++ b/akka-stream-tck/src/test/scala/akka/stream/tck/FusableProcessorTest.scala @@ -5,10 +5,13 @@ package akka.stream.tck import java.util.concurrent.atomic.AtomicInteger import akka.stream.impl.{ Ast, ActorBasedFlowMaterializer } +import akka.stream.scaladsl.MaterializedMap import akka.stream.{ FlowMaterializer, MaterializerSettings } import org.reactivestreams.{ Publisher, Processor } import akka.stream.impl.fusing.Map +import scala.concurrent.Promise + class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] { val processorCounter = new AtomicInteger @@ -21,7 +24,7 @@ class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] { val flowName = getClass.getSimpleName + "-" + processorCounter.incrementAndGet() - val processor = materializer.asInstanceOf[ActorBasedFlowMaterializer].processorForNode( + val (processor, _ns) = materializer.asInstanceOf[ActorBasedFlowMaterializer].processorForNode( Ast.Fused(List(Map[Int, Int](identity)), "identity"), flowName, 1) processor.asInstanceOf[Processor[Int, Int]] diff --git a/akka-stream-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala b/akka-stream-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala index b8ca43d12f..c84a26b85e 100644 --- a/akka-stream-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala +++ b/akka-stream-tck/src/test/scala/akka/stream/tck/TransformProcessorTest.scala @@ -8,11 +8,14 @@ import akka.stream.impl.ActorBasedFlowMaterializer import akka.stream.impl.Ast import akka.stream.FlowMaterializer import java.util.concurrent.atomic.AtomicInteger +import akka.stream.scaladsl.MaterializedMap import org.reactivestreams.Processor import org.reactivestreams.Publisher import akka.stream.stage.PushStage import akka.stream.stage.Context +import scala.concurrent.Promise + class TransformProcessorTest extends AkkaIdentityProcessorVerification[Int] { val processorCounter = new AtomicInteger @@ -30,7 +33,7 @@ class TransformProcessorTest extends AkkaIdentityProcessorVerification[Int] { override def onPush(in: Any, ctx: Context[Any]) = ctx.push(in) } - val processor = materializer.asInstanceOf[ActorBasedFlowMaterializer].processorForNode( + val (processor, _) = materializer.asInstanceOf[ActorBasedFlowMaterializer].processorForNode( Ast.StageFactory(mkStage, "transform"), flowName, 1) processor.asInstanceOf[Processor[Int, Int]] diff --git a/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestKit.scala b/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestKit.scala index 471b1a567b..d921c2db4d 100644 --- a/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestKit.scala +++ b/akka-stream-testkit/src/test/scala/akka/stream/testkit/StreamTestKit.scala @@ -90,7 +90,11 @@ object StreamTestKit { def sendError(cause: Exception): Unit = subscriber.onError(cause) } - case class SubscriberProbe[I]()(implicit system: ActorSystem) extends Subscriber[I] { + object SubscriberProbe { + def apply[I]()(implicit system: ActorSystem): SubscriberProbe[I] = new SubscriberProbe[I]() + } + + class SubscriberProbe[I]()(implicit system: ActorSystem) extends Subscriber[I] { val probe = TestProbe() def expectSubscription(): Subscription = probe.expectMsgType[OnSubscribe].subscription @@ -136,14 +140,13 @@ object StreamTestKit { def onNext(element: I): Unit = probe.ref ! OnNext(element) def onComplete(): Unit = probe.ref ! OnComplete def onError(cause: Throwable): Unit = probe.ref ! OnError(cause) - - // Keeping equality - // FIXME: This and PublisherProbe should not be a case class so that we don't need this equality reversal - override def equals(that: Any): Boolean = this eq that.asInstanceOf[AnyRef] - override def hashCode(): Int = System.identityHashCode(this) } - case class PublisherProbe[I]()(implicit system: ActorSystem) extends Publisher[I] { + object PublisherProbe { + def apply[I]()(implicit system: ActorSystem): PublisherProbe[I] = new PublisherProbe[I]() + } + + class PublisherProbe[I]()(implicit system: ActorSystem) extends Publisher[I] { val probe: TestProbe = TestProbe() def subscribe(subscriber: Subscriber[_ >: I]): Unit = { 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 2d212fca78..bc4848a8a2 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/DslConsistencySpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/DslConsistencySpec.scala @@ -19,6 +19,12 @@ class DslConsistencySpec extends WordSpec with Matchers { val sSinkClass = classOf[akka.stream.scaladsl.Sink[_]] val jSinkClass = classOf[akka.stream.javadsl.Sink[_]] + val sKeyClass = classOf[akka.stream.scaladsl.Key] + val jKeyClass = classOf[akka.stream.javadsl.Key[_]] + + val sMaterializedMapClass = classOf[akka.stream.scaladsl.MaterializedMap] + val jMaterializedMapClass = classOf[akka.stream.javadsl.MaterializedMap] + val jFlowGraphClass = classOf[akka.stream.javadsl.FlowGraph] val sFlowGraphClass = classOf[akka.stream.scaladsl.FlowGraph] @@ -41,8 +47,8 @@ class DslConsistencySpec extends WordSpec with Matchers { jSinkClass -> Set(), sFlowGraphClass -> Set("builder"), - jFlowGraphClass → Set("graph"), - jPartialFlowGraphClass → Set("graph")) + jFlowGraphClass → Set("graph", "cyclesAllowed"), + jPartialFlowGraphClass → Set("graph", "cyclesAllowed", "disconnectedAllowed")) def materializing(m: Method): Boolean = m.getParameterTypes.contains(classOf[FlowMaterializer]) @@ -57,6 +63,8 @@ class DslConsistencySpec extends WordSpec with Matchers { ("Source" -> List(sSourceClass, jSourceClass)) :: ("Flow" -> List(sFlowClass, jFlowClass)) :: ("Sink" -> List(sSinkClass, jSinkClass)) :: + ("Key" -> List(sKeyClass, jKeyClass)) :: + ("MaterializedMap" -> List(sMaterializedMapClass, jMaterializedMapClass)) :: ("FlowGraph" -> List(sFlowGraphClass, jFlowGraphClass)) :: ("PartialFlowGraph" -> List(sPartialFlowGraphClass, jPartialFlowGraphClass)) :: Nil foreach { diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/SslTlsFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/SslTlsFlowSpec.scala deleted file mode 100644 index 10e579fec5..0000000000 --- a/akka-stream-tests/src/test/scala/akka/stream/io/SslTlsFlowSpec.scala +++ /dev/null @@ -1,222 +0,0 @@ -/** - * Copyright (C) 2009-2014 Typesafe Inc. - */ -package akka.stream.io - -import java.io.{ OutputStreamWriter, BufferedWriter, InputStreamReader, BufferedReader } -import java.net.InetSocketAddress -import akka.actor.{ ActorSystem, Props } -import akka.stream.io.SslTlsCipher.SessionNegotiation -import akka.stream.io.StreamTcp.{ OutgoingTcpConnection, TcpServerBinding } -import akka.stream.scaladsl.Flow -import akka.stream.testkit.AkkaSpec -import akka.testkit.TestProbe -import akka.util.ByteString -import java.security.{ KeyStore, SecureRandom } -import java.util.concurrent.atomic.AtomicInteger -import javax.net.ssl._ -import sun.rmi.transport.tcp.TCPConnection -import scala.concurrent.{ Future, Await } -import scala.concurrent.duration._ -import akka.stream.scaladsl.Source -import akka.stream.scaladsl.Sink - -object SslTlsFlowSpec { - - import TcpHelper._ - - def initSslContext(): SSLContext = { - - val password = "changeme" - - val keyStore = KeyStore.getInstance(KeyStore.getDefaultType) - keyStore.load(getClass.getResourceAsStream("/keystore"), password.toCharArray) - - val trustStore = KeyStore.getInstance(KeyStore.getDefaultType) - trustStore.load(getClass.getResourceAsStream("/truststore"), password.toCharArray) - - val keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm) - keyManagerFactory.init(keyStore, password.toCharArray) - - val trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm) - trustManagerFactory.init(trustStore) - - val context = SSLContext.getInstance("TLS") - context.init(keyManagerFactory.getKeyManagers, trustManagerFactory.getTrustManagers, new SecureRandom) - context - } - - private val clientServerId = new AtomicInteger(0) - - def createClientCipher(context: SSLContext)(implicit system: ActorSystem): SslTlsCipher = - createClientCipher(context, clientServerId.incrementAndGet()) - - def createClientCipher(context: SSLContext, id: Int)(implicit system: ActorSystem): SslTlsCipher = { - val cengine = context.createSSLEngine - cengine.setEnabledCipherSuites(Array("TLS_RSA_WITH_AES_128_CBC_SHA")) - cengine.setUseClientMode(true) - - val requester = TestProbe() - system.actorOf(Props(classOf[SslTlsCipherActor], requester.ref, SessionNegotiation(cengine)), s"ssl-client-$id") - requester.expectMsgType[SslTlsCipher] - } - - def createServerCipher(context: SSLContext)(implicit system: ActorSystem): SslTlsCipher = - createServerCipher(context, clientServerId.incrementAndGet()) - - def createServerCipher(context: SSLContext, id: Int)(implicit system: ActorSystem): SslTlsCipher = { - val sengine = context.createSSLEngine - sengine.setEnabledCipherSuites(Array("TLS_RSA_WITH_AES_128_CBC_SHA")) - sengine.setUseClientMode(false) - - val requester = TestProbe() - system.actorOf(Props(classOf[SslTlsCipherActor], requester.ref, SessionNegotiation(sengine)), s"ssl-server-$id") - requester.expectMsgType[SslTlsCipher] - } - - def createClientServerCipherPair(implicit system: ActorSystem): (SslTlsCipher, SslTlsCipher) = { - val context = initSslContext() - val id = clientServerId.incrementAndGet() - val ccipher = createClientCipher(context, id) - val scipher = createServerCipher(context, id) - - (ccipher, scipher) - } - - class JavaSslConnection(socket: SSLSocket) { - private val reader = new BufferedReader(new InputStreamReader(socket.getInputStream)) - private val writer = new BufferedWriter(new OutputStreamWriter(socket.getOutputStream)) - def readLn() = reader.readLine() - def writeLn(msg: String) = { - writer.write(msg + '\n') - writer.flush() - } - def close(): Unit = socket.close() - } - - def newJavaSslClientConnection(context: SSLContext, address: InetSocketAddress): JavaSslConnection = - new JavaSslConnection(context.getSocketFactory.createSocket(address.getHostName, address.getPort).asInstanceOf[SSLSocket]) - - class JavaSslServer(context: SSLContext) { - val address = temporaryServerAddress - private val serverSocket = context.getServerSocketFactory.createServerSocket(address.getPort).asInstanceOf[SSLServerSocket] - def acceptOne() = new JavaSslConnection(serverSocket.accept().asInstanceOf[SSLSocket]) - def close(): Unit = serverSocket.close() - } -} - -class SslTlsFlowSpec extends AkkaSpec("akka.actor.default-mailbox.mailbox-type = akka.dispatch.UnboundedMailbox") with TcpHelper { - import akka.stream.io.SslTlsFlowSpec._ - - val duration = 5.seconds - - def concurrently(block1: ⇒ Unit, block2: ⇒ String): String = { - import scala.concurrent.ExecutionContext.Implicits.global - Await.result(Future.sequence(List(Future(block1), Future(block2))), duration)(1).asInstanceOf[String] - } - - def replyFirstLineInUpperCase(scipher: SslTlsCipher): Unit = { - val ssessionf = Source(scipher.sessionInbound).runWith(Sink.head) - val ssession = Await.result(ssessionf, duration) - val sdata = ssession.data - Source(sdata).map(bs ⇒ ByteString(bs.decodeString("utf-8").split('\n').head.toUpperCase + '\n')). - runWith(Sink(scipher.plainTextOutbound)) - } - - def replyFirstLineInUpperCase(clientConnection: JavaSslConnection): Unit = { - clientConnection.writeLn(clientConnection.readLn().toUpperCase()) - } - - def sendLineAndReceiveResponse(ccipher: SslTlsCipher, message: String): String = { - val csessionf = Source(ccipher.sessionInbound).runWith(Sink.head) - Source(List(ByteString(message + '\n'))).runWith(Sink(ccipher.plainTextOutbound)) - val csession = Await.result(csessionf, duration) - val cdata = csession.data - Await.result(Source(cdata).map(_.decodeString("utf-8").split('\n').head).runWith(Sink.head), duration) - } - - def sendLineAndReceiveResponse(connection: JavaSslConnection, message: String): String = { - connection.writeLn(message) - val result = connection.readLn() - result - } - - def sendBackAndForthAndValidateReply(ccipher: SslTlsCipher, scipher: SslTlsCipher, message: String): Unit = { - val result = concurrently(replyFirstLineInUpperCase(scipher), sendLineAndReceiveResponse(ccipher, message)) - result should be(message.toUpperCase) - } - - def connectIncomingConnection(serverBinding: TcpServerBinding, scipher: SslTlsCipher): Unit = { - // connect the incoming tcp stream to the server cipher - Source(serverBinding.connectionStream).foreach { - case StreamTcp.IncomingTcpConnection(remoteAddress, inputStream, outputStream) ⇒ - scipher.cipherTextOutbound.subscribe(outputStream) - inputStream.subscribe(scipher.cipherTextInbound) - } - } - - def connectOutgoingConnection(clientConnection: OutgoingTcpConnection, ccipher: SslTlsCipher): Unit = { - // connect the outgoing tcp stream to the client cipher - ccipher.cipherTextOutbound.subscribe(clientConnection.outputStream) - clientConnection.inputStream.subscribe(ccipher.cipherTextInbound) - } - // Only here for SSL debug convenience - //System.setProperty("javax.net.debug", "all") - - "SslTls Cipher" must { - - "work on a simple stream" in { - val (ccipher, scipher) = createClientServerCipherPair(system) - - // connect two ciphers directly - ccipher.cipherTextOutbound.subscribe(scipher.cipherTextInbound) - scipher.cipherTextOutbound.subscribe(ccipher.cipherTextInbound) - - sendBackAndForthAndValidateReply(ccipher, scipher, "I'm the simple stream client!") - } - - "work on a simple stream over TCP" in { - val (ccipher, scipher) = createClientServerCipherPair(system) - - val serverBinding = bind() - val clientConnection = connect(serverBinding.localAddress) - - connectIncomingConnection(serverBinding, scipher) - - connectOutgoingConnection(clientConnection, ccipher) - - sendBackAndForthAndValidateReply(ccipher, scipher, "I'm the TCP stream client!") - } - - "work on a simple stream over TCP between java client and stream server" in { - val message = "I'm the TCP java client and stream server" - val context = initSslContext() - val scipher = createServerCipher(context) - val serverBinding = bind() - - connectIncomingConnection(serverBinding, scipher) - - val clientConnection = newJavaSslClientConnection(context, serverBinding.localAddress) - - val result = concurrently(replyFirstLineInUpperCase(scipher), sendLineAndReceiveResponse(clientConnection, message)) - clientConnection.close() - result should be(message.toUpperCase) - } - - "work on a simple stream over TCP between stream client and java server" in { - val message = "I'm the TCP stream client and java server" - val context = initSslContext() - val ccipher = createClientCipher(context) - - val server = new JavaSslServer(context) - val clientConnection = connect(server.address) - val serverConnection = server.acceptOne() - - connectOutgoingConnection(clientConnection, ccipher) - - val result = concurrently(replyFirstLineInUpperCase(serverConnection), sendLineAndReceiveResponse(ccipher, message)) - serverConnection.close() - result should be(message.toUpperCase) - } - } -} diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/TcpFlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TcpFlowSpec.scala index effe9503e4..d798e9ea41 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/TcpFlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/TcpFlowSpec.scala @@ -3,14 +3,15 @@ */ package akka.stream.io +import akka.stream.io.StreamTcp.{ TcpServerBinding, IncomingTcpConnection } import akka.stream.scaladsl.Flow import akka.stream.testkit.AkkaSpec import akka.util.ByteString import scala.collection.immutable import scala.concurrent.Await +import scala.concurrent.{ Future, Await } import scala.concurrent.duration._ -import akka.stream.scaladsl.Source -import akka.stream.scaladsl.Sink +import akka.stream.scaladsl._ class TcpFlowSpec extends AkkaSpec with TcpHelper { import akka.stream.io.TcpHelper._ @@ -23,50 +24,47 @@ class TcpFlowSpec extends AkkaSpec with TcpHelper { val server = new Server() - val (tcpProcessor, serverConnection) = connect(server) + val tcpReadProbe = new TcpReadProbe() + val tcpWriteProbe = new TcpWriteProbe() + Source(tcpWriteProbe.publisherProbe).via(StreamTcp(system).connect(server.address).flow).to(Sink(tcpReadProbe.subscriberProbe)).run() + val serverConnection = server.waitAccept() - val tcpReadProbe = new TcpReadProbe(tcpProcessor) - val tcpWriteProbe = new TcpWriteProbe(tcpProcessor) - - serverConnection.write(testData) - serverConnection.read(5) - tcpReadProbe.read(5) should be(testData) - tcpWriteProbe.write(testData) - serverConnection.waitRead() should be(testData) + validateServerClientCommunication(testData, serverConnection, tcpReadProbe, tcpWriteProbe) tcpWriteProbe.close() tcpReadProbe.close() - //client.read() should be(ByteString.empty) server.close() } "be able to write a sequence of ByteStrings" in { val server = new Server() - val (tcpProcessor, serverConnection) = connect(server) - val testInput = (0 to 255).map(ByteString(_)) val expectedOutput = ByteString(Array.tabulate(256)(_.asInstanceOf[Byte])) + Source(testInput).via(StreamTcp(system).connect(server.address).flow).to(Sink.ignore).run() + + val serverConnection = server.waitAccept() serverConnection.read(256) - Source(tcpProcessor).runWith(Sink.ignore) - - Source(testInput).runWith(Sink.publisher).subscribe(tcpProcessor) serverConnection.waitRead() should be(expectedOutput) - } "be able to read a sequence of ByteStrings" in { val server = new Server() - val (tcpProcessor, serverConnection) = connect(server) - val testInput = (0 to 255).map(ByteString(_)) val expectedOutput = ByteString(Array.tabulate(256)(_.asInstanceOf[Byte])) - for (in ← testInput) serverConnection.write(in) - new TcpWriteProbe(tcpProcessor) // Just register an idle upstream + val idle = new TcpWriteProbe() // Just register an idle upstream + val resultFuture = + Source(idle.publisherProbe) + .via(StreamTcp(system).connect(server.address).flow) + .fold(ByteString.empty)((acc, in) ⇒ acc ++ in) + val serverConnection = server.waitAccept() + + for (in ← testInput) { + serverConnection.write(in) + } - val resultFuture = Source(tcpProcessor).fold(ByteString.empty)((acc, in) ⇒ acc ++ in) serverConnection.confirmedClose() Await.result(resultFuture, 3.seconds) should be(expectedOutput) @@ -75,10 +73,11 @@ class TcpFlowSpec extends AkkaSpec with TcpHelper { "half close the connection when output stream is closed" in { val testData = ByteString(1, 2, 3, 4, 5) val server = new Server() - val (tcpProcessor, serverConnection) = connect(server) - val tcpWriteProbe = new TcpWriteProbe(tcpProcessor) - val tcpReadProbe = new TcpReadProbe(tcpProcessor) + val tcpWriteProbe = new TcpWriteProbe() + val tcpReadProbe = new TcpReadProbe() + Source(tcpWriteProbe.publisherProbe).via(StreamTcp(system).connect(server.address).flow).to(Sink(tcpReadProbe.subscriberProbe)).run() + val serverConnection = server.waitAccept() tcpWriteProbe.close() // FIXME: expect PeerClosed on server @@ -89,12 +88,13 @@ class TcpFlowSpec extends AkkaSpec with TcpHelper { } "stop reading when the input stream is cancelled" in { - val testData = ByteString(1, 2, 3, 4, 5) val server = new Server() - val (tcpProcessor, serverConnection) = connect(server) + val testData = ByteString(1, 2, 3, 4, 5) - val tcpWriteProbe = new TcpWriteProbe(tcpProcessor) - val tcpReadProbe = new TcpReadProbe(tcpProcessor) + val tcpWriteProbe = new TcpWriteProbe() + val tcpReadProbe = new TcpReadProbe() + Source(tcpWriteProbe.publisherProbe).via(StreamTcp(system).connect(server.address).flow).to(Sink(tcpReadProbe.subscriberProbe)).run() + val serverConnection = server.waitAccept() tcpReadProbe.close() // FIXME: expect PeerClosed on server @@ -107,18 +107,20 @@ class TcpFlowSpec extends AkkaSpec with TcpHelper { } "keep write side open when remote half-closes" in { - val testData = ByteString(1, 2, 3, 4, 5) val server = new Server() - val (tcpProcessor, serverConnection) = connect(server) + val testData = ByteString(1, 2, 3, 4, 5) - val tcpWriteProbe = new TcpWriteProbe(tcpProcessor) - val tcpReadProbe = new TcpReadProbe(tcpProcessor) + val tcpWriteProbe = new TcpWriteProbe() + val tcpReadProbe = new TcpReadProbe() + + Source(tcpWriteProbe.publisherProbe).via(StreamTcp(system).connect(server.address).flow).to(Sink(tcpReadProbe.subscriberProbe)).run() + val serverConnection = server.waitAccept() // FIXME: here (and above tests) add a chitChat() method ensuring this works even after prior communication // there should be a chitchat and non-chitchat version serverConnection.confirmedClose() - tcpReadProbe.subscriberProbe.expectComplete() + tcpReadProbe.subscriberProbe.expectCompletedOrSubscriptionFollowedByComplete() serverConnection.read(5) tcpWriteProbe.write(testData) @@ -132,13 +134,15 @@ class TcpFlowSpec extends AkkaSpec with TcpHelper { // Client gets a PeerClosed event and does not know that the write side is also closed val testData = ByteString(1, 2, 3, 4, 5) val server = new Server() - val (tcpProcessor, serverConnection) = connect(server) - val tcpWriteProbe = new TcpWriteProbe(tcpProcessor) - val tcpReadProbe = new TcpReadProbe(tcpProcessor) + val tcpWriteProbe = new TcpWriteProbe() + val tcpReadProbe = new TcpReadProbe() + + Source(tcpWriteProbe.publisherProbe).via(StreamTcp(system).connect(server.address).flow).to(Sink(tcpReadProbe.subscriberProbe)).run() + val serverConnection = server.waitAccept() serverConnection.abort() - tcpReadProbe.subscriberProbe.expectError() + tcpReadProbe.subscriberProbe.expectErrorOrSubscriptionFollowedByError() tcpWriteProbe.tcpWriteSubscription.expectCancellation() } @@ -146,49 +150,111 @@ class TcpFlowSpec extends AkkaSpec with TcpHelper { pending } + "materialize correctly when used in multiple flows" in { + val testData = ByteString(1, 2, 3, 4, 5) + + val server = new Server() + + val tcpReadProbe1 = new TcpReadProbe() + val tcpWriteProbe1 = new TcpWriteProbe() + val tcpReadProbe2 = new TcpReadProbe() + val tcpWriteProbe2 = new TcpWriteProbe() + val outgoingFlow = StreamTcp(system).connect(server.address) + + val mm1 = Source(tcpWriteProbe1.publisherProbe).via(outgoingFlow.flow).to(Sink(tcpReadProbe1.subscriberProbe)).run() + val serverConnection1 = server.waitAccept() + val mm2 = Source(tcpWriteProbe2.publisherProbe).via(outgoingFlow.flow).to(Sink(tcpReadProbe2.subscriberProbe)).run() + val serverConnection2 = server.waitAccept() + + validateServerClientCommunication(testData, serverConnection1, tcpReadProbe1, tcpWriteProbe1) + validateServerClientCommunication(testData, serverConnection2, tcpReadProbe2, tcpWriteProbe2) + // Since we have already communicated over the connections we can have short timeouts for the futures + val outgoingConnection1 = Await.result(mm1.get(outgoingFlow.key), 100 millis) + val outgoingConnection2 = Await.result(mm2.get(outgoingFlow.key), 100 millis) + + outgoingConnection1.remoteAddress.getPort should be(server.address.getPort) + outgoingConnection2.remoteAddress.getPort should be(server.address.getPort) + outgoingConnection1.localAddress.getPort should not be (outgoingConnection2.localAddress.getPort) + + tcpWriteProbe1.close() + tcpReadProbe1.close() + + server.close() + } + } "TCP listen stream" must { + // Reusing handler + val echoHandler = ForeachSink[IncomingTcpConnection] { incoming ⇒ + incoming.stream.join(Flow.empty).run() + } + "be able to implement echo" in { + import system.dispatcher val serverAddress = temporaryServerAddress - val server = echoServer(serverAddress) - val conn = connect(serverAddress) + val binding = StreamTcp(system).bind(serverAddress) + val echoServer = binding.to(echoHandler).run() + + val echoServerFinish = echoServer.get(echoHandler) + val echoServerBinding = echoServer.get(binding) + + // make sure that the server has bound to the socket + Await.result(echoServerBinding, 3.seconds) val testInput = (0 to 255).map(ByteString(_)) val expectedOutput = ByteString(Array.tabulate(256)(_.asInstanceOf[Byte])) - - Source(testInput).runWith(Sink(conn.outputStream)) - val resultFuture = Source(conn.inputStream).fold(ByteString.empty)((acc, in) ⇒ acc ++ in) + val resultFuture = + Source(testInput).via(StreamTcp(system).connect(serverAddress).flow).fold(ByteString.empty)((acc, in) ⇒ acc ++ in) Await.result(resultFuture, 3.seconds) should be(expectedOutput) - server.close() - server.awaitTermination(3.seconds) + echoServerBinding.foreach(_.close) + Await.result(echoServerFinish, 3.seconds) } "work with a chain of echoes" in { + import system.dispatcher val serverAddress = temporaryServerAddress - val server = echoServer(serverAddress) + val binding = StreamTcp(system).bind(serverAddress) + val echoServer = binding.to(echoHandler).run() - val conn1 = connect(serverAddress) - val conn2 = connect(serverAddress) - val conn3 = connect(serverAddress) + val echoServerFinish = echoServer.get(echoHandler) + val echoServerBinding = echoServer.get(binding) + + // make sure that the server has bound to the socket + Await.result(echoServerBinding, 3.seconds) + + val echoConnection = StreamTcp(system).connect(serverAddress).flow val testInput = (0 to 255).map(ByteString(_)) val expectedOutput = ByteString(Array.tabulate(256)(_.asInstanceOf[Byte])) - Source(testInput).runWith(Sink(conn1.outputStream)) - conn1.inputStream.subscribe(conn2.outputStream) - conn2.inputStream.subscribe(conn3.outputStream) - val resultFuture = Source(conn3.inputStream).fold(ByteString.empty)((acc, in) ⇒ acc ++ in) + val resultFuture = + Source(testInput) + .via(echoConnection) // The echoConnection is reusable + .via(echoConnection) + .via(echoConnection) + .via(echoConnection) + .fold(ByteString.empty)((acc, in) ⇒ acc ++ in) Await.result(resultFuture, 3.seconds) should be(expectedOutput) - server.close() - server.awaitTermination(3.seconds) + echoServerBinding.foreach(_.close) + Await.result(echoServerFinish, 3.seconds) } } + def validateServerClientCommunication(testData: ByteString, + serverConnection: ServerConnection, + readProbe: TcpReadProbe, + writeProbe: TcpWriteProbe): Unit = { + serverConnection.write(testData) + serverConnection.read(5) + readProbe.read(5) should be(testData) + writeProbe.write(testData) + serverConnection.waitRead() should be(testData) + } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala b/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala index 1b3e7614c4..e8856efc42 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/io/TcpHelper.scala @@ -143,10 +143,9 @@ trait TcpHelper { this: TestKitBase ⇒ def abort(): Unit = connectionActor ! ClientClose(Tcp.Abort) } - class TcpReadProbe(tcpProcessor: Processor[ByteString, ByteString]) { + class TcpReadProbe() { val subscriberProbe = StreamTestKit.SubscriberProbe[ByteString]() - tcpProcessor.subscribe(subscriberProbe) - val tcpReadSubscription = subscriberProbe.expectSubscription() + lazy val tcpReadSubscription = subscriberProbe.expectSubscription() def read(count: Int): ByteString = { var result = ByteString.empty @@ -160,10 +159,9 @@ trait TcpHelper { this: TestKitBase ⇒ def close(): Unit = tcpReadSubscription.cancel() } - class TcpWriteProbe(tcpProcessor: Processor[ByteString, ByteString]) { + class TcpWriteProbe() { val publisherProbe = StreamTestKit.PublisherProbe[ByteString]() - publisherProbe.subscribe(tcpProcessor) - val tcpWriteSubscription = publisherProbe.expectSubscription() + lazy val tcpWriteSubscription = publisherProbe.expectSubscription() var demand = 0L def write(bytes: ByteString): Unit = { @@ -175,37 +173,4 @@ trait TcpHelper { this: TestKitBase ⇒ def close(): Unit = tcpWriteSubscription.sendComplete() } - class EchoServer(termination: Future[Unit], closeable: Closeable) extends Closeable { - def close(): Unit = closeable.close() - def awaitTermination(atMost: Duration): Unit = Await.result(termination, atMost) - def terminationFuture: Future[Unit] = termination - } - - def connect(server: Server): (Processor[ByteString, ByteString], ServerConnection) = { - val tcpProbe = TestProbe() - tcpProbe.send(IO(StreamTcp), StreamTcp.Connect(server.address)) - val client = server.waitAccept() - val outgoingConnection = tcpProbe.expectMsgType[StreamTcp.OutgoingTcpConnection] - - (outgoingConnection.processor, client) - } - - def connect(serverAddress: InetSocketAddress): StreamTcp.OutgoingTcpConnection = { - val connectProbe = TestProbe() - connectProbe.send(IO(StreamTcp), StreamTcp.Connect(serverAddress)) - connectProbe.expectMsgType[StreamTcp.OutgoingTcpConnection] - } - - def bind(serverAddress: InetSocketAddress = temporaryServerAddress): StreamTcp.TcpServerBinding = { - val bindProbe = TestProbe() - bindProbe.send(IO(StreamTcp), StreamTcp.Bind(serverAddress)) - bindProbe.expectMsgType[StreamTcp.TcpServerBinding] - } - - def echoServer(serverAddress: InetSocketAddress = temporaryServerAddress): EchoServer = { - val binding = bind(serverAddress) - new EchoServer(Source(binding.connectionStream).foreach { conn ⇒ - conn.inputStream.subscribe(conn.outputStream) - }, binding) - } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala new file mode 100644 index 0000000000..da42647053 --- /dev/null +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowJoinSpec.scala @@ -0,0 +1,48 @@ +/** + * Copyright (C) 2014 Typesafe Inc. + */ +package akka.stream.scaladsl + +import akka.stream.{ FlowMaterializer, MaterializerSettings } +import akka.stream.testkit.{ StreamTestKit, AkkaSpec } +import com.typesafe.config.ConfigFactory +import scala.concurrent.Await +import scala.concurrent.duration._ + +class FlowJoinSpec extends AkkaSpec(ConfigFactory.parseString("akka.loglevel=INFO")) { + + val settings = MaterializerSettings(system) + .withInputBuffer(initialSize = 2, maxSize = 16) + .withFanOutBuffer(initialSize = 1, maxSize = 16) + + implicit val mat = FlowMaterializer(settings) + + "A Flow using join" must { + "allow for cycles" in { + val end = 47 + val (even, odd) = (0 to end).partition(_ % 2 == 0) + val size = even.size + 2 * odd.size + val result = Set() ++ even ++ odd ++ odd.map(_ * 10) + val source = Source(0 to end) + val in = UndefinedSource[Int] + val out = UndefinedSink[Int] + val probe = StreamTestKit.SubscriberProbe[Int]() + val sink = Sink.head[Seq[Int]] + + val flow1 = Flow() { implicit b ⇒ + import FlowGraphImplicits._ + val merge = Merge[Int] + val broadcast = Broadcast[Int] + source ~> merge ~> broadcast ~> Flow[Int].grouped(1000) ~> sink + in ~> merge + broadcast ~> out + in -> out + } + + val flow2 = Flow[Int].filter(_ % 2 == 1).map(_ * 10).take((end + 1) / 2) + + val mm = flow1.join(flow2).run() + Await.result(mm get sink, 1.second).toSet should be(result) + } + } +} diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala index dfcb133af0..09a9ab60e4 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/FlowSpec.scala @@ -9,7 +9,7 @@ import akka.stream.stage.Stage import scala.collection.immutable import scala.concurrent.duration._ import akka.actor._ -import akka.stream.{ TransformerLike, MaterializerSettings } +import akka.stream.MaterializerSettings import akka.stream.FlowMaterializer import akka.stream.impl._ import akka.stream.impl.Ast._ @@ -57,7 +57,7 @@ object FlowSpec { optimizations: Optimizations, brokenMessage: Any) extends ActorBasedFlowMaterializer(settings, dispatchers, supervisor, flowNameCounter, namePrefix, optimizations) { - override def processorForNode[In, Out](op: AstNode, flowName: String, n: Int): Processor[In, Out] = { + override def processorForNode[In, Out](op: AstNode, flowName: String, n: Int): (Processor[In, Out], MaterializedMap) = { val props = op match { case f: Fused ⇒ Props(new BrokenActorInterpreter(settings, f.ops, brokenMessage)).withDispatcher(settings.dispatcher) case Map(f) ⇒ Props(new BrokenActorInterpreter(settings, List(fusing.Map(f)), brokenMessage)) @@ -73,7 +73,7 @@ object FlowSpec { case o ⇒ ActorProcessorFactory.props(this, o) } val impl = actorOf(props, s"$flowName-$n-${op.name}") - ActorProcessorFactory(impl) + (ActorProcessorFactory(impl), MaterializedMap.empty) } } diff --git a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala index 6fd38b64bd..99cde03dd3 100644 --- a/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala +++ b/akka-stream-tests/src/test/scala/akka/stream/scaladsl/SourceSpec.scala @@ -70,4 +70,58 @@ class SourceSpec extends AkkaSpec { c2.expectError(ex) } } + + "Source with additional keys" must { + "materialize keys properly" in { + val ks = Source.subscriber[Int] + val mk1 = new Key { + override type MaterializedType = String + override def materialize(map: MaterializedMap) = map.get(ks).toString + } + val mk2 = new Key { + override type MaterializedType = String + override def materialize(map: MaterializedMap) = map.get(mk1).toUpperCase + } + val sp = StreamTestKit.SubscriberProbe[Int]() + val mm = ks.withKey(mk1).withKey(mk2).to(Sink(sp)).run() + val s = mm.get(ks) + mm.get(mk1) should be(s.toString) + mm.get(mk2) should be(s.toString.toUpperCase) + val p = Source.singleton(1).runWith(Sink.publisher) + p.subscribe(s) + val sub = sp.expectSubscription() + sub.request(1) + sp.expectNext(1) + sp.expectComplete() + } + + "materialize keys properly when used in a graph" in { + val ks = Source.subscriber[Int] + val mk1 = new Key { + override type MaterializedType = String + override def materialize(map: MaterializedMap) = map.get(ks).toString + } + val mk2 = new Key { + override type MaterializedType = String + override def materialize(map: MaterializedMap) = map.get(mk1).toUpperCase + } + val sp = StreamTestKit.SubscriberProbe[Int]() + val mks = ks.withKey(mk1).withKey(mk2) + val mm = FlowGraph { implicit b ⇒ + import FlowGraphImplicits._ + val bcast = Broadcast[Int] + mks ~> bcast ~> Sink(sp) + bcast ~> Sink.ignore + }.run() + val s = mm.get(ks) + mm.get(mk1) should be(s.toString) + mm.get(mk2) should be(s.toString.toUpperCase) + val p = Source.singleton(1).runWith(Sink.publisher) + p.subscribe(s) + val sub = sp.expectSubscription() + sub.request(1) + sp.expectNext(1) + sp.expectComplete() + } + } } \ No newline at end of file diff --git a/akka-stream/src/main/scala/akka/stream/FlowMaterializer.scala b/akka-stream/src/main/scala/akka/stream/FlowMaterializer.scala index ceb05771ca..5a4ead82b3 100644 --- a/akka-stream/src/main/scala/akka/stream/FlowMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/FlowMaterializer.scala @@ -7,6 +7,7 @@ import java.util.Locale import java.util.concurrent.TimeUnit import akka.stream.impl._ +import akka.stream.scaladsl.Key import scala.collection.immutable @@ -151,7 +152,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](source: scaladsl.Source[In], sink: scaladsl.Sink[Out], ops: List[Ast.AstNode]): scaladsl.MaterializedMap + def materialize[In, Out](source: scaladsl.Source[In], sink: scaladsl.Sink[Out], ops: List[Ast.AstNode], keys: List[Key]): scaladsl.MaterializedMap /** * Create publishers and subscribers for fan-in and fan-out operations. diff --git a/akka-stream/src/main/scala/akka/stream/impl/ActorBasedFlowMaterializer.scala b/akka-stream/src/main/scala/akka/stream/impl/ActorBasedFlowMaterializer.scala index 2a239c39d7..7c33fe8e1b 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/ActorBasedFlowMaterializer.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/ActorBasedFlowMaterializer.scala @@ -10,7 +10,7 @@ import akka.event.Logging import akka.stream.impl.fusing.ActorInterpreter import scala.annotation.tailrec import scala.collection.immutable -import scala.concurrent.{ ExecutionContext, Await, Future } +import scala.concurrent.{ Promise, ExecutionContext, Await, Future } import akka.actor._ import akka.stream.{ FlowMaterializer, MaterializerSettings, OverflowStrategy, TimerTransformer } import akka.stream.MaterializationException @@ -92,6 +92,13 @@ private[akka] object Ast { override def name = "concatFlatten" } + case class DirectProcessor(p: () ⇒ Processor[Any, Any]) extends AstNode { + override def name = "processor" + } + + case class DirectProcessorWithKey(p: () ⇒ (Processor[Any, Any], Any), key: Key) extends AstNode { + override def name = "processorWithKey" + } sealed trait JunctionAstNode { def name: String } @@ -101,8 +108,8 @@ private[akka] object Ast { sealed trait FanOutAstNode extends JunctionAstNode // FIXME Why do we need this? - case object IdentityAstNode extends JunctionAstNode { - override def name = "identity" + case class IdentityAstNode(id: Int) extends JunctionAstNode { + override val name = s"identity$id" } case object Merge extends FanInAstNode { @@ -179,19 +186,20 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting @tailrec private[this] def processorChain(topProcessor: Processor[_, _], ops: List[AstNode], flowName: String, - n: Int): Processor[_, _] = + n: Int, + materializedMap: MaterializedMap): (Processor[_, _], MaterializedMap) = ops match { case op :: tail ⇒ - val opProcessor = processorForNode[Any, Any](op, flowName, n) + val (opProcessor, opMap) = processorForNode[Any, Any](op, flowName, n) opProcessor.subscribe(topProcessor.asInstanceOf[Subscriber[Any]]) - processorChain(opProcessor, tail, flowName, n - 1) + processorChain(opProcessor, tail, flowName, n - 1, materializedMap.merge(opMap)) case Nil ⇒ - topProcessor + (topProcessor, materializedMap) } //FIXME Optimize the implementation of the optimizer (no joke) // AstNodes are in reverse order, Fusable Ops are in order - private[this] final def optimize(ops: List[Ast.AstNode]): (List[Ast.AstNode], Int) = { + private[this] final def optimize(ops: List[Ast.AstNode], mmFuture: Future[MaterializedMap]): (List[Ast.AstNode], Int) = { @tailrec def analyze(rest: List[Ast.AstNode], optimized: List[Ast.AstNode], fuseCandidates: List[Stage[_, _]]): (List[Ast.AstNode], Int) = { //The `verify` phase @@ -283,13 +291,11 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting } } val result = analyze(ops, Nil, Nil) - //println(s"before: $ops") - //println(s"after: ${result._1}") result } // Ops come in reverse order - override def materialize[In, Out](source: Source[In], sink: Sink[Out], rawOps: List[Ast.AstNode]): MaterializedMap = { + override def materialize[In, Out](source: Source[In], sink: Sink[Out], rawOps: List[Ast.AstNode], keys: List[Key]): MaterializedMap = { val flowName = createFlowName() //FIXME: Creates Id even when it is not used in all branches below def throwUnknownType(typeName: String, s: AnyRef): Nothing = @@ -318,25 +324,34 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting case s: Sink[_] ⇒ throwUnknownType("Sink", s) } - val (sourceValue, sinkValue) = + val mmPromise = Promise[MaterializedMap] + val mmFuture = mmPromise.future + + val (sourceValue, sinkValue, pipeMap) = if (rawOps.isEmpty) { if (isActive(sink)) { val (sub, value) = createSink(flowName) - (attachSource(sub, flowName), value) + (attachSource(sub, flowName), value, MaterializedMap.empty) } else if (isActive(source)) { val (pub, value) = createSource(flowName) - (value, attachSink(pub, flowName)) + (value, attachSink(pub, flowName), MaterializedMap.empty) } else { - val id = processorForNode[In, Out](identityStageNode, flowName, 1) - (attachSource(id, flowName), attachSink(id, flowName)) + val (id, empty) = processorForNode[In, Out](identityStageNode, flowName, 1) + (attachSource(id, flowName), attachSink(id, flowName), empty) } } else { - val (ops, opsSize) = if (optimizations.isEnabled) optimize(rawOps) else (rawOps, rawOps.length) - val last = processorForNode[Any, Out](ops.head, flowName, opsSize) - val first = processorChain(last, ops.tail, flowName, opsSize - 1).asInstanceOf[Processor[In, Any]] - (attachSource(first, flowName), attachSink(last, flowName)) + val (ops, opsSize) = if (optimizations.isEnabled) optimize(rawOps, mmFuture) else (rawOps, rawOps.length) + val (last, lastMap) = processorForNode[Any, Out](ops.head, flowName, opsSize) + val (first, map) = processorChain(last, ops.tail, flowName, opsSize - 1, lastMap) + (attachSource(first.asInstanceOf[Processor[In, Any]], flowName), attachSink(last, flowName), map) } - new MaterializedPipe(source, sourceValue, sink, sinkValue) + val sourceMap = if (source.isInstanceOf[KeyedSource[_]]) pipeMap.updated(source, sourceValue) else pipeMap + val sourceSinkMap = if (sink.isInstanceOf[KeyedSink[_]]) sourceMap.updated(sink, sinkValue) else sourceMap + + if (keys.isEmpty) sourceSinkMap + else (sourceSinkMap /: keys) { + case (mm, k) ⇒ mm.updated(k, k.materialize(mm)) + } } //FIXME Should this be a dedicated AstNode? private[this] val identityStageNode = Ast.StageFactory(() ⇒ FlowOps.identityStage[Any], "identity") @@ -349,8 +364,15 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting /** * INTERNAL API */ - private[akka] def processorForNode[In, Out](op: AstNode, flowName: String, n: Int): Processor[In, Out] = - ActorProcessorFactory[In, Out](actorOf(ActorProcessorFactory.props(this, op), s"$flowName-$n-${op.name}")) + private[akka] def processorForNode[In, Out](op: AstNode, flowName: String, n: Int): (Processor[In, Out], MaterializedMap) = op match { + // FIXME #16376 should probably be replaced with an ActorFlowProcessor similar to ActorFlowSource/Sink + case Ast.DirectProcessor(p) ⇒ (p().asInstanceOf[Processor[In, Out]], MaterializedMap.empty) + case Ast.DirectProcessorWithKey(p, key) ⇒ + val (processor, value) = p() + (processor.asInstanceOf[Processor[In, Out]], MaterializedMap.empty.updated(key, value)) + case _ ⇒ + (ActorProcessorFactory[In, Out](actorOf(ActorProcessorFactory.props(this, op), s"$flowName-$n-${op.name}")), MaterializedMap.empty) + } def actorOf(props: Props, name: String): ActorRef = supervisor match { case ref: LocalActorRef ⇒ @@ -400,8 +422,9 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting val subscriber = ActorSubscriber[In](impl) (List(subscriber), publishers) - case identity @ Ast.IdentityAstNode ⇒ // FIXME Why is IdentityAstNode a JunctionAStNode? - val id = List(processorForNode[In, Out](identityStageNode, identity.name, 1)) // FIXME is `identity.name` appropriate/unique here? + case identity @ Ast.IdentityAstNode(_) ⇒ // FIXME Why is IdentityAstNode a JunctionAStNode? + // We can safely ignore the materialized map that gets created here since it will be empty + val id = List(processorForNode[In, Out](identityStageNode, identity.name, 1)._1) // FIXME is `identity.name` appropriate/unique here? (id, id) } diff --git a/akka-stream/src/main/scala/akka/stream/impl/DirectedGraphBuilder.scala b/akka-stream/src/main/scala/akka/stream/impl/DirectedGraphBuilder.scala index 0e03e44d57..0a9c885224 100644 --- a/akka-stream/src/main/scala/akka/stream/impl/DirectedGraphBuilder.scala +++ b/akka-stream/src/main/scala/akka/stream/impl/DirectedGraphBuilder.scala @@ -24,6 +24,7 @@ private[akka] final case class Vertex[E, V](label: V) { def isolated: Boolean = inDegree == 0 && outDegree == 0 + // FIXME #16381 this is at the wrong level def isSink: Boolean = outEdgeSet.isEmpty def successors: Set[Vertex[E, V]] = outEdgeSet.map(_.to) diff --git a/akka-stream/src/main/scala/akka/stream/io/StreamIO.scala b/akka-stream/src/main/scala/akka/stream/io/StreamIO.scala deleted file mode 100644 index 50faa6705b..0000000000 --- a/akka-stream/src/main/scala/akka/stream/io/StreamIO.scala +++ /dev/null @@ -1,260 +0,0 @@ -/** - * Copyright (C) 2009-2014 Typesafe Inc. - */ -package akka.stream.io - -import java.io.Closeable - -import akka.util.ByteString -import org.reactivestreams.{ Processor, Publisher, Subscriber } -import java.net.InetSocketAddress -import akka.actor._ -import scala.collection._ -import scala.concurrent.duration.{ Duration, FiniteDuration } -import akka.io.Inet.SocketOption -import akka.io.Tcp -import akka.stream.impl.{ ActorPublisher, ExposedPublisher, ActorProcessor } -import akka.stream.MaterializerSettings -import akka.io.IO -import java.net.URLEncoder -import akka.japi.Util - -object StreamTcp extends ExtensionId[StreamTcpExt] with ExtensionIdProvider { - - override def lookup = StreamTcp - override def createExtension(system: ExtendedActorSystem): StreamTcpExt = new StreamTcpExt(system) - override def get(system: ActorSystem): StreamTcpExt = super.get(system) - - case class OutgoingTcpConnection(remoteAddress: InetSocketAddress, - localAddress: InetSocketAddress, - processor: Processor[ByteString, ByteString]) { - def outputStream: Subscriber[ByteString] = processor - def inputStream: Publisher[ByteString] = processor - } - - sealed abstract case class TcpServerBinding(localAddress: InetSocketAddress, - connectionStream: Publisher[IncomingTcpConnection]) extends Closeable - - object TcpServerBinding { - def apply(localAddress: InetSocketAddress, connectionStream: Publisher[IncomingTcpConnection]): TcpServerBinding = - new TcpServerBinding(localAddress, connectionStream) { - override def close() = () - } - - def apply(localAddress: InetSocketAddress, connectionStream: Publisher[IncomingTcpConnection], closeable: Closeable): TcpServerBinding = - new TcpServerBinding(localAddress, connectionStream) { - override def close() = closeable.close() - } - } - - case class IncomingTcpConnection(remoteAddress: InetSocketAddress, - inputStream: Publisher[ByteString], - outputStream: Subscriber[ByteString]) { - def handleWith(processor: Processor[ByteString, ByteString]): Unit = { - processor.subscribe(outputStream) - inputStream.subscribe(processor) - } - } - - /** - * The Connect message is sent to the StreamTcp manager actor, which is obtained via - * `IO(StreamTcp)`. The manager replies with a [[StreamTcp.OutgoingTcpConnection]] - * message. - * - * @param remoteAddress the address to connect to - * @param settings if Some the passed [[MaterializerSettings]] will be used during stream actor creation, - * otherwise the ActorSystem's default settings will be used - * @param localAddress optionally specifies a specific address to bind to - * @param options Please refer to [[akka.io.TcpSO]] for a list of all supported options. - * @param connectTimeout the desired timeout for connection establishment, infinite means "no timeout" - * @param idleTimeout the desired idle timeout on the connection, infinite means "no timeout" - */ - case class Connect(remoteAddress: InetSocketAddress, - localAddress: Option[InetSocketAddress] = None, - materializerSettings: Option[MaterializerSettings] = None, - options: immutable.Traversable[SocketOption] = Nil, - connectTimeout: Duration = Duration.Inf, - idleTimeout: Duration = Duration.Inf) { - /** - * Java API - */ - def withMaterializerSettings(materializerSettings: MaterializerSettings): Connect = - copy(materializerSettings = Option(materializerSettings)) - - /** - * Java API - */ - def withLocalAddress(localAddress: InetSocketAddress): Connect = - copy(localAddress = Option(localAddress)) - - /** - * Java API - */ - def withSocketOptions(options: java.lang.Iterable[SocketOption]): Connect = - copy(options = Util.immutableSeq(options)) - - /** - * Java API - */ - def withConnectTimeout(connectTimeout: Duration): Connect = - copy(connectTimeout = connectTimeout) - - /** - * Java API - */ - def withIdleTimeout(idleTimeout: Duration): Connect = - copy(idleTimeout = idleTimeout) - } - - /** - * The Bind message is send to the StreamTcp manager actor, which is obtained via - * `IO(StreamTcp)`, in order to bind to a listening socket. The manager - * replies with a [[StreamTcp.TcpServerBinding]]. If the local port is set to 0 in - * the Bind message, then the [[StreamTcp.TcpServerBinding]] message should be inspected to find - * the actual port which was bound to. - * - * @param settings if Some, these materializer settings will be used for stream actors, - * else the ActorSystem's default materializer settings will be used. - * @param localAddress the socket address to bind to; use port zero for automatic assignment (i.e. an ephemeral port) - * @param backlog the number of unaccepted connections the O/S - * kernel will hold for this port before refusing connections. - * @param options Please refer to [[akka.io.TcpSO]] for a list of all supported options. - * @param idleTimeout the desired idle timeout on the accepted connections, infinite means "no timeout" - */ - case class Bind(localAddress: InetSocketAddress, - settings: Option[MaterializerSettings] = None, - backlog: Int = 100, - options: immutable.Traversable[SocketOption] = Nil, - idleTimeout: Duration = Duration.Inf) { - - /** - * Java API - */ - def withBacklog(backlog: Int): Bind = copy(backlog = backlog) - - /** - * Java API - */ - def withSocketOptions(options: java.lang.Iterable[SocketOption]): Bind = - copy(options = Util.immutableSeq(options)) - - /** - * Java API - */ - def withIdleTimeout(idleTimeout: Duration): Bind = - copy(idleTimeout = idleTimeout) - } - -} - -/** - * Java API: Factory methods for the messages of `StreamTcp`. - */ -object StreamTcpMessage { - /** - * Java API: The Connect message is sent to the StreamTcp manager actor, which is obtained via - * `StreamTcp.get(system).manager()`. The manager replies with a [[StreamTcp.OutgoingTcpConnection]] - * message. - * - * @param remoteAddress is the address to connect to - * @param localAddress optionally specifies a specific address to bind to - * @param options Please refer to [[akka.io.TcpSO]] for a list of all supported options. - * @param connectTimeout the desired timeout for connection establishment, infinite means "no timeout" - * @param idleTimeout the desired idle timeout on the connection, infinite means "no timeout" - */ - def connect( - settings: MaterializerSettings, - remoteAddress: InetSocketAddress, - localAddress: InetSocketAddress, - options: java.lang.Iterable[SocketOption], - connectTimeout: Duration, - idleTimeout: Duration): StreamTcp.Connect = - StreamTcp.Connect(remoteAddress, Option(localAddress), Option(settings), Util.immutableSeq(options), connectTimeout, idleTimeout) - - /** - * Java API: Message to Connect to the given `remoteAddress` without binding to a local address and without - * specifying options. - */ - def connect(settings: MaterializerSettings, remoteAddress: InetSocketAddress): StreamTcp.Connect = - StreamTcp.Connect(remoteAddress, materializerSettings = Option(settings)) - - /** - * Java API: The Bind message is send to the StreamTcp manager actor, which is obtained via - * `StreamTcp.get(system).manager()`, in order to bind to a listening socket. The manager - * replies with a [[StreamTcp.TcpServerBinding]]. If the local port is set to 0 in - * the Bind message, then the [[StreamTcp.TcpServerBinding]] message should be inspected to find - * the actual port which was bound to. - * - * @param settings if Some, these materializer settings will be used for stream actors, - * else the ActorSystem's default materializer settings will be used. - * @param localAddress the socket address to bind to; use port zero for automatic assignment (i.e. an ephemeral port) - * @param backlog the number of unaccepted connections the O/S - * kernel will hold for this port before refusing connections. - * @param options Please refer to [[akka.io.TcpSO]] for a list of all supported options. - * @param idleTimeout the desired idle timeout on the accepted connections, infinite means "no timeout" - */ - def bind(settings: MaterializerSettings, - localAddress: InetSocketAddress, - backlog: Int, - options: java.lang.Iterable[SocketOption], - idleTimeout: Duration): StreamTcp.Bind = - StreamTcp.Bind(localAddress, Option(settings), backlog, Util.immutableSeq(options), idleTimeout) - - /** - * Java API: Message to open a listening socket without specifying options. - */ - def bind(settings: MaterializerSettings, localAddress: InetSocketAddress): StreamTcp.Bind = - StreamTcp.Bind(localAddress, Option(settings)) -} - -/** - * INTERNAL API - */ -private[akka] class StreamTcpExt(system: ExtendedActorSystem) extends IO.Extension { - val manager: ActorRef = system.systemActorOf(Props[StreamTcpManager], name = "IO-TCP-STREAM") -} - -/** - * INTERNAL API - */ -private[akka] object StreamTcpManager { - private[akka] case class ExposedProcessor(processor: Processor[ByteString, ByteString]) -} - -/** - * INTERNAL API - */ -private[akka] class StreamTcpManager extends Actor { - import StreamTcpManager._ - - var nameCounter = 0 - def encName(prefix: String, address: InetSocketAddress) = { - nameCounter += 1 - s"$prefix-$nameCounter-${URLEncoder.encode(address.toString, "utf-8")}" - } - - def receive: Receive = { - case StreamTcp.Connect(remoteAddress, localAddress, maybeMaterializerSettings, options, connectTimeout, idleTimeout) ⇒ - val connTimeout = connectTimeout match { - case x: FiniteDuration ⇒ Some(x) - case _ ⇒ None - } - val materializerSettings = maybeMaterializerSettings getOrElse MaterializerSettings(context.system) - - val processorActor = context.actorOf(TcpStreamActor.outboundProps( - Tcp.Connect(remoteAddress, localAddress, options, connTimeout, pullMode = true), - requester = sender(), - settings = materializerSettings), name = encName("client", remoteAddress)) - processorActor ! ExposedProcessor(ActorProcessor[ByteString, ByteString](processorActor)) - - case StreamTcp.Bind(localAddress, maybeMaterializerSettings, backlog, options, idleTimeout) ⇒ - val materializerSettings = maybeMaterializerSettings getOrElse MaterializerSettings(context.system) - - val publisherActor = context.actorOf(TcpListenStreamActor.props( - Tcp.Bind(context.system.deadLetters, localAddress, backlog, options, pullMode = true), - requester = sender(), - materializerSettings), name = encName("server", localAddress)) - publisherActor ! ExposedPublisher(ActorPublisher[Any](publisherActor)) - } -} - diff --git a/akka-stream/src/main/scala/akka/stream/io/StreamTcp.scala b/akka-stream/src/main/scala/akka/stream/io/StreamTcp.scala new file mode 100644 index 0000000000..6302d37e32 --- /dev/null +++ b/akka-stream/src/main/scala/akka/stream/io/StreamTcp.scala @@ -0,0 +1,274 @@ +/** + * Copyright (C) 2009-2014 Typesafe Inc. + */ +package akka.stream.io + +import akka.actor._ +import akka.io.Inet.SocketOption +import akka.io.Tcp +import akka.pattern.ask +import akka.stream.impl._ +import akka.stream.MaterializerSettings +import akka.stream.scaladsl._ +import akka.util.{ ByteString, Timeout } +import java.io.Closeable +import java.net.{ InetSocketAddress, URLEncoder } +import org.reactivestreams.{ Processor, Publisher, Subscriber, Subscription } +import scala.collection._ +import scala.concurrent.duration._ +import scala.concurrent.{ Promise, ExecutionContext, Future } +import scala.util.control.NoStackTrace +import scala.util.{ Failure, Success } + +object StreamTcp extends ExtensionId[StreamTcpExt] with ExtensionIdProvider { + + override def lookup = StreamTcp + override def createExtension(system: ExtendedActorSystem): StreamTcpExt = new StreamTcpExt(system) + override def get(system: ActorSystem): StreamTcpExt = super.get(system) + + /** + * The materialized result of an outgoing TCP connection stream. + */ + case class OutgoingTcpConnection(remoteAddress: InetSocketAddress, localAddress: InetSocketAddress) + + /** + * A flow representing an outgoing TCP connection, and the key used to get information about the materialized connection. + */ + case class OutgoingTcpFlow(flow: Flow[ByteString, ByteString], key: Key { type MaterializedType = Future[StreamTcp.OutgoingTcpConnection] }) + + /** + * The materialized result of a bound server socket. + */ + abstract sealed case class TcpServerBinding(localAddress: InetSocketAddress) extends Closeable + + /** + * INTERNAL API + */ + private[akka] object TcpServerBinding { + def apply(localAddress: InetSocketAddress): TcpServerBinding = + new TcpServerBinding(localAddress) { + override def close() = () + } + + def apply(localAddress: InetSocketAddress, closeable: Closeable): TcpServerBinding = + new TcpServerBinding(localAddress) { + override def close() = closeable.close() + } + } + + /** + * An incoming TCP connection. + */ + case class IncomingTcpConnection(remoteAddress: InetSocketAddress, stream: Flow[ByteString, ByteString]) + + /** + * The exception thrown on bind or accept failures. + */ + class IncomingTcpException(msg: String) extends RuntimeException(msg) with NoStackTrace +} + +/** + * INTERNAL API + */ +private[akka] class DelayedInitProcessor[I, O](val implFuture: Future[Processor[I, O]])(implicit ec: ExecutionContext) extends Processor[I, O] { + @volatile private var impl: Processor[I, O] = _ + private val setVarFuture = implFuture.andThen { case Success(p) ⇒ impl = p } + + override def onSubscribe(s: Subscription): Unit = implFuture.onComplete { + case Success(impl) ⇒ impl.onSubscribe(s) + case Failure(_) ⇒ s.cancel() + } + + override def onError(t: Throwable): Unit = { + if (impl eq null) setVarFuture.onSuccess { case p ⇒ p.onError(t) } + else impl.onError(t) + } + + override def onComplete(): Unit = { + if (impl eq null) setVarFuture.onSuccess { case p ⇒ p.onComplete() } + else impl.onComplete() + } + + override def onNext(t: I): Unit = impl.onNext(t) + + override def subscribe(s: Subscriber[_ >: O]): Unit = setVarFuture.onComplete { + case Success(impl) ⇒ impl.subscribe(s) + case Failure(e) ⇒ s.onError(e) + } +} + +/** + * INTERNAL API + */ +private[akka] object StreamTcpExt { + /** + * INTERNAL API + */ + class PreMaterializedOutgoingKey extends Key { + type MaterializedType = Future[StreamTcp.OutgoingTcpConnection] + + override def materialize(map: MaterializedMap) = + throw new IllegalArgumentException("This key have already been materialized by the TCP Processor") + } +} + +class StreamTcpExt(val system: ExtendedActorSystem) extends Extension { + import StreamTcpExt._ + import StreamTcp._ + + private val manager: ActorRef = system.systemActorOf(Props[StreamTcpManager], name = "IO-TCP-STREAM") + + /** + * Creates a Flow that represents a TCP connection to a remote host. The actual connection is only attempted + * when the Flow is materialized. The returned Flow is reusable, each new materialization will attempt to open + * a new connection to the remote host. + * + * @param remoteAddress the address to connect to + * @param localAddress optionally specifies a specific address to bind to + * @param options Please refer to [[akka.io.TcpSO]] for a list of all supported options. + * @param connectTimeout the desired timeout for connection establishment, infinite means "no timeout" + * @param idleTimeout the desired idle timeout on the connection, infinite means "no timeout" + * + */ + def connect(remoteAddress: InetSocketAddress, + localAddress: Option[InetSocketAddress] = None, + options: immutable.Traversable[SocketOption] = Nil, + connectTimeout: Duration = Duration.Inf, + idleTimeout: Duration = Duration.Inf): OutgoingTcpFlow = { + implicit val t = Timeout(3.seconds) + import system.dispatcher + + val key = new PreMaterializedOutgoingKey() + + val pipe = Pipe(key) { () ⇒ + { + val promise = Promise[OutgoingTcpConnection] + val future = (StreamTcp(system).manager ? StreamTcpManager.Connect(remoteAddress, localAddress, None, options, connectTimeout, idleTimeout)) + .mapTo[StreamTcpManager.ConnectReply] + future.map(r ⇒ OutgoingTcpConnection(r.remoteAddress, r.localAddress)).onComplete(promise.complete(_)) + (new DelayedInitProcessor[ByteString, ByteString](future.map(_.processor)), promise.future) + } + } + + StreamTcp.OutgoingTcpFlow(pipe, key) + } + + /** + * Returns a Source that represents a port listening to incoming connections. The actual binding to the local port + * happens when the Source is first materialized. This Source is not reusable until the listen port becomes available + * again. + * + * @param localAddress the socket address to bind to; use port zero for automatic assignment (i.e. an ephemeral port) + * @param backlog the number of unaccepted connections the O/S + * kernel will hold for this port before refusing connections. + * @param options Please refer to [[akka.io.TcpSO]] for a list of all supported options. + * @param idleTimeout the desired idle timeout on the accepted connections, infinite means "no timeout" + */ + def bind(localAddress: InetSocketAddress, + backlog: Int = 100, + options: immutable.Traversable[SocketOption] = Nil, + idleTimeout: Duration = Duration.Inf): KeyedSource[IncomingTcpConnection] { type MaterializedType = Future[TcpServerBinding] } = { + new KeyedActorFlowSource[IncomingTcpConnection] { + implicit val t = Timeout(3.seconds) + import system.dispatcher + + override def attach(flowSubscriber: Subscriber[IncomingTcpConnection], + materializer: ActorBasedFlowMaterializer, + flowName: String): MaterializedType = { + val bindingFuture = (StreamTcp(system).manager ? StreamTcpManager.Bind(localAddress, None, backlog, options, idleTimeout)) + .mapTo[StreamTcpManager.BindReply] + + bindingFuture.map(_.connectionStream).onComplete { + case Success(impl) ⇒ impl.subscribe(flowSubscriber) + case Failure(e) ⇒ flowSubscriber.onError(e) + } + + bindingFuture.map { bf ⇒ TcpServerBinding(bf.localAddress, bf.closeable) } + } + + override type MaterializedType = Future[TcpServerBinding] + } + } + +} + +/** + * INTERNAL API + */ +private[io] object StreamTcpManager { + /** + * INTERNAL API + */ + private[io] case class ConnectReply(remoteAddress: InetSocketAddress, + localAddress: InetSocketAddress, + processor: Processor[ByteString, ByteString]) + + /** + * INTERNAL API + */ + private[io] case class Connect(remoteAddress: InetSocketAddress, + localAddress: Option[InetSocketAddress] = None, + materializerSettings: Option[MaterializerSettings] = None, + options: immutable.Traversable[SocketOption] = Nil, + connectTimeout: Duration = Duration.Inf, + idleTimeout: Duration = Duration.Inf) + /** + * INTERNAL API + */ + private[io] case class Bind(localAddress: InetSocketAddress, + settings: Option[MaterializerSettings] = None, + backlog: Int = 100, + options: immutable.Traversable[SocketOption] = Nil, + idleTimeout: Duration = Duration.Inf) + + /** + * INTERNAL API + */ + private[io] case class BindReply(localAddress: InetSocketAddress, + connectionStream: Publisher[StreamTcp.IncomingTcpConnection], + closeable: Closeable) + + /** + * INTERNAL API + */ + private[io] case class ExposedProcessor(processor: Processor[ByteString, ByteString]) + +} + +/** + * INTERNAL API + */ +private[akka] class StreamTcpManager extends Actor { + import akka.stream.io.StreamTcpManager._ + + var nameCounter = 0 + def encName(prefix: String, address: InetSocketAddress) = { + nameCounter += 1 + s"$prefix-$nameCounter-${URLEncoder.encode(address.toString, "utf-8")}" + } + + def receive: Receive = { + case Connect(remoteAddress, localAddress, maybeMaterializerSettings, options, connectTimeout, idleTimeout) ⇒ + val connTimeout = connectTimeout match { + case x: FiniteDuration ⇒ Some(x) + case _ ⇒ None + } + val materializerSettings = maybeMaterializerSettings getOrElse MaterializerSettings(context.system) + + val processorActor = context.actorOf(TcpStreamActor.outboundProps( + Tcp.Connect(remoteAddress, localAddress, options, connTimeout, pullMode = true), + requester = sender(), + settings = materializerSettings), name = encName("client", remoteAddress)) + processorActor ! ExposedProcessor(ActorProcessor[ByteString, ByteString](processorActor)) + + case Bind(localAddress, maybeMaterializerSettings, backlog, options, idleTimeout) ⇒ + val materializerSettings = maybeMaterializerSettings getOrElse MaterializerSettings(context.system) + + val publisherActor = context.actorOf(TcpListenStreamActor.props( + Tcp.Bind(context.system.deadLetters, localAddress, backlog, options, pullMode = true), + requester = sender(), + materializerSettings), name = encName("server", localAddress)) + // this sends the ExposedPublisher message to the publisher actor automatically + ActorPublisher[Any](publisherActor) + } +} diff --git a/akka-stream/src/main/scala/akka/stream/io/TcpConnectionStream.scala b/akka-stream/src/main/scala/akka/stream/io/TcpConnectionStream.scala index 5902f5daf9..f8d04cbf5e 100644 --- a/akka-stream/src/main/scala/akka/stream/io/TcpConnectionStream.scala +++ b/akka-stream/src/main/scala/akka/stream/io/TcpConnectionStream.scala @@ -222,7 +222,7 @@ private[akka] class OutboundTcpStreamActor(val connectCmd: Connect, val requeste connection ! Register(self, keepOpenOnPeerClosed = true, useResumeWriting = false) tcpOutputs.setConnection(connection) tcpInputs.setConnection(connection) - requester ! StreamTcp.OutgoingTcpConnection(remoteAddress, localAddress, exposedProcessor) + requester ! StreamTcpManager.ConnectReply(remoteAddress, localAddress, exposedProcessor) initSteps.become(Actor.emptyBehavior) case f: CommandFailed ⇒ val ex = new TcpStreamException("Connection failed.") diff --git a/akka-stream/src/main/scala/akka/stream/io/TcpListenStreamActor.scala b/akka-stream/src/main/scala/akka/stream/io/TcpListenStreamActor.scala index 71faf8464b..5a5f6c8dee 100644 --- a/akka-stream/src/main/scala/akka/stream/io/TcpListenStreamActor.scala +++ b/akka-stream/src/main/scala/akka/stream/io/TcpListenStreamActor.scala @@ -10,8 +10,9 @@ import akka.io.Tcp._ import akka.io.{ IO, Tcp } import akka.stream.MaterializerSettings import akka.stream.impl._ +import akka.stream.scaladsl.{ Pipe, Flow } import akka.util.ByteString -import org.reactivestreams.Publisher +import org.reactivestreams.{ Processor, Publisher } import scala.util.control.NoStackTrace @@ -19,8 +20,6 @@ import scala.util.control.NoStackTrace * INTERNAL API */ private[akka] object TcpListenStreamActor { - class TcpListenStreamException(msg: String) extends RuntimeException(msg) with NoStackTrace - def props(bindCmd: Tcp.Bind, requester: ActorRef, settings: MaterializerSettings): Props = { Props(new TcpListenStreamActor(bindCmd, requester, settings)) } @@ -72,7 +71,7 @@ private[akka] class TcpListenStreamActor(bindCmd: Tcp.Bind, requester: ActorRef, nextPhase(runningPhase) listener ! ResumeAccepting(1) val target = self - requester ! StreamTcp.TcpServerBinding( + requester ! StreamTcpManager.BindReply( localAddress, primaryOutputs.getExposedPublisher.asInstanceOf[Publisher[StreamTcp.IncomingTcpConnection]], new Closeable { @@ -80,7 +79,7 @@ private[akka] class TcpListenStreamActor(bindCmd: Tcp.Bind, requester: ActorRef, }) subreceive.become(running) case f: CommandFailed ⇒ - val ex = new TcpListenStreamException("Bind failed") + val ex = new StreamTcp.IncomingTcpException("Bind failed") requester ! Status.Failure(ex) fail(ex) } @@ -90,7 +89,7 @@ private[akka] class TcpListenStreamActor(bindCmd: Tcp.Bind, requester: ActorRef, pendingConnection = (c, sender()) pump() case f: CommandFailed ⇒ - fail(new TcpListenStreamException(s"Command [${f.cmd}] failed")) + fail(new StreamTcp.IncomingTcpException(s"Command [${f.cmd}] failed")) case Unbind ⇒ cancel() pump() @@ -133,7 +132,7 @@ private[akka] class TcpListenStreamActor(bindCmd: Tcp.Bind, requester: ActorRef, val (connected: Connected, connection: ActorRef) = incomingConnections.dequeueInputElement() val tcpStreamActor = context.actorOf(TcpStreamActor.inboundProps(connection, settings)) val processor = ActorProcessor[ByteString, ByteString](tcpStreamActor) - primaryOutputs.enqueueOutputElement(StreamTcp.IncomingTcpConnection(connected.remoteAddress, processor, processor)) + primaryOutputs.enqueueOutputElement(StreamTcp.IncomingTcpConnection(connected.remoteAddress, Pipe(() ⇒ processor))) } def fail(e: Throwable): Unit = { 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 f1568f964d..033c701944 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Flow.scala @@ -19,6 +19,10 @@ object Flow { def adapt[I, O](flow: scaladsl.Flow[I, O]): javadsl.Flow[I, O] = new Flow(flow) + /** Create a `Flow` which can process elements of type `T`. */ + def empty[T](): javadsl.Flow[T, T] = + Flow.create() + /** Create a `Flow` which can process elements of type `T`. */ def create[T](): javadsl.Flow[T, T] = Flow.adapt[T, T](scaladsl.Pipe.empty[T]) @@ -31,7 +35,7 @@ object Flow { * Creates a `Flow` by using an empty [[FlowGraphBuilder]] on a block that expects a [[FlowGraphBuilder]] and * returns the `UndefinedSource` and `UndefinedSink`. */ - def apply[I, O](block: japi.Function[FlowGraphBuilder, akka.japi.Pair[UndefinedSource[I], UndefinedSink[O]]]): Flow[I, O] = { + def create[I, O](block: japi.Function[FlowGraphBuilder, akka.japi.Pair[UndefinedSource[I], UndefinedSink[O]]]): Flow[I, O] = { val sFlow = scaladsl.Flow() { b ⇒ val pair = block.apply(b.asJava) pair.first.asScala → pair.second.asScala @@ -51,6 +55,12 @@ object Flow { new Flow[I, O](sFlow) } + /** + * Create a flow from a seemingly disconnected Source and Sink pair. + */ + def create[I, O](sink: javadsl.Sink[I], source: javadsl.Source[O]): Flow[I, O] = + new Flow(scaladsl.Flow(sink.asScala, source.asScala)) + } /** Create a `Flow` which can process elements of type `T`. */ @@ -73,6 +83,12 @@ class Flow[-In, +Out](delegate: scaladsl.Flow[In, Out]) { def to(sink: javadsl.Sink[Out]): javadsl.Sink[In] = new Sink(delegate.to(sink.asScala)) + /** + * Join this [[Flow]] to another [[Flow]], by cross connecting the inputs and outputs, creating a [[RunnableFlow]] + */ + def join(flow: javadsl.Flow[Out, In]): javadsl.RunnableFlow = + new RunnableFlowAdapter(delegate.join(flow.asScala)) + /** * Connect the `KeyedSource` to this `Flow` and then connect it to the `KeyedSink` and run it. * @@ -340,6 +356,13 @@ class Flow[-In, +Out](delegate: scaladsl.Flow[In, Out]) { def concat(second: javadsl.Source[In]): javadsl.Flow[In, Out] = new Flow(delegate.concat(second.asScala)) + /** + * Add a key that will have a value available after materialization. + * The key can only use other keys if they have been added to the flow + * before this key. + */ + def withKey[T](key: javadsl.Key[T]): Flow[In, Out] = + new Flow(delegate.withKey(key.asScala)) } /** 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 58f536b0af..badb3d8f7d 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/MaterializedMap.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/MaterializedMap.scala @@ -5,6 +5,7 @@ package akka.stream.javadsl import akka.stream.javadsl import akka.stream.scaladsl +import scala.collection.JavaConverters.asJavaIteratorConverter /** * Java API @@ -14,6 +15,8 @@ import akka.stream.scaladsl * [[akka.stream.javadsl.Source#subscriber]] or [[akka.stream.javadsl.Sink#publisher]]. */ class MaterializedMap(delegate: scaladsl.MaterializedMap) { + def asScala: scaladsl.MaterializedMap = delegate + /** * Retrieve a materialized `Source`, e.g. the `Subscriber` of a [[akka.stream.javadsl.Source#subscriber]]. */ @@ -26,4 +29,49 @@ class MaterializedMap(delegate: scaladsl.MaterializedMap) { def get[D](key: javadsl.KeyedSink[_, D]): D = delegate.get(key.asScala).asInstanceOf[D] + /** + * Retrieve a materialized `Key`. + */ + def get[T](key: Key[T]): T = + delegate.get(key.asScala).asInstanceOf[T] + + /** + * Merge two materialized maps. + */ + def merge(otherMap: MaterializedMap): MaterializedMap = + if (this.isEmpty) otherMap + else if (otherMap.isEmpty) this + else new MaterializedMap(this.asScala.merge(otherMap.asScala)) + + /** + * Update the materialized map with a new value. + */ + def updated(key: Object, value: Object): MaterializedMap = + new MaterializedMap(delegate.updated(key, value)) + + /** + * Check if this map is empty. + */ + def isEmpty: Boolean = delegate.isEmpty + + /** + * An iterator over the key value pairs in this materialized map. + */ + def iterator: java.util.Iterator[akka.japi.Pair[Object, Object]] = { + delegate.iterator.map { case (a, b) ⇒ new akka.japi.Pair(a.asInstanceOf[Object], b.asInstanceOf[Object]) } asJava + } +} + +/** + * Java API + * + * A key that is not directly tied to a sink or source instance. + */ +class Key[T](delegate: scaladsl.Key) { + def asScala: scaladsl.Key = delegate + + /** + * Materialize the value for this key. All Sink and Source keys have been materialized and exist in the map. + */ + def materialize(map: MaterializedMap): Object = delegate.materialize(map.asScala).asInstanceOf[Object] } 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 5f77564070..c40d793135 100644 --- a/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala +++ b/akka-stream/src/main/scala/akka/stream/javadsl/Source.scala @@ -438,6 +438,13 @@ class Source[+Out](delegate: scaladsl.Source[Out]) { def flatten[U](strategy: akka.stream.FlattenStrategy[Out, U]): javadsl.Source[U] = new Source(delegate.flatten(strategy)) + /** + * Add a key that will have a value available after materialization. + * The key can only use other keys if they have been added to the source + * before this key. This also includes the keyed source if applicable. + */ + def withKey[T](key: javadsl.Key[T]): javadsl.Source[Out] = + new Source(delegate.withKey(key.asScala)) } /** diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/ActorFlowSource.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/ActorFlowSource.scala index 95332f9599..cac0278108 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/ActorFlowSource.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/ActorFlowSource.scala @@ -56,8 +56,10 @@ sealed trait ActorFlowSource[+Out] extends Source[Out] { override def to(sink: Sink[Out]): RunnableFlow = Pipe.empty[Out].withSource(this).to(sink) + override def withKey(key: Key): Source[Out] = Pipe.empty[Out].withSource(this).withKey(key) + /** INTERNAL API */ - override private[scaladsl] def andThen[U](op: AstNode) = SourcePipe(this, List(op)) //FIXME raw addition of AstNodes + override private[scaladsl] def andThen[U](op: AstNode) = SourcePipe(this, List(op), Nil) //FIXME raw addition of AstNodes } /** diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala index f24318248a..f4b008641c 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Flow.scala @@ -30,6 +30,11 @@ trait Flow[-In, +Out] extends FlowOps[Out] { */ def to(sink: Sink[Out]): Sink[In] + /** + * Join this [[Flow]] to another [[Flow]], by cross connecting the inputs and outputs, creating a [[RunnableFlow]] + */ + def join(flow: Flow[Out, In]): RunnableFlow + /** * * Connect the `Source` to this `Flow` and then connect it to the `Sink` and run it. The returned tuple contains @@ -59,9 +64,20 @@ trait Flow[-In, +Out] extends FlowOps[Out] { } } + /** + * Add a key that will have a value available after materialization. + * The key can only use other keys if they have been added to the flow + * before this key. + */ + def withKey(key: Key): Flow[In, Out] } object Flow { + /** + * Creates an empty `Flow` of type `T` + */ + def empty[T]: Flow[T, T] = Pipe.empty[T] + /** * Helper to create `Flow` without a [[Source]] or a [[Sink]]. * Example usage: `Flow[Int]` @@ -80,13 +96,18 @@ object Flow { * a [[FlowGraphBuilder]] and returns the `UndefinedSource` and `UndefinedSink`. */ def apply[I, O](graph: PartialFlowGraph)(block: FlowGraphBuilder ⇒ (UndefinedSource[I], UndefinedSink[O])): Flow[I, O] = - createFlowFromBuilder(new FlowGraphBuilder(graph.graph), block) + createFlowFromBuilder(new FlowGraphBuilder(graph), block) private def createFlowFromBuilder[I, O](builder: FlowGraphBuilder, block: FlowGraphBuilder ⇒ (UndefinedSource[I], UndefinedSink[O])): Flow[I, O] = { val (in, out) = block(builder) builder.partialBuild().toFlow(in, out) } + + /** + * Create a [[Flow]] from a seemingly disconnected [[Source]] and [[Sink]] pair. + */ + def apply[I, O](sink: Sink[I], source: Source[O]): Flow[I, O] = GraphFlow(sink, source) } /** diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/FlowGraph.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/FlowGraph.scala index fe4bc04ec1..dc0ecd3295 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/FlowGraph.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/FlowGraph.scala @@ -3,18 +3,15 @@ */ package akka.stream.scaladsl -import akka.stream.impl.Ast.FanInAstNode -import akka.stream.impl.Ast -import java.util +import java.util.concurrent.atomic.{ AtomicInteger, AtomicReference } -import org.reactivestreams.Publisher -import org.reactivestreams.Subscriber +import akka.stream.FlowMaterializer +import akka.stream.impl.Ast +import akka.stream.impl.Ast.FanInAstNode +import akka.stream.impl.{ DirectedGraphBuilder, Edge } +import org.reactivestreams._ import scala.language.existentials -import org.reactivestreams.Subscriber -import org.reactivestreams.Publisher -import akka.stream.FlowMaterializer -import akka.stream.impl.{ DirectedGraphBuilder, Edge, Vertex ⇒ GVertex } /** * Fan-in and fan-out vertices in the [[FlowGraph]] implements @@ -57,8 +54,16 @@ private[akka] sealed trait Junction[T] extends JunctionInPort[T] with JunctionOu override private[akka] def next = this } +private[akka] object Identity { + private val id = new AtomicInteger(1) + def getId: Int = id.getAndIncrement +} + private[akka] final class Identity[T]() extends FlowGraphInternal.InternalVertex with Junction[T] { - def name: Option[String] = Some("identity") + import Identity._ + + // This vertex can not have a name or else there can only be one instance in the whole graph + def name: Option[String] = None override private[akka] val vertex = this override val minimumInputCount: Int = 1 @@ -66,7 +71,7 @@ private[akka] final class Identity[T]() extends FlowGraphInternal.InternalVertex override val minimumOutputCount: Int = 1 override val maximumOutputCount: Int = 1 - override private[akka] def astNode = Ast.IdentityAstNode + override private[akka] val astNode = Ast.IdentityAstNode(getId) final override private[scaladsl] def newInstance() = new Identity[T]() } @@ -495,7 +500,7 @@ private[akka] object FlowGraphInternal { /** * These are unique keys, case class equality would break them. - * In the case of KeyedSources we MUST compare by object equality, in order to avoid ambigiousities in materialization. + * In the case of KeyedSources we MUST compare by object equality, in order to avoid ambiguities in materialization. */ final override def equals(other: Any): Boolean = other match { case v: SinkVertex ⇒ (sink, v.sink) match { @@ -550,11 +555,65 @@ private[akka] object FlowGraphInternal { } + /** + * INTERNAL API + * + * This is a minimalistic processor to tie a loop that when we know that we are materializing a flow + * and only have one upstream and one downstream. + * + * It can only be used with a SourceVertex/SinkVertex during a flow join, since if the graph would + * be copied into another graph then the SourceVertex/SinkVertex would still point to the same instance + * of the IdentityProcessor. + */ + class IdentityProcessor extends Processor[Any, Any] { + import akka.stream.actor.ActorSubscriber.OnSubscribe + import akka.stream.actor.ActorSubscriberMessage._ + + @volatile private var subscriber: Subscriber[Any] = null + private val state = new AtomicReference[AnyRef]() + + override def onSubscribe(s: Subscription) = + if (subscriber != null) subscriber.onSubscribe(s) + else state.getAndSet(OnSubscribe(s)) match { + case sub: Subscriber[Any] ⇒ sub.onSubscribe(s) + case _ ⇒ + } + + override def onError(t: Throwable) = + if (subscriber != null) subscriber.onError(t) + else state.getAndSet(OnError(t)) match { + case sub: Subscriber[Any] ⇒ sub.onError(t) + case _ ⇒ + } + + override def onComplete() = + if (subscriber != null) subscriber.onComplete() + else state.getAndSet(OnComplete) match { + case sub: Subscriber[Any] ⇒ sub.onComplete() + case _ ⇒ + } + + override def onNext(t: Any) = + if (subscriber != null) subscriber.onNext(t) + else throw new IllegalStateException("IdentityProcessor received onNext before signaling demand") + + override def subscribe(sub: Subscriber[_ >: Any]) = + if (subscriber != null) sub.onError(new IllegalStateException("IdentityProcessor can only be subscribed to once")) + else { + subscriber = sub.asInstanceOf[Subscriber[Any]] + if (!state.compareAndSet(null, sub)) state.get match { + case OnSubscribe(s) ⇒ sub.onSubscribe(s) + case OnError(t) ⇒ sub.onError(t) + case OnComplete ⇒ sub.onComplete() + case s ⇒ throw new IllegalStateException(s"IdentityProcessor found unknown state $s") + } + } + } } object FlowGraphBuilder { private[scaladsl] def apply[T](partialFlowGraph: PartialFlowGraph)(block: FlowGraphBuilder ⇒ T): T = { - val builder = new FlowGraphBuilder(partialFlowGraph.graph) + val builder = new FlowGraphBuilder(partialFlowGraph) block(builder) } } @@ -563,16 +622,21 @@ object FlowGraphBuilder { * Builder of [[FlowGraph]] and [[PartialFlowGraph]]. * Syntactic sugar is provided by [[FlowGraphImplicits]]. */ -class FlowGraphBuilder private[akka] (_graph: DirectedGraphBuilder[FlowGraphInternal.EdgeLabel, FlowGraphInternal.Vertex]) { +class FlowGraphBuilder private[akka] ( + _graph: DirectedGraphBuilder[FlowGraphInternal.EdgeLabel, FlowGraphInternal.Vertex], + private var cyclesAllowed: Boolean, + private var disconnectedAllowed: Boolean) { + import FlowGraphInternal._ private val graph = new DirectedGraphBuilder[FlowGraphInternal.EdgeLabel, FlowGraphInternal.Vertex]() - private[akka] def this() = this(new DirectedGraphBuilder[FlowGraphInternal.EdgeLabel, FlowGraphInternal.Vertex]()) + private[akka] def this() = this(new DirectedGraphBuilder[FlowGraphInternal.EdgeLabel, FlowGraphInternal.Vertex](), false, false) private var edgeQualifier = 0 importGraph(_graph) - private var cyclesAllowed = false + private[akka] def this(flowGraph: FlowGraphLike) = + this(flowGraph.graph, flowGraph.cyclesAllowed, flowGraph.disconnectedAllowed) private def addSourceToPipeEdge[In, Out](source: Source[In], pipe: Pipe[In, Out], junctionIn: JunctionInPort[Out]): this.type = { val sourceVertex = SourceVertex(source) @@ -650,7 +714,7 @@ class FlowGraphBuilder private[akka] (_graph: DirectedGraphBuilder[FlowGraphInte def addEdge[In, Out](source: Source[In], flow: Flow[In, Out], junctionIn: JunctionInPort[Out]): this.type = { (source, flow) match { case (spipe: SourcePipe[In], pipe: Pipe[In, Out]) ⇒ - addSourceToPipeEdge(spipe.input, Pipe(spipe.ops).appendPipe(pipe), junctionIn) + addSourceToPipeEdge(spipe.input, Pipe(spipe).appendPipe(pipe), junctionIn) case (gsource: GraphSource[_, In], _) ⇒ val tOut = UndefinedSink[In] val tIn = UndefinedSource[Out] @@ -669,7 +733,7 @@ class FlowGraphBuilder private[akka] (_graph: DirectedGraphBuilder[FlowGraphInte def addEdge[In, Out](junctionOut: JunctionOutPort[In], flow: Flow[In, Out], sink: Sink[Out]): this.type = { (flow, sink) match { case (pipe: Pipe[In, Out], spipe: SinkPipe[Out]) ⇒ - addPipeToSinkEdge(junctionOut, pipe.appendPipe(Pipe(spipe.ops)), spipe.output) + addPipeToSinkEdge(junctionOut, pipe.appendPipe(Pipe(spipe)), spipe.output) case (_, gsink: GraphSink[Out, _]) ⇒ val tOut = UndefinedSink[In] val tIn = UndefinedSource[Out] @@ -689,15 +753,15 @@ class FlowGraphBuilder private[akka] (_graph: DirectedGraphBuilder[FlowGraphInte (source, flow, sink) match { case (sourcePipe: SourcePipe[In], pipe: Pipe[In, Out], sinkPipe: SinkPipe[Out]) ⇒ val src = sourcePipe.input - val newPipe = Pipe(sourcePipe.ops).via(pipe).via(Pipe(sinkPipe.ops)) + val newPipe = Pipe(sourcePipe).via(pipe).via(Pipe(sinkPipe)) 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).via(pipe) + val newPipe = Pipe(sourcePipe).via(pipe) 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.via(Pipe(sinkPipe.ops)) + val newPipe = pipe.via(Pipe(sinkPipe)) val snk = sinkPipe.output addEdge(source, newPipe, snk) // recursive, but now it is a Source-Pipe-Sink case (_, gflow: GraphFlow[In, _, _, Out], _) ⇒ @@ -736,7 +800,7 @@ class FlowGraphBuilder private[akka] (_graph: DirectedGraphBuilder[FlowGraphInte def addEdge[In, Out](source: UndefinedSource[In], flow: Flow[In, Out], sink: Sink[Out]): this.type = { (flow, sink) match { case (pipe: Pipe[In, Out], spipe: SinkPipe[Out]) ⇒ - addGraphEdge(source, SinkVertex(spipe.output), pipe.appendPipe(Pipe(spipe.ops)), inputPort = UnlabeledPort, outputPort = UnlabeledPort) + addGraphEdge(source, SinkVertex(spipe.output), pipe.appendPipe(Pipe(spipe)), inputPort = UnlabeledPort, outputPort = UnlabeledPort) case (gflow: GraphFlow[In, _, _, Out], _) ⇒ val tOut = UndefinedSink[In] val tIn = UndefinedSource[Out] @@ -759,7 +823,7 @@ class FlowGraphBuilder private[akka] (_graph: DirectedGraphBuilder[FlowGraphInte def addEdge[In, Out](source: Source[In], flow: Flow[In, Out], sink: UndefinedSink[Out]): this.type = { (flow, source) match { case (pipe: Pipe[In, Out], spipe: SourcePipe[Out]) ⇒ - addGraphEdge(SourceVertex(spipe.input), sink, Pipe(spipe.ops).appendPipe(pipe), inputPort = UnlabeledPort, outputPort = UnlabeledPort) + addGraphEdge(SourceVertex(spipe.input), sink, Pipe(spipe).appendPipe(pipe), inputPort = UnlabeledPort, outputPort = UnlabeledPort) case (_, gsource: GraphSource[_, In]) ⇒ val tOut1 = UndefinedSource[In] val tOut2 = UndefinedSink[In] @@ -813,7 +877,7 @@ class FlowGraphBuilder private[akka] (_graph: DirectedGraphBuilder[FlowGraphInte graph.remove(existing.label) sink match { case spipe: SinkPipe[Out] ⇒ - val pipe = edge.label.pipe.appendPipe(Pipe(spipe.ops)) + val pipe = edge.label.pipe.appendPipe(Pipe(spipe)) addOrReplaceSinkEdge(edge.from.label, SinkVertex(spipe.output), pipe, edge.label.inputPort, edge.label.outputPort) case gsink: GraphSink[Out, _] ⇒ gsink.importAndConnect(this, token) @@ -833,7 +897,7 @@ class FlowGraphBuilder private[akka] (_graph: DirectedGraphBuilder[FlowGraphInte graph.remove(existing.label) source match { case spipe: SourcePipe[In] ⇒ - val pipe = Pipe(spipe.ops).appendPipe(edge.label.pipe) + val pipe = Pipe(spipe).appendPipe(edge.label.pipe) addOrReplaceSourceEdge(SourceVertex(spipe.input), edge.to.label, pipe, edge.label.inputPort, edge.label.outputPort) case gsource: GraphSource[_, In] ⇒ gsource.importAndConnect(this, token) @@ -853,7 +917,10 @@ class FlowGraphBuilder private[akka] (_graph: DirectedGraphBuilder[FlowGraphInte * by first importing the other `PartialFlowGraph` with [[#importPartialFlowGraph]] * and then connect them with this method. */ - def connect[A, B](out: UndefinedSink[A], flow: Flow[A, B], in: UndefinedSource[B]): this.type = { + def connect[A, B](out: UndefinedSink[A], flow: Flow[A, B], in: UndefinedSource[B]): this.type = + connect(out, flow, in, false) + + private[scaladsl] def connect[A, B](out: UndefinedSink[A], flow: Flow[A, B], in: UndefinedSource[B], joining: Boolean): this.type = { require(graph.contains(out), s"Couldn't connect from [$out], no matching UndefinedSink") require(graph.contains(in), s"Couldn't connect to [$in], no matching UndefinedSource") @@ -861,11 +928,42 @@ class FlowGraphBuilder private[akka] (_graph: DirectedGraphBuilder[FlowGraphInte val inEdge = graph.get(in).outgoing.head flow match { case pipe: Pipe[A, B] ⇒ - val newPipe = outEdge.label.pipe.appendPipe(pipe.asInstanceOf[Pipe[Any, Nothing]]).appendPipe(inEdge.label.pipe) graph.remove(out) graph.remove(in) - addOrReplaceGraphEdge(outEdge.from.label, inEdge.to.label, newPipe, inEdge.label.inputPort, outEdge.label.outputPort) + if (out == inEdge.to.label && in == outEdge.from.label) { + require(joining == true, "Connecting an edge to itself should only happen when joining flows") + val newPipe = outEdge.label.pipe.appendPipe(pipe.asInstanceOf[Pipe[Any, Nothing]]) + val identityProcessor = new IdentityProcessor + addEdge(Source(identityProcessor), newPipe, Sink(identityProcessor)) + } else if (joining == true) { + val identityProcessor = new IdentityProcessor + val sinkVertex = SinkVertex(Sink(identityProcessor)) + val sourceVertex = SourceVertex(Source(identityProcessor)) + val newPipe = outEdge.label.pipe.appendPipe(pipe.asInstanceOf[Pipe[Any, Nothing]]) + outEdge.from.label match { + case s: SourceVertex ⇒ + // direct source to sink connection, needs an identity vertex in between + val id = new Identity[Any] + addOrReplaceSinkEdge(outEdge.from.label, id, newPipe, UnlabeledPort, outEdge.label.outputPort) + addOrReplaceSinkEdge(id, sinkVertex, Pipe.empty[Any], UnlabeledPort, UnlabeledPort) + case _ ⇒ + addOrReplaceSinkEdge(outEdge.from.label, sinkVertex, newPipe, UnlabeledPort, outEdge.label.outputPort) + } + inEdge.to.label match { + case s: SinkVertex ⇒ + // direct source to sink connection, needs an identity vertex in between + val id = new Identity[Any] + addOrReplaceSourceEdge(id, inEdge.to.label, inEdge.label.pipe, inEdge.label.inputPort, UnlabeledPort) + addOrReplaceSourceEdge(sourceVertex, id, Pipe.empty[Any], UnlabeledPort, UnlabeledPort) + case _ ⇒ + addOrReplaceSourceEdge(sourceVertex, inEdge.to.label, inEdge.label.pipe, inEdge.label.inputPort, UnlabeledPort) + } + } else { + val newPipe = outEdge.label.pipe.appendPipe(pipe.asInstanceOf[Pipe[Any, Nothing]]).appendPipe(inEdge.label.pipe) + addOrReplaceGraphEdge(outEdge.from.label, inEdge.to.label, newPipe, inEdge.label.inputPort, outEdge.label.outputPort) + } case gflow: GraphFlow[A, _, _, B] ⇒ + require(joining == false, "Graph flows should have been split up to pipes while joining") gflow.importAndConnect(this, out, in) case x ⇒ throwUnsupportedValue(x) } @@ -917,12 +1015,20 @@ class FlowGraphBuilder private[akka] (_graph: DirectedGraphBuilder[FlowGraphInte cyclesAllowed = true } + /** + * Allow multiple apparently disconnected graphs in the same graph. + * They might still be connected through source/sink pairs. + */ + private[scaladsl] def allowDisconnected(): Unit = { + disconnectedAllowed = true + } + private def checkAddSourceSinkPrecondition(vertex: Vertex): Unit = { checkAmbigiousKeyedElement(vertex) vertex match { case node @ (_: UndefinedSource[_] | _: UndefinedSink[_]) ⇒ - require(!graph.contains(node), s"[$node] instance is already used in this flow graph") + require(!graph.contains(node), s"[$node] instance is already used in this flow graph [${graph.nodes.map(_.toString)}]") case _ ⇒ // ok } } @@ -990,7 +1096,7 @@ class FlowGraphBuilder private[akka] (_graph: DirectedGraphBuilder[FlowGraphInte */ private[akka] def partialBuild(): PartialFlowGraph = { checkPartialBuildPreconditions() - new PartialFlowGraph(graph.copy()) + new PartialFlowGraph(graph.copy(), cyclesAllowed, disconnectedAllowed) } private def checkPartialBuildPreconditions(): Unit = { @@ -1048,7 +1154,8 @@ class FlowGraphBuilder private[akka] (_graph: DirectedGraphBuilder[FlowGraphInte require(graph.exists(_.inDegree == 0), "Graph must have at least one source") - require(graph.isWeaklyConnected, "Graph must be connected") + if (!disconnectedAllowed) + require(graph.isWeaklyConnected, "Graph must be connected") } } @@ -1064,7 +1171,7 @@ object FlowGraph { * Build a [[FlowGraph]] from scratch. */ def apply(block: FlowGraphBuilder ⇒ Unit): FlowGraph = - apply(new DirectedGraphBuilder[FlowGraphInternal.EdgeLabel, FlowGraphInternal.Vertex])(block) + apply(new FlowGraphBuilder())(block) /** * Continue building a [[FlowGraph]] from an existing `PartialFlowGraph`. @@ -1072,17 +1179,9 @@ object FlowGraph { * [[FlowGraphBuilder#attachSource]] and [[FlowGraphBuilder#attachSink]] */ def apply(partialFlowGraph: PartialFlowGraph)(block: FlowGraphBuilder ⇒ Unit): FlowGraph = - apply(partialFlowGraph.graph)(block) + apply(new FlowGraphBuilder(partialFlowGraph))(block) - /** - * Continue building a [[FlowGraph]] from an existing `FlowGraph`. - * For example you can connect more output flows to a [[Broadcast]] vertex. - */ - def apply(flowGraph: FlowGraph)(block: FlowGraphBuilder ⇒ Unit): FlowGraph = - apply(flowGraph.graph)(block) - - private def apply(graph: DirectedGraphBuilder[FlowGraphInternal.EdgeLabel, FlowGraphInternal.Vertex])(block: FlowGraphBuilder ⇒ Unit): FlowGraph = { - val builder = new FlowGraphBuilder(graph) + private def apply(builder: FlowGraphBuilder)(block: FlowGraphBuilder ⇒ Unit): FlowGraph = { block(builder) builder.build() } @@ -1128,19 +1227,12 @@ class FlowGraph private[akka] (private[akka] val graph: DirectedGraphBuilder[Flo /** * Run FlowGraph that only contains one edge from a `Source` to a `Sink`. */ - 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[KeyedSink[_], Any] = sinkVertex match { - case SinkVertex(sink: KeyedSink[_]) ⇒ Map(sink -> mf.get(sink)) - case _ ⇒ Map.empty - } - new MaterializedFlowGraph(materializedSources, materializedSinks) - } + private def runSimple(sourceVertex: SourceVertex, sinkVertex: SinkVertex, pipe: Pipe[Any, Nothing])(implicit materializer: FlowMaterializer): MaterializedMap = + pipe.withSource(sourceVertex.source).withSink(sinkVertex.sink).run() + /** + * This is the normal marterialization of a graph. + */ private def runGraph()(implicit materializer: FlowMaterializer): MaterializedMap = { // start with sinks @@ -1152,7 +1244,7 @@ class FlowGraph private[akka] (private[akka] val graph: DirectedGraphBuilder[Flo downstreamSubscriber: Map[E, Subscriber[Any]] = Map.empty, upstreamPublishers: Map[E, Publisher[Any]] = Map.empty, sources: Map[SourceVertex, SinkPipe[Any]] = Map.empty, - materializedSinks: Map[KeyedSink[_], Any] = Map.empty) + materializedMap: MaterializedMap = MaterializedMap.empty) val result = startingNodes.foldLeft(Memo()) { case (memo, start) ⇒ @@ -1165,18 +1257,13 @@ class FlowGraph private[akka] (private[akka] val graph: DirectedGraphBuilder[Flo val pipe = edge.label.pipe // returns the materialized sink, if any - def connectToDownstream(publisher: Publisher[Any]): Option[(KeyedSink[_], Any)] = { + def connectToDownstream(publisher: Publisher[Any]): MaterializedMap = { val f = pipe.withSource(PublisherSource(publisher)) edge.to.label match { - 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.withSink(SubscriberSink(memo.downstreamSubscriber(edge))).run() - None } } @@ -1190,10 +1277,10 @@ class FlowGraph private[akka] (private[akka] val graph: DirectedGraphBuilder[Flo case v: InternalVertex ⇒ if (memo.upstreamPublishers.contains(edge)) { // vertex already materialized - val materializedSink = connectToDownstream(memo.upstreamPublishers(edge)) + val materializedMap = connectToDownstream(memo.upstreamPublishers(edge)) memo.copy( visited = memo.visited + edge, - materializedSinks = memo.materializedSinks ++ materializedSink) + materializedMap = memo.materializedMap.merge(materializedMap)) } else { val op = v.astNode @@ -1205,32 +1292,28 @@ class FlowGraph private[akka] (private[akka] val graph: DirectedGraphBuilder[Flo val edgePublishers = edge.from.outgoing.toSeq.sortBy(_.label.outputPort).zip(publishers).toMap val publisher = edgePublishers(edge) - val materializedSink = connectToDownstream(publisher) + val materializedMap = connectToDownstream(publisher) memo.copy( visited = memo.visited + edge, downstreamSubscriber = memo.downstreamSubscriber ++ edgeSubscribers, upstreamPublishers = memo.upstreamPublishers ++ edgePublishers, - materializedSinks = memo.materializedSinks ++ materializedSink) + materializedMap = memo.materializedMap.merge(materializedMap)) } } } } - } - // connect all input sources as the last thing - val materializedSources = result.sources.foldLeft(Map.empty[KeyedSource[_], Any]) { + // connect all input sources as the last thing (also picks up materialized keys) + val materializedMap: MaterializedMap = result.sources.foldLeft(result.materializedMap) { case (acc, (SourceVertex(source), pipe)) ⇒ - val mf = pipe.withSource(source).run() - source match { - case sourceKey: KeyedSource[_] ⇒ acc.updated(sourceKey, mf.get(sourceKey)) - case _ ⇒ acc - } + + acc.merge(pipe.withSource(source).run()) } - new MaterializedFlowGraph(materializedSources, result.materializedSinks) + materializedMap } } @@ -1246,23 +1329,16 @@ object PartialFlowGraph { * Build a [[PartialFlowGraph]] from scratch. */ def apply(block: FlowGraphBuilder ⇒ Unit): PartialFlowGraph = - apply(new DirectedGraphBuilder[FlowGraphInternal.EdgeLabel, FlowGraphInternal.Vertex])(block) + apply(new FlowGraphBuilder())(block) /** * Continue building a [[PartialFlowGraph]] from an existing `PartialFlowGraph`. */ def apply(partialFlowGraph: PartialFlowGraph)(block: FlowGraphBuilder ⇒ Unit): PartialFlowGraph = - apply(partialFlowGraph.graph)(block) + apply(new FlowGraphBuilder(partialFlowGraph))(block) - /** - * Continue building a [[PartialFlowGraph]] from an existing `PartialFlowGraph`. - */ - def apply(flowGraph: FlowGraph)(block: FlowGraphBuilder ⇒ Unit): PartialFlowGraph = - apply(flowGraph.graph)(block) - - private def apply(graph: DirectedGraphBuilder[FlowGraphInternal.EdgeLabel, FlowGraphInternal.Vertex])(block: FlowGraphBuilder ⇒ Unit): PartialFlowGraph = { + private def apply(builder: FlowGraphBuilder)(block: FlowGraphBuilder ⇒ Unit): PartialFlowGraph = { // FlowGraphBuilder does a full import on the passed graph, so no defensive copy needed - val builder = new FlowGraphBuilder(graph) block(builder) builder.partialBuild() } @@ -1275,7 +1351,10 @@ object PartialFlowGraph { * Build a `PartialFlowGraph` by starting with one of the `apply` methods in * in [[FlowGraph$ companion object]]. Syntactic sugar is provided by [[FlowGraphImplicits]]. */ -class PartialFlowGraph private[akka] (private[akka] val graph: DirectedGraphBuilder[FlowGraphInternal.EdgeLabel, FlowGraphInternal.Vertex]) { +class PartialFlowGraph private[akka] (private[akka] val graph: DirectedGraphBuilder[FlowGraphInternal.EdgeLabel, FlowGraphInternal.Vertex], + private[scaladsl] override val cyclesAllowed: Boolean, + private[scaladsl] override val disconnectedAllowed: Boolean) extends FlowGraphLike { + import FlowGraphInternal._ def undefinedSources: Set[UndefinedSource[_]] = @@ -1293,7 +1372,6 @@ class PartialFlowGraph private[akka] (private[akka] val graph: DirectedGraphBuil * no [[UndefinedSource]] in the graph, and you need to provide it as a parameter. */ def toSource[O](out: UndefinedSink[O]): Source[O] = { - require(graph.contains(out), s"Couldn't create Source with [$out], no matching UndefinedSink") checkUndefinedSinksAndSources(sources = Nil, sinks = List(out), description = "Source") GraphSource(this, out, Pipe.empty[O]) } @@ -1333,31 +1411,14 @@ class PartialFlowGraph private[akka] (private[akka] val graph: DirectedGraphBuil } /** - * Returned by [[FlowGraph#run]] and can be used to retrieve the materialized - * `Source` inputs or `Sink` outputs. + * INTERNAL API + * + * Common things that the builder needs to extract from FlowGraph and PartialFlowGraph */ -private[scaladsl] class MaterializedFlowGraph(materializedSources: Map[KeyedSource[_], Any], materializedSinks: Map[KeyedSink[_], Any]) - extends MaterializedMap { - - override def get(key: Source[_]): key.MaterializedType = - key match { - case k: KeyedSource[_] ⇒ materializedSources.get(k) match { - case Some(matSource) ⇒ matSource.asInstanceOf[key.MaterializedType] - case None ⇒ - throw new IllegalArgumentException(s"Source key [$key] doesn't exist in this flow graph") - } - case _ ⇒ ().asInstanceOf[key.MaterializedType] - } - - def get(key: Sink[_]): key.MaterializedType = - key match { - case k: KeyedSink[_] ⇒ materializedSinks.get(k) match { - case Some(matSink) ⇒ matSink.asInstanceOf[key.MaterializedType] - case None ⇒ - throw new IllegalArgumentException(s"Sink key [$key] doesn't exist in this flow graph") - } - case _ ⇒ ().asInstanceOf[key.MaterializedType] - } +private[scaladsl] trait FlowGraphLike { + private[scaladsl] def graph: DirectedGraphBuilder[FlowGraphInternal.EdgeLabel, FlowGraphInternal.Vertex] + private[scaladsl] def cyclesAllowed: Boolean + private[scaladsl] def disconnectedAllowed: Boolean } /** diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/GraphFlow.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/GraphFlow.scala index 2e5805680b..79c6282986 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/GraphFlow.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/GraphFlow.scala @@ -6,6 +6,39 @@ package akka.stream.scaladsl import akka.stream.impl.Ast.AstNode import scala.annotation.unchecked.uncheckedVariance +import scala.collection.immutable + +/** + * INTERNAL API + */ +private[scaladsl] object GraphFlow { + + /** + * Create a [[GraphFlow]] from this [[Flow]] + */ + def apply[In, Out](flow: Flow[In, Out]) = flow match { + case gFlow: GraphFlow[In, _, _, Out] ⇒ gFlow + case _ ⇒ Flow() { implicit b ⇒ + import FlowGraphImplicits._ + val in = UndefinedSource[In] + val out = UndefinedSink[Out] + in ~> flow ~> out + in -> out + } + } + + /** + * Create a [[GraphFlow]] from a seemingly disconnected [[Source]] and [[Sink]] pair. + */ + def apply[I, O](sink: Sink[I], source: Source[O]) = Flow() { implicit b ⇒ + import FlowGraphImplicits._ + val in = UndefinedSource[I] + val out = UndefinedSink[O] + in ~> Flow[I] ~> sink + source ~> Flow[O] ~> out + in -> out + } +} private[scaladsl] case class GraphFlow[-In, CIn, COut, +Out]( inPipe: Pipe[In, CIn], @@ -14,16 +47,15 @@ private[scaladsl] case class GraphFlow[-In, CIn, COut, +Out]( out: UndefinedSink[COut], outPipe: Pipe[COut, Out]) extends Flow[In, Out] { - override type Repr[+O] = GraphFlow[In @uncheckedVariance, CIn, COut, O] private[scaladsl] def prepend[T](pipe: Pipe[T, In]): GraphFlow[T, CIn, COut, Out] = copy(inPipe = pipe.appendPipe(inPipe)) private[scaladsl] def prepend(pipe: SourcePipe[In]): GraphSource[COut, Out] = { - val newGraph = PartialFlowGraph(graph) { b ⇒ - b.attachSource(in, pipe.appendPipe(inPipe)) - } - GraphSource(newGraph, out, outPipe) + val b = new FlowGraphBuilder() + val (nIn, nOut) = remap(b) + b.attachSource(nIn, pipe.appendPipe(inPipe)) + GraphSource(b.partialBuild(), nOut, outPipe) } private[scaladsl] def remap(builder: FlowGraphBuilder): (UndefinedSource[CIn], UndefinedSink[COut]) = { @@ -48,6 +80,7 @@ private[scaladsl] case class GraphFlow[-In, CIn, COut, +Out]( (b.partialBuild(), oOut) } GraphFlow(inPipe, in, newGraph, nOut, gFlow.outPipe) + case x ⇒ FlowGraphInternal.throwUnsupportedValue(x) } override def to(sink: Sink[Out]) = sink match { @@ -65,6 +98,26 @@ private[scaladsl] case class GraphFlow[-In, CIn, COut, +Out]( case sink: Sink[Out] ⇒ to(Pipe.empty.withSink(sink)) // recursive, but now it is a SinkPipe } + override def join(flow: Flow[Out, In]): RunnableFlow = flow match { + case pipe: Pipe[Out, In] ⇒ FlowGraph(graph) { b ⇒ + b.connect(out, outPipe.via(pipe).via(inPipe), in, joining = true) + b.allowCycles() + b.allowDisconnected() + } + case gFlow: GraphFlow[Out, _, _, In] ⇒ + FlowGraph(graph) { b ⇒ + val (oIn, oOut) = gFlow.remap(b) + b.connect(out, outPipe.via(gFlow.inPipe), oIn, joining = true) + b.connect(oOut, gFlow.outPipe.via(inPipe), in, joining = true) + b.allowCycles() + b.allowDisconnected() + } + case x ⇒ FlowGraphInternal.throwUnsupportedValue(x) + } + + // FIXME #16379 This key will be materalized to early + override def withKey(key: Key): Flow[In, Out] = this.copy(outPipe = outPipe.withKey(key)) + override private[scaladsl] def andThen[T](op: AstNode): Repr[T] = copy(outPipe = outPipe.andThen(op)) } @@ -107,6 +160,9 @@ private[scaladsl] case class GraphSource[COut, +Out](graph: PartialFlowGraph, ou to(Pipe.empty.withSink(sink)) // recursive, but now it is a SinkPipe } + // FIXME #16379 This key will be materalized to early + override def withKey(key: Key): Source[Out] = this.copy(outPipe = outPipe.withKey(key)) + override private[scaladsl] def andThen[T](op: AstNode): Repr[T] = copy(outPipe = outPipe.andThen(op)) } diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/MaterializedMap.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/MaterializedMap.scala index 3436900d4c..e90ff0f3d9 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/MaterializedMap.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/MaterializedMap.scala @@ -18,4 +18,85 @@ trait MaterializedMap { * Retrieve a materialized `Sink`, e.g. the `Publisher` of a [[PublisherSink]]. */ def get(key: Sink[_]): key.MaterializedType + + /** + * Retrieve a materialized `Key`. + */ + def get(key: Key): key.MaterializedType + + /** + * Merge two materialized maps. + */ + def merge(otherMap: MaterializedMap): MaterializedMap + + /** + * Update the materialized map with a new value. + */ + def updated(key: AnyRef, value: Any): MaterializedMap + + /** + * Check if this map is empty. + */ + def isEmpty: Boolean + + /** + * An iterator over the key value pairs in this materialized map. + */ + def iterator: Iterator[(AnyRef, Any)] +} + +object MaterializedMap { + private val emptyInstance = MaterializedMapImpl(Map.empty) + + def empty: MaterializedMap = emptyInstance +} + +/** + * A key that is not directly tied to a sink or source instance. + * + * FIXME #16380 Clean up the overlap between Keys/Sinks/Sources + */ +trait Key { + type MaterializedType + + /** + * Materialize the value for this key. All Sink and Source keys have been materialized and exist in the map. + */ + def materialize(map: MaterializedMap): MaterializedType +} + +private[stream] case class MaterializedMapImpl(map: Map[AnyRef, Any]) extends MaterializedMap { + private def failure(keyType: String, key: AnyRef) = new IllegalArgumentException(s"$keyType [$key] doesn't exist in this flow") + + override def get(key: Source[_]): key.MaterializedType = key match { + case _: KeyedSource[_] ⇒ map.get(key) match { + case Some(v) ⇒ v.asInstanceOf[key.MaterializedType] + case None ⇒ throw failure("Source", key) + } + case _ ⇒ ().asInstanceOf[key.MaterializedType] + } + + override def get(key: Sink[_]): key.MaterializedType = key match { + case _: KeyedSink[_] ⇒ map.get(key) match { + case Some(v) ⇒ v.asInstanceOf[key.MaterializedType] + case None ⇒ throw failure("Sink", key) + } + case _ ⇒ ().asInstanceOf[key.MaterializedType] + } + + override def get(key: Key): key.MaterializedType = map.get(key) match { + case Some(v) ⇒ v.asInstanceOf[key.MaterializedType] + case None ⇒ throw failure("Key", key) + } + + override def merge(otherMap: MaterializedMap) = + if (map.isEmpty) otherMap + else if (otherMap.isEmpty) this + else MaterializedMapImpl(map ++ otherMap.iterator) + + override def updated(key: AnyRef, value: Any) = MaterializedMapImpl(map.updated(key, value)) + + override def isEmpty = map.isEmpty + + override def iterator = map.iterator } diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Pipe.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Pipe.scala index 768d364f1f..65f4e192c2 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Pipe.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Pipe.scala @@ -3,27 +3,44 @@ */ package akka.stream.scaladsl +import akka.stream.impl.Ast import akka.stream.impl.Ast.AstNode +import org.reactivestreams.Processor import scala.annotation.unchecked.uncheckedVariance +import scala.collection.immutable import scala.language.{ existentials, higherKinds } import akka.stream.FlowMaterializer private[akka] object Pipe { - private val emptyInstance = Pipe[Any, Any](ops = Nil) + private val emptyInstance = Pipe[Any, Any](ops = Nil, keys = Nil) def empty[T]: Pipe[T, T] = emptyInstance.asInstanceOf[Pipe[T, T]] + + // FIXME #16376 should probably be replaced with an ActorFlowProcessor similar to ActorFlowSource/Sink + private[stream] def apply[In, Out](p: () ⇒ Processor[In, Out]): Pipe[In, Out] = + Pipe(List(Ast.DirectProcessor(() ⇒ p().asInstanceOf[Processor[Any, Any]])), Nil) + + // FIXME #16376 should probably be replaced with an ActorFlowProcessor similar to ActorFlowSource/Sink + private[stream] def apply[In, Out](key: Key)(p: () ⇒ (Processor[In, Out], Any)): Pipe[In, Out] = + Pipe(List(Ast.DirectProcessorWithKey(() ⇒ p().asInstanceOf[(Processor[Any, Any], Any)], key)), Nil) + + private[stream] def apply[In, Out](source: SourcePipe[_]): Pipe[In, Out] = + Pipe(source.ops, source.keys) + + private[stream] def apply[In, Out](sink: SinkPipe[_]): Pipe[In, Out] = + Pipe(sink.ops, sink.keys) } /** * Flow with one open input and one open output. */ -private[akka] final case class Pipe[-In, +Out](ops: List[AstNode]) extends Flow[In, Out] { +private[akka] final case class Pipe[-In, +Out](ops: List[AstNode], keys: List[Key]) extends Flow[In, Out] { override type Repr[+O] = Pipe[In @uncheckedVariance, O] - override private[scaladsl] def andThen[U](op: AstNode): Repr[U] = this.copy(ops = op :: ops) // FIXME raw addition of AstNodes + override private[scaladsl] def andThen[U](op: AstNode): Repr[U] = Pipe(ops = op :: ops, keys) // FIXME raw addition of AstNodes - private[stream] def withSink(out: Sink[Out]): SinkPipe[In] = SinkPipe(out, ops) + private[stream] def withSink(out: Sink[Out]): SinkPipe[In] = SinkPipe(out, ops, keys) - private[stream] def withSource(in: Source[In]): SourcePipe[Out] = SourcePipe(in, ops) + private[stream] def withSource(in: Source[In]): SourcePipe[Out] = SourcePipe(in, ops, keys) override def via[T](flow: Flow[Out, T]): Flow[In, T] = flow match { case p: Pipe[Out, T] ⇒ this.appendPipe(p) @@ -37,71 +54,59 @@ private[akka] final case class Pipe[-In, +Out](ops: List[AstNode]) extends Flow[ case d: Sink[Out] ⇒ this.withSink(d) } - private[stream] def appendPipe[T](pipe: Pipe[Out, T]): Pipe[In, T] = Pipe(pipe.ops ++: ops) // FIXME raw addition of AstNodes + override def join(flow: Flow[Out, In]): RunnableFlow = flow match { + case p: Pipe[Out, In] ⇒ GraphFlow(this).join(p) + case gf: GraphFlow[Out, _, _, In] ⇒ gf.join(this) + case x ⇒ FlowGraphInternal.throwUnsupportedValue(x) + } + + override def withKey(key: Key): Pipe[In, Out] = Pipe(ops, keys :+ key) + + private[stream] def appendPipe[T](pipe: Pipe[Out, T]): Pipe[In, T] = Pipe(pipe.ops ::: ops, keys ::: pipe.keys) // FIXME raw addition of AstNodes } /** * Pipe with open input and attached output. Can be used as a `Subscriber`. */ -private[stream] final case class SinkPipe[-In](output: Sink[_], ops: List[AstNode]) extends Sink[In] { +private[stream] final case class SinkPipe[-In](output: Sink[_], ops: List[AstNode], keys: List[Key]) extends Sink[In] { - private[stream] def withSource(in: Source[In]): RunnablePipe = RunnablePipe(in, output, ops) + private[stream] def withSource(in: Source[In]): RunnablePipe = RunnablePipe(in, output, ops, keys) - private[stream] def prependPipe[T](pipe: Pipe[T, In]): SinkPipe[T] = SinkPipe(output, ops ::: pipe.ops) // FIXME raw addition of AstNodes + private[stream] def prependPipe[T](pipe: Pipe[T, In]): SinkPipe[T] = SinkPipe(output, ops ::: pipe.ops, keys ::: pipe.keys) // FIXME raw addition of AstNodes } /** * Pipe with open output and attached input. Can be used as a `Publisher`. */ -private[stream] final case class SourcePipe[+Out](input: Source[_], ops: List[AstNode]) extends Source[Out] { +private[stream] final case class SourcePipe[+Out](input: Source[_], ops: List[AstNode], keys: List[Key]) extends Source[Out] { override type Repr[+O] = SourcePipe[O] - override private[scaladsl] def andThen[U](op: AstNode): Repr[U] = SourcePipe(input, op :: ops) // FIXME raw addition of AstNodes + override private[scaladsl] def andThen[U](op: AstNode): Repr[U] = SourcePipe(input, op :: ops, keys) // FIXME raw addition of AstNodes - private[stream] def withSink(out: Sink[Out]): RunnablePipe = RunnablePipe(input, out, ops) + private[stream] def withSink(out: Sink[Out]): RunnablePipe = RunnablePipe(input, out, ops, keys) - private[stream] def appendPipe[T](pipe: Pipe[Out, T]): SourcePipe[T] = SourcePipe(input, pipe.ops ++: ops) // FIXME raw addition of AstNodes + private[stream] def appendPipe[T](pipe: Pipe[Out, T]): SourcePipe[T] = SourcePipe(input, pipe.ops ::: ops, keys ::: pipe.keys) // FIXME raw addition of AstNodes override def via[T](flow: Flow[Out, T]): Source[T] = flow match { - case p: Pipe[Out, T] ⇒ appendPipe(p) + case p: Pipe[Out, T] ⇒ this.appendPipe(p) case g: GraphFlow[Out, _, _, T] ⇒ g.prepend(this) case x ⇒ FlowGraphInternal.throwUnsupportedValue(x) } override def to(sink: Sink[Out]): RunnableFlow = sink match { - case sp: SinkPipe[Out] ⇒ RunnablePipe(input, sp.output, sp.ops ++: ops) // FIXME raw addition of AstNodes + case sp: SinkPipe[Out] ⇒ RunnablePipe(input, sp.output, sp.ops ::: ops, keys ::: sp.keys) // FIXME raw addition of AstNodes case g: GraphSink[Out, _] ⇒ g.prepend(this) case d: Sink[Out] ⇒ this.withSink(d) } + + override def withKey(key: Key): SourcePipe[Out] = SourcePipe(input, ops, keys :+ key) } /** * Pipe with attached input and output, can be executed. */ -private[stream] final case class RunnablePipe(input: Source[_], output: Sink[_], ops: List[AstNode]) extends RunnableFlow { +private[stream] final case class RunnablePipe(input: Source[_], output: Sink[_], ops: List[AstNode], keys: List[Key]) 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 - * `Source` input or `Sink` output. - */ -private[stream] class MaterializedPipe(sourceKey: AnyRef, matSource: Any, sinkKey: AnyRef, matSink: Any) extends MaterializedMap { - override def get(key: Source[_]): key.MaterializedType = - key match { - case _: KeyedSource[_] ⇒ - if (key == sourceKey) matSource.asInstanceOf[key.MaterializedType] - else throw new IllegalArgumentException(s"Source key [$key] doesn't match the source [$sourceKey] of this flow") - case _ ⇒ ().asInstanceOf[key.MaterializedType] - } - - override def get(key: Sink[_]): key.MaterializedType = - key match { - case _: KeyedSink[_] ⇒ - if (key == sinkKey) matSink.asInstanceOf[key.MaterializedType] - else throw new IllegalArgumentException(s"Sink key [$key] doesn't match the sink [$sinkKey] of this flow") - case _ ⇒ ().asInstanceOf[key.MaterializedType] - } + materializer.materialize(input, output, ops, keys) } diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala index f9774426ab..9aacf2c05d 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Sink.scala @@ -42,7 +42,7 @@ object Sink { * a [[FlowGraphBuilder]] and returns the `UndefinedSource`. */ def apply[T](graph: PartialFlowGraph)(block: FlowGraphBuilder ⇒ UndefinedSource[T]): Sink[T] = - createSinkFromBuilder(new FlowGraphBuilder(graph.graph), block) + createSinkFromBuilder(new FlowGraphBuilder(graph), block) private def createSinkFromBuilder[T](builder: FlowGraphBuilder, block: FlowGraphBuilder ⇒ UndefinedSource[T]): Sink[T] = { val in = block(builder) diff --git a/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala b/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala index cdd0003161..f0f8680d15 100644 --- a/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala +++ b/akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala @@ -73,6 +73,12 @@ trait Source[+Out] extends FlowOps[Out] { */ def ++[Out2 >: Out](second: Source[Out2]): Source[Out2] = concat(second) + /** + * Add a key that will have a value available after materialization. + * The key can only use other keys if they have been added to the source + * before this key. This also includes the keyed source if applicable. + */ + def withKey(key: Key): Source[Out] } object Source { @@ -139,7 +145,7 @@ object Source { * a [[FlowGraphBuilder]] and returns the `UndefinedSink`. */ def apply[T](graph: PartialFlowGraph)(block: FlowGraphBuilder ⇒ UndefinedSink[T]): Source[T] = - createSourceFromBuilder(new FlowGraphBuilder(graph.graph), block) + createSourceFromBuilder(new FlowGraphBuilder(graph), block) private def createSourceFromBuilder[T](builder: FlowGraphBuilder, block: FlowGraphBuilder ⇒ UndefinedSink[T]): Source[T] = { val out = block(builder)