!str #15121 MaterializerSettings now uses a config object
Resolves #15121
This commit is contained in:
parent
a25d0fcde9
commit
cc4fd5ca2c
76 changed files with 597 additions and 476 deletions
|
|
@ -4,11 +4,12 @@
|
||||||
|
|
||||||
package docs.http
|
package docs.http
|
||||||
|
|
||||||
import akka.stream.testkit.AkkaSpec
|
|
||||||
import akka.actor.ActorSystem
|
import akka.actor.ActorSystem
|
||||||
import akka.util.Timeout
|
|
||||||
import scala.concurrent.duration._
|
|
||||||
import akka.http.model._
|
import akka.http.model._
|
||||||
|
import akka.stream.testkit.AkkaSpec
|
||||||
|
import akka.util.Timeout
|
||||||
|
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
class HttpServerExampleSpec
|
class HttpServerExampleSpec
|
||||||
extends AkkaSpec("akka.actor.default-mailbox.mailbox-type = akka.dispatch.UnboundedMailbox") {
|
extends AkkaSpec("akka.actor.default-mailbox.mailbox-type = akka.dispatch.UnboundedMailbox") {
|
||||||
|
|
@ -16,17 +17,15 @@ class HttpServerExampleSpec
|
||||||
|
|
||||||
"binding example" in {
|
"binding example" in {
|
||||||
//#bind-example
|
//#bind-example
|
||||||
import akka.pattern.ask
|
|
||||||
|
|
||||||
import akka.io.IO
|
|
||||||
import akka.http.Http
|
import akka.http.Http
|
||||||
|
import akka.io.IO
|
||||||
|
import akka.pattern.ask
|
||||||
|
import akka.stream.FlowMaterializer
|
||||||
import akka.stream.scaladsl.Flow
|
import akka.stream.scaladsl.Flow
|
||||||
import akka.stream.{ MaterializerSettings, FlowMaterializer }
|
|
||||||
|
|
||||||
implicit val system = ActorSystem()
|
implicit val system = ActorSystem()
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings())
|
implicit val materializer = FlowMaterializer()
|
||||||
implicit val askTimeout: Timeout = 500.millis
|
implicit val askTimeout: Timeout = 500.millis
|
||||||
|
|
||||||
val bindingFuture = IO(Http) ? Http.Bind(interface = "localhost", port = 8080)
|
val bindingFuture = IO(Http) ? Http.Bind(interface = "localhost", port = 8080)
|
||||||
|
|
@ -42,23 +41,21 @@ class HttpServerExampleSpec
|
||||||
//#bind-example
|
//#bind-example
|
||||||
}
|
}
|
||||||
"full-server-example" in {
|
"full-server-example" in {
|
||||||
import akka.pattern.ask
|
|
||||||
|
|
||||||
import akka.io.IO
|
|
||||||
import akka.http.Http
|
import akka.http.Http
|
||||||
|
import akka.io.IO
|
||||||
|
import akka.pattern.ask
|
||||||
|
import akka.stream.FlowMaterializer
|
||||||
import akka.stream.scaladsl.Flow
|
import akka.stream.scaladsl.Flow
|
||||||
import akka.stream.{ MaterializerSettings, FlowMaterializer }
|
|
||||||
|
|
||||||
implicit val system = ActorSystem()
|
implicit val system = ActorSystem()
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings())
|
implicit val materializer = FlowMaterializer()
|
||||||
implicit val askTimeout: Timeout = 500.millis
|
implicit val askTimeout: Timeout = 500.millis
|
||||||
|
|
||||||
val bindingFuture = IO(Http) ? Http.Bind(interface = "localhost", port = 8080)
|
val bindingFuture = IO(Http) ? Http.Bind(interface = "localhost", port = 8080)
|
||||||
|
|
||||||
//#full-server-example
|
//#full-server-example
|
||||||
import HttpMethods._
|
import akka.http.model.HttpMethods._
|
||||||
|
|
||||||
val requestHandler: HttpRequest ⇒ HttpResponse = {
|
val requestHandler: HttpRequest ⇒ HttpResponse = {
|
||||||
case HttpRequest(GET, Uri.Path("/"), _, _, _) ⇒
|
case HttpRequest(GET, Uri.Path("/"), _, _, _) ⇒
|
||||||
|
|
|
||||||
|
|
@ -6,6 +6,7 @@ package akka.http.model.japi;
|
||||||
|
|
||||||
import akka.actor.ActorSystem;
|
import akka.actor.ActorSystem;
|
||||||
import akka.http.HttpExt;
|
import akka.http.HttpExt;
|
||||||
|
import akka.stream.MaterializerSettings;
|
||||||
|
|
||||||
public final class Http {
|
public final class Http {
|
||||||
private Http(){}
|
private Http(){}
|
||||||
|
|
@ -18,4 +19,8 @@ public final class Http {
|
||||||
public static Object bind(String host, int port) {
|
public static Object bind(String host, int port) {
|
||||||
return Accessors$.MODULE$.Bind(host, 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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -31,13 +31,13 @@ object Http extends ExtensionKey[HttpExt] {
|
||||||
localAddress: Option[InetSocketAddress],
|
localAddress: Option[InetSocketAddress],
|
||||||
options: immutable.Traversable[Inet.SocketOption],
|
options: immutable.Traversable[Inet.SocketOption],
|
||||||
settings: Option[ClientConnectionSettings],
|
settings: Option[ClientConnectionSettings],
|
||||||
materializerSettings: MaterializerSettings) extends SetupOutgoingChannel
|
materializerSettings: Option[MaterializerSettings]) extends SetupOutgoingChannel
|
||||||
object Connect {
|
object Connect {
|
||||||
def apply(host: String, port: Int = 80,
|
def apply(host: String, port: Int = 80,
|
||||||
localAddress: Option[InetSocketAddress] = None,
|
localAddress: Option[InetSocketAddress] = None,
|
||||||
options: immutable.Traversable[Inet.SocketOption] = Nil,
|
options: immutable.Traversable[Inet.SocketOption] = Nil,
|
||||||
settings: Option[ClientConnectionSettings] = None,
|
settings: Option[ClientConnectionSettings] = None,
|
||||||
materializerSettings: MaterializerSettings = MaterializerSettings()): Connect =
|
materializerSettings: Option[MaterializerSettings] = None): Connect =
|
||||||
apply(new InetSocketAddress(host, port), localAddress, options, settings, materializerSettings)
|
apply(new InetSocketAddress(host, port), localAddress, options, settings, materializerSettings)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -100,12 +100,12 @@ object Http extends ExtensionKey[HttpExt] {
|
||||||
backlog: Int,
|
backlog: Int,
|
||||||
options: immutable.Traversable[Inet.SocketOption],
|
options: immutable.Traversable[Inet.SocketOption],
|
||||||
serverSettings: Option[ServerSettings],
|
serverSettings: Option[ServerSettings],
|
||||||
materializerSettings: MaterializerSettings)
|
materializerSettings: Option[MaterializerSettings])
|
||||||
object Bind {
|
object Bind {
|
||||||
def apply(interface: String, port: Int = 80, backlog: Int = 100,
|
def apply(interface: String, port: Int = 80, backlog: Int = 100,
|
||||||
options: immutable.Traversable[Inet.SocketOption] = Nil,
|
options: immutable.Traversable[Inet.SocketOption] = Nil,
|
||||||
serverSettings: Option[ServerSettings] = None,
|
serverSettings: Option[ServerSettings] = None,
|
||||||
materializerSettings: MaterializerSettings = MaterializerSettings()): Bind =
|
materializerSettings: Option[MaterializerSettings] = None): Bind =
|
||||||
apply(new InetSocketAddress(interface, port), backlog, options, serverSettings, materializerSettings)
|
apply(new InetSocketAddress(interface, port), backlog, options, serverSettings, materializerSettings)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -4,17 +4,18 @@
|
||||||
|
|
||||||
package akka.http
|
package akka.http
|
||||||
|
|
||||||
import scala.util.{ Failure, Success }
|
|
||||||
import scala.concurrent.duration._
|
|
||||||
import akka.io.IO
|
|
||||||
import akka.util.Timeout
|
|
||||||
import akka.stream.io.StreamTcp
|
|
||||||
import akka.stream.FlowMaterializer
|
|
||||||
import akka.http.client._
|
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
|
import akka.http.client._
|
||||||
import akka.http.server.{ HttpServerPipeline, ServerSettings }
|
import akka.http.server.{ HttpServerPipeline, ServerSettings }
|
||||||
|
import akka.io.IO
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
|
import akka.stream.FlowMaterializer
|
||||||
|
import akka.stream.io.StreamTcp
|
||||||
import akka.stream.scaladsl.Flow
|
import akka.stream.scaladsl.Flow
|
||||||
|
import akka.util.Timeout
|
||||||
|
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
import scala.util.{ Failure, Success }
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -27,18 +28,19 @@ private[http] class HttpManager(httpSettings: HttpExt#Settings) extends Actor wi
|
||||||
private[this] var clientPipelines = Map.empty[ClientConnectionSettings, HttpClientPipeline]
|
private[this] var clientPipelines = Map.empty[ClientConnectionSettings, HttpClientPipeline]
|
||||||
|
|
||||||
def receive = {
|
def receive = {
|
||||||
case connect @ Http.Connect(remoteAddress, localAddress, options, settings, materializerSettings) ⇒
|
case connect @ Http.Connect(remoteAddress, localAddress, options, clientConnectionSettings, materializerSettings) ⇒
|
||||||
log.debug("Attempting connection to {}", remoteAddress)
|
log.debug("Attempting connection to {}", remoteAddress)
|
||||||
val commander = sender()
|
val commander = sender()
|
||||||
val effectiveSettings = ClientConnectionSettings(settings)
|
val effectiveSettings = ClientConnectionSettings(clientConnectionSettings)
|
||||||
val tcpConnect = StreamTcp.Connect(materializerSettings, remoteAddress, localAddress, options,
|
|
||||||
|
val tcpConnect = StreamTcp.Connect(remoteAddress, localAddress, materializerSettings, options,
|
||||||
effectiveSettings.connectingTimeout, effectiveSettings.idleTimeout)
|
effectiveSettings.connectingTimeout, effectiveSettings.idleTimeout)
|
||||||
val askTimeout = Timeout(effectiveSettings.connectingTimeout + 5.seconds) // FIXME: how can we improve this?
|
val askTimeout = Timeout(effectiveSettings.connectingTimeout + 5.seconds) // FIXME: how can we improve this?
|
||||||
val tcpConnectionFuture = IO(StreamTcp)(context.system).ask(tcpConnect)(askTimeout)
|
val tcpConnectionFuture = IO(StreamTcp)(context.system).ask(tcpConnect)(askTimeout)
|
||||||
tcpConnectionFuture onComplete {
|
tcpConnectionFuture onComplete {
|
||||||
case Success(tcpConn: StreamTcp.OutgoingTcpConnection) ⇒
|
case Success(tcpConn: StreamTcp.OutgoingTcpConnection) ⇒
|
||||||
val pipeline = clientPipelines.getOrElse(effectiveSettings, {
|
val pipeline = clientPipelines.getOrElse(effectiveSettings, {
|
||||||
val pl = new HttpClientPipeline(effectiveSettings, FlowMaterializer(materializerSettings), log)
|
val pl = new HttpClientPipeline(effectiveSettings, FlowMaterializer(), log)
|
||||||
clientPipelines = clientPipelines.updated(effectiveSettings, pl)
|
clientPipelines = clientPipelines.updated(effectiveSettings, pl)
|
||||||
pl
|
pl
|
||||||
})
|
})
|
||||||
|
|
@ -51,17 +53,17 @@ private[http] class HttpManager(httpSettings: HttpExt#Settings) extends Actor wi
|
||||||
case x ⇒ throw new IllegalStateException("Unexpected response to `Connect` from StreamTcp: " + x)
|
case x ⇒ throw new IllegalStateException("Unexpected response to `Connect` from StreamTcp: " + x)
|
||||||
}
|
}
|
||||||
|
|
||||||
case Http.Bind(endpoint, backlog, options, settings, materializerSettings) ⇒
|
case Http.Bind(endpoint, backlog, options, serverSettings, materializerSettings) ⇒
|
||||||
log.debug("Binding to {}", endpoint)
|
log.debug("Binding to {}", endpoint)
|
||||||
val commander = sender()
|
val commander = sender()
|
||||||
val effectiveSettings = ServerSettings(settings)
|
val effectiveSettings = ServerSettings(serverSettings)
|
||||||
val tcpBind = StreamTcp.Bind(materializerSettings, endpoint, backlog, options)
|
val tcpBind = StreamTcp.Bind(endpoint, materializerSettings, backlog, options)
|
||||||
val askTimeout = Timeout(effectiveSettings.bindTimeout + 5.seconds) // FIXME: how can we improve this?
|
val askTimeout = Timeout(effectiveSettings.bindTimeout + 5.seconds) // FIXME: how can we improve this?
|
||||||
val tcpServerBindingFuture = IO(StreamTcp)(context.system).ask(tcpBind)(askTimeout)
|
val tcpServerBindingFuture = IO(StreamTcp)(context.system).ask(tcpBind)(askTimeout)
|
||||||
tcpServerBindingFuture onComplete {
|
tcpServerBindingFuture onComplete {
|
||||||
case Success(StreamTcp.TcpServerBinding(localAddress, connectionStream)) ⇒
|
case Success(StreamTcp.TcpServerBinding(localAddress, connectionStream)) ⇒
|
||||||
log.info("Bound to {}", endpoint)
|
log.info("Bound to {}", endpoint)
|
||||||
implicit val materializer = FlowMaterializer(materializerSettings)
|
implicit val materializer = FlowMaterializer()
|
||||||
val httpServerPipeline = new HttpServerPipeline(effectiveSettings, materializer, log)
|
val httpServerPipeline = new HttpServerPipeline(effectiveSettings, materializer, log)
|
||||||
val httpConnectionStream = Flow(connectionStream)
|
val httpConnectionStream = Flow(connectionStream)
|
||||||
.map(httpServerPipeline)
|
.map(httpServerPipeline)
|
||||||
|
|
|
||||||
|
|
@ -4,9 +4,8 @@
|
||||||
|
|
||||||
package akka.http.model.japi
|
package akka.http.model.japi
|
||||||
|
|
||||||
import akka.http.{ HttpExt, model }
|
import akka.http.model
|
||||||
import akka.actor.ActorSystem
|
import akka.stream.MaterializerSettings
|
||||||
import java.net.InetSocketAddress
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -22,5 +21,10 @@ private[http] object Accessors {
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
private[http] def Uri(uri: model.Uri): Uri = JavaUri(uri)
|
private[http] def Uri(uri: model.Uri): Uri = JavaUri(uri)
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
private[http] def Bind(host: String, port: Int): AnyRef = akka.http.Http.Bind(host, port)
|
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))
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -24,8 +24,7 @@ public abstract class JavaTestServer {
|
||||||
public static void main(String[] args) throws IOException, InterruptedException {
|
public static void main(String[] args) throws IOException, InterruptedException {
|
||||||
ActorSystem system = ActorSystem.create();
|
ActorSystem system = ActorSystem.create();
|
||||||
|
|
||||||
MaterializerSettings settings = MaterializerSettings.create();
|
final FlowMaterializer materializer = FlowMaterializer.create(system);
|
||||||
final FlowMaterializer materializer = FlowMaterializer.create(settings, system);
|
|
||||||
|
|
||||||
ActorRef httpManager = Http.get(system).manager();
|
ActorRef httpManager = Http.get(system).manager();
|
||||||
Future<Object> binding = ask(httpManager, Http.bind("localhost", 8080), 1000);
|
Future<Object> binding = ask(httpManager, Http.bind("localhost", 8080), 1000);
|
||||||
|
|
|
||||||
|
|
@ -4,29 +4,31 @@
|
||||||
|
|
||||||
package akka.http
|
package akka.http
|
||||||
|
|
||||||
|
import java.io.{ BufferedReader, BufferedWriter, InputStreamReader, OutputStreamWriter }
|
||||||
import java.net.Socket
|
import java.net.Socket
|
||||||
import java.io.{ InputStreamReader, BufferedReader, OutputStreamWriter, BufferedWriter }
|
|
||||||
import akka.stream.testkit.StreamTestKit.{ SubscriberProbe, PublisherProbe }
|
|
||||||
import com.typesafe.config.{ ConfigFactory, Config }
|
|
||||||
import scala.annotation.tailrec
|
|
||||||
import scala.concurrent.duration._
|
|
||||||
import scala.concurrent.Await
|
|
||||||
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
|
|
||||||
import akka.actor.ActorSystem
|
import akka.actor.ActorSystem
|
||||||
import akka.testkit.TestProbe
|
|
||||||
import akka.io.IO
|
|
||||||
import akka.stream.{ FlowMaterializer, MaterializerSettings }
|
|
||||||
import akka.stream.testkit.StreamTestKit
|
|
||||||
import akka.stream.impl.SynchronousPublisherFromIterable
|
|
||||||
import akka.stream.scaladsl.Flow
|
|
||||||
import akka.http.server.ServerSettings
|
|
||||||
import akka.http.client.ClientConnectionSettings
|
import akka.http.client.ClientConnectionSettings
|
||||||
import akka.http.model._
|
import akka.http.model._
|
||||||
import akka.http.util._
|
|
||||||
import headers._
|
|
||||||
import HttpMethods._
|
|
||||||
import HttpEntity._
|
import HttpEntity._
|
||||||
|
import HttpMethods._
|
||||||
import TestUtils._
|
import TestUtils._
|
||||||
|
import akka.http.server.ServerSettings
|
||||||
|
import akka.http.util._
|
||||||
|
import akka.io.IO
|
||||||
|
import akka.stream.{ MaterializerSettings, FlowMaterializer }
|
||||||
|
import akka.stream.impl.SynchronousPublisherFromIterable
|
||||||
|
import akka.stream.scaladsl.Flow
|
||||||
|
import akka.stream.testkit.StreamTestKit
|
||||||
|
import akka.stream.testkit.StreamTestKit.{ PublisherProbe, SubscriberProbe }
|
||||||
|
import akka.testkit.TestProbe
|
||||||
|
import com.typesafe.config.{ Config, ConfigFactory }
|
||||||
|
import headers._
|
||||||
|
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
|
||||||
|
|
||||||
|
import scala.annotation.tailrec
|
||||||
|
import scala.concurrent.Await
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
||||||
val testConf: Config = ConfigFactory.parseString("""
|
val testConf: Config = ConfigFactory.parseString("""
|
||||||
|
|
@ -35,7 +37,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
||||||
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
|
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
|
|
||||||
val materializerSettings = MaterializerSettings(dispatcher = "akka.test.stream-dispatcher")
|
val materializerSettings = MaterializerSettings(system)
|
||||||
val materializer = FlowMaterializer(materializerSettings)
|
val materializer = FlowMaterializer(materializerSettings)
|
||||||
|
|
||||||
"The server-side HTTP infrastructure" should {
|
"The server-side HTTP infrastructure" should {
|
||||||
|
|
@ -43,7 +45,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
||||||
"properly bind and unbind a server" in {
|
"properly bind and unbind a server" in {
|
||||||
val (hostname, port) = temporaryServerHostnameAndPort()
|
val (hostname, port) = temporaryServerHostnameAndPort()
|
||||||
val commander = TestProbe()
|
val commander = TestProbe()
|
||||||
commander.send(IO(Http), Http.Bind(hostname, port, materializerSettings = materializerSettings))
|
commander.send(IO(Http), Http.Bind(hostname, port, materializerSettings = Some(materializerSettings)))
|
||||||
|
|
||||||
val Http.ServerBinding(localAddress, connectionStream) = commander.expectMsgType[Http.ServerBinding]
|
val Http.ServerBinding(localAddress, connectionStream) = commander.expectMsgType[Http.ServerBinding]
|
||||||
localAddress.getHostName shouldEqual hostname
|
localAddress.getHostName shouldEqual hostname
|
||||||
|
|
@ -118,7 +120,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
||||||
val connectionStream: SubscriberProbe[Http.IncomingConnection] = {
|
val connectionStream: SubscriberProbe[Http.IncomingConnection] = {
|
||||||
val commander = TestProbe()
|
val commander = TestProbe()
|
||||||
val settings = configOverrides.toOption.map(ServerSettings.apply)
|
val settings = configOverrides.toOption.map(ServerSettings.apply)
|
||||||
commander.send(IO(Http), Http.Bind(hostname, port, serverSettings = settings, materializerSettings = materializerSettings))
|
commander.send(IO(Http), Http.Bind(hostname, port, serverSettings = settings, materializerSettings = Some(materializerSettings)))
|
||||||
val probe = StreamTestKit.SubscriberProbe[Http.IncomingConnection]
|
val probe = StreamTestKit.SubscriberProbe[Http.IncomingConnection]
|
||||||
commander.expectMsgType[Http.ServerBinding].connectionStream.subscribe(probe)
|
commander.expectMsgType[Http.ServerBinding].connectionStream.subscribe(probe)
|
||||||
probe
|
probe
|
||||||
|
|
@ -127,7 +129,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
||||||
|
|
||||||
def openNewClientConnection[T](settings: Option[ClientConnectionSettings] = None): (PublisherProbe[(HttpRequest, T)], SubscriberProbe[(HttpResponse, T)]) = {
|
def openNewClientConnection[T](settings: Option[ClientConnectionSettings] = None): (PublisherProbe[(HttpRequest, T)], SubscriberProbe[(HttpResponse, T)]) = {
|
||||||
val commander = TestProbe()
|
val commander = TestProbe()
|
||||||
commander.send(IO(Http), Http.Connect(hostname, port, settings = settings, materializerSettings = materializerSettings))
|
commander.send(IO(Http), Http.Connect(hostname, port, settings = settings, materializerSettings = Some(materializerSettings)))
|
||||||
val connection = commander.expectMsgType[Http.OutgoingConnection]
|
val connection = commander.expectMsgType[Http.OutgoingConnection]
|
||||||
connection.remoteAddress.getPort shouldEqual port
|
connection.remoteAddress.getPort shouldEqual port
|
||||||
connection.remoteAddress.getHostName shouldEqual hostname
|
connection.remoteAddress.getHostName shouldEqual hostname
|
||||||
|
|
|
||||||
|
|
@ -4,18 +4,19 @@
|
||||||
|
|
||||||
package akka.http
|
package akka.http
|
||||||
|
|
||||||
import com.typesafe.config.{ ConfigFactory, Config }
|
|
||||||
import scala.concurrent.Future
|
|
||||||
import scala.util.{ Failure, Success }
|
|
||||||
import scala.concurrent.duration._
|
|
||||||
import akka.util.Timeout
|
|
||||||
import akka.stream.{ MaterializerSettings, FlowMaterializer }
|
|
||||||
import akka.stream.scaladsl.Flow
|
|
||||||
import akka.io.IO
|
|
||||||
import akka.actor.ActorSystem
|
import akka.actor.ActorSystem
|
||||||
import akka.pattern.ask
|
import akka.http.model.HttpMethods._
|
||||||
import akka.http.model._
|
import akka.http.model._
|
||||||
import HttpMethods._
|
import akka.io.IO
|
||||||
|
import akka.pattern.ask
|
||||||
|
import akka.stream.FlowMaterializer
|
||||||
|
import akka.stream.scaladsl.Flow
|
||||||
|
import akka.util.Timeout
|
||||||
|
import com.typesafe.config.{ Config, ConfigFactory }
|
||||||
|
|
||||||
|
import scala.concurrent.Future
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
import scala.util.{ Failure, Success }
|
||||||
|
|
||||||
object TestClient extends App {
|
object TestClient extends App {
|
||||||
val testConf: Config = ConfigFactory.parseString("""
|
val testConf: Config = ConfigFactory.parseString("""
|
||||||
|
|
@ -23,9 +24,9 @@ object TestClient extends App {
|
||||||
akka.log-dead-letters = off
|
akka.log-dead-letters = off
|
||||||
""")
|
""")
|
||||||
implicit val system = ActorSystem("ServerTest", testConf)
|
implicit val system = ActorSystem("ServerTest", testConf)
|
||||||
import system.dispatcher
|
import akka.http.TestClient.system.dispatcher
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings())
|
implicit val materializer = FlowMaterializer()
|
||||||
implicit val askTimeout: Timeout = 500.millis
|
implicit val askTimeout: Timeout = 500.millis
|
||||||
val host = "spray.io"
|
val host = "spray.io"
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -30,7 +30,7 @@ object TestServer extends App {
|
||||||
case _: HttpRequest ⇒ HttpResponse(404, entity = "Unknown resource!")
|
case _: HttpRequest ⇒ HttpResponse(404, entity = "Unknown resource!")
|
||||||
}
|
}
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings())
|
implicit val materializer = FlowMaterializer()
|
||||||
|
|
||||||
implicit val askTimeout: Timeout = 500.millis
|
implicit val askTimeout: Timeout = 500.millis
|
||||||
val bindingFuture = IO(Http) ? Http.Bind(interface = "localhost", port = 8080)
|
val bindingFuture = IO(Http) ? Http.Bind(interface = "localhost", port = 8080)
|
||||||
|
|
|
||||||
|
|
@ -31,7 +31,7 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
|
||||||
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
|
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
|
|
||||||
val materializer = FlowMaterializer(MaterializerSettings())
|
val materializer = FlowMaterializer()
|
||||||
override def afterAll() = system.shutdown()
|
override def afterAll() = system.shutdown()
|
||||||
|
|
||||||
"HttpEntity" - {
|
"HttpEntity" - {
|
||||||
|
|
|
||||||
|
|
@ -37,7 +37,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
|
|
||||||
val BOLT = HttpMethods.register(HttpMethod.custom("BOLT", safe = false, idempotent = true, entityAccepted = true))
|
val BOLT = HttpMethods.register(HttpMethod.custom("BOLT", safe = false, idempotent = true, entityAccepted = true))
|
||||||
val materializer = FlowMaterializer(MaterializerSettings())
|
val materializer = FlowMaterializer()
|
||||||
|
|
||||||
"The request parsing logic should" - {
|
"The request parsing logic should" - {
|
||||||
"properly parse a request" - {
|
"properly parse a request" - {
|
||||||
|
|
|
||||||
|
|
@ -34,7 +34,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
|
||||||
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
|
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
|
|
||||||
val materializer = FlowMaterializer(MaterializerSettings())
|
val materializer = FlowMaterializer()
|
||||||
val ServerOnTheMove = StatusCodes.registerCustom(331, "Server on the move")
|
val ServerOnTheMove = StatusCodes.registerCustom(331, "Server on the move")
|
||||||
|
|
||||||
"The response parsing logic should" - {
|
"The response parsing logic should" - {
|
||||||
|
|
|
||||||
|
|
@ -28,7 +28,7 @@ class RequestRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll
|
||||||
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
|
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
|
|
||||||
val materializer = FlowMaterializer(MaterializerSettings())
|
val materializer = FlowMaterializer()
|
||||||
|
|
||||||
"The request preparation logic should" - {
|
"The request preparation logic should" - {
|
||||||
"properly render an unchunked" - {
|
"properly render an unchunked" - {
|
||||||
|
|
|
||||||
|
|
@ -28,7 +28,7 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
|
|
||||||
val ServerOnTheMove = StatusCodes.registerCustom(330, "Server on the move")
|
val ServerOnTheMove = StatusCodes.registerCustom(330, "Server on the move")
|
||||||
val materializer = FlowMaterializer(MaterializerSettings())
|
val materializer = FlowMaterializer()
|
||||||
|
|
||||||
"The response preparation logic should properly render" - {
|
"The response preparation logic should properly render" - {
|
||||||
"a response with no body" - {
|
"a response with no body" - {
|
||||||
|
|
|
||||||
|
|
@ -5,25 +5,21 @@
|
||||||
package akka.http
|
package akka.http
|
||||||
package server
|
package server
|
||||||
|
|
||||||
import akka.http.model.HttpEntity.{ LastChunk, Chunk, ChunkStreamPart }
|
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
|
||||||
|
|
||||||
import akka.event.NoLogging
|
import akka.event.NoLogging
|
||||||
import akka.http.model.headers.Host
|
import akka.http.model.HttpEntity.{ Chunk, ChunkStreamPart, LastChunk }
|
||||||
import akka.http.model._
|
import akka.http.model._
|
||||||
|
import akka.http.model.headers.Host
|
||||||
import akka.http.util._
|
import akka.http.util._
|
||||||
|
import akka.stream.FlowMaterializer
|
||||||
import akka.stream.io.StreamTcp
|
import akka.stream.io.StreamTcp
|
||||||
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
|
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
|
||||||
import akka.stream.{ FlowMaterializer, MaterializerSettings }
|
|
||||||
import akka.util.ByteString
|
import akka.util.ByteString
|
||||||
import org.scalatest._
|
import org.scalatest._
|
||||||
|
|
||||||
import scala.concurrent.duration.FiniteDuration
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterAll with Inside {
|
class HttpServerPipelineSpec extends AkkaSpec with Matchers with BeforeAndAfterAll with Inside {
|
||||||
val materializerSettings = MaterializerSettings(dispatcher = "akka.test.stream-dispatcher")
|
val materializer = FlowMaterializer()
|
||||||
val materializer = FlowMaterializer(materializerSettings)
|
|
||||||
|
|
||||||
"The server implementation should" should {
|
"The server implementation should" should {
|
||||||
"deliver an empty request as soon as all headers are received" in new TestSetup {
|
"deliver an empty request as soon as all headers are received" in new TestSetup {
|
||||||
|
|
|
||||||
|
|
@ -4,23 +4,27 @@
|
||||||
|
|
||||||
package akka.http
|
package akka.http
|
||||||
|
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
import akka.http.marshalling.{ MultipartMarshallers, ToEntityMarshallers }
|
||||||
|
import akka.http.model._
|
||||||
|
import HttpCharsets._
|
||||||
|
import MediaTypes._
|
||||||
|
import akka.http.util._
|
||||||
|
import akka.stream.{ FlowMaterializer, MaterializerSettings }
|
||||||
|
import headers._
|
||||||
|
import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers }
|
||||||
|
|
||||||
import scala.collection.immutable.ListMap
|
import scala.collection.immutable.ListMap
|
||||||
import scala.concurrent.Await
|
import scala.concurrent.Await
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import org.scalatest.{ BeforeAndAfterAll, Matchers, FreeSpec }
|
|
||||||
import akka.actor.ActorSystem
|
|
||||||
import akka.http.marshalling.{ ToEntityMarshallers, MultipartMarshallers }
|
|
||||||
import akka.stream.{ FlowMaterializer, MaterializerSettings }
|
|
||||||
import akka.http.util._
|
|
||||||
import akka.http.model._
|
|
||||||
import headers._
|
|
||||||
import MediaTypes._
|
|
||||||
import HttpCharsets._
|
|
||||||
|
|
||||||
class MarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll with MultipartMarshallers {
|
class MarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll with MultipartMarshallers {
|
||||||
implicit val system = ActorSystem(getClass.getSimpleName)
|
implicit val system = ActorSystem(getClass.getSimpleName)
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
val materializerSettings = MaterializerSettings(dispatcher = "akka.test.stream-dispatcher")
|
|
||||||
|
val materializerSettings = MaterializerSettings(system)
|
||||||
|
.withDispatcher("akka.test.stream-dispatcher")
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(materializerSettings)
|
implicit val materializer = FlowMaterializer(materializerSettings)
|
||||||
|
|
||||||
"The PredefinedToEntityMarshallers." - {
|
"The PredefinedToEntityMarshallers." - {
|
||||||
|
|
|
||||||
|
|
@ -4,24 +4,29 @@
|
||||||
|
|
||||||
package akka.http
|
package akka.http
|
||||||
|
|
||||||
import scala.xml.NodeSeq
|
|
||||||
import scala.concurrent.duration._
|
|
||||||
import scala.concurrent.{ Future, Await }
|
|
||||||
import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers }
|
|
||||||
import org.scalatest.matchers.Matcher
|
|
||||||
import akka.actor.ActorSystem
|
import akka.actor.ActorSystem
|
||||||
import akka.stream.scaladsl.Flow
|
|
||||||
import akka.stream.{ FlowMaterializer, MaterializerSettings }
|
|
||||||
import akka.http.unmarshalling.Unmarshalling
|
|
||||||
import akka.http.util._
|
|
||||||
import akka.http.model._
|
import akka.http.model._
|
||||||
import MediaTypes._
|
import MediaTypes._
|
||||||
|
import akka.http.unmarshalling.Unmarshalling
|
||||||
|
import akka.http.util._
|
||||||
|
import akka.stream.scaladsl.Flow
|
||||||
|
import akka.stream.{ FlowMaterializer, MaterializerSettings }
|
||||||
import headers._
|
import headers._
|
||||||
|
import org.scalatest.matchers.Matcher
|
||||||
|
import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers }
|
||||||
|
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
import scala.concurrent.{ Await, Future }
|
||||||
|
import scala.xml.NodeSeq
|
||||||
|
|
||||||
class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
|
class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
|
||||||
implicit val system = ActorSystem(getClass.getSimpleName)
|
implicit val system = ActorSystem(getClass.getSimpleName)
|
||||||
val materializerSettings = MaterializerSettings(dispatcher = "akka.test.stream-dispatcher")
|
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
|
|
||||||
|
val materializerSettings = MaterializerSettings(system)
|
||||||
|
.withDispatcher("akka.test.stream-dispatcher")
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(materializerSettings)
|
implicit val materializer = FlowMaterializer(materializerSettings)
|
||||||
|
|
||||||
"The PredefinedFromEntityUnmarshallers." - {
|
"The PredefinedFromEntityUnmarshallers." - {
|
||||||
|
|
|
||||||
27
akka-stream/src/main/resources/reference.conf
Normal file
27
akka-stream/src/main/resources/reference.conf
Normal file
|
|
@ -0,0 +1,27 @@
|
||||||
|
#####################################
|
||||||
|
# Akka Stream Reference Config File #
|
||||||
|
#####################################
|
||||||
|
|
||||||
|
akka {
|
||||||
|
stream {
|
||||||
|
|
||||||
|
# Default flow materializer settings
|
||||||
|
materializer {
|
||||||
|
|
||||||
|
# Initial size of buffers used in stream elements
|
||||||
|
initial-input-buffer-size = 4
|
||||||
|
# Maximum size of buffers used in stream elements
|
||||||
|
max-input-buffer-size = 16
|
||||||
|
|
||||||
|
# Initial size of fan-out buffers used in stream elements
|
||||||
|
initial-fan-out-buffer-size = 4
|
||||||
|
# Maximum size of fan-out buffers used in stream elements
|
||||||
|
max-fan-out-buffer-size = 16
|
||||||
|
|
||||||
|
# Fully qualified config path which holds the dispatcher configuration
|
||||||
|
# to be used by FlowMaterialiser when creating Actors.
|
||||||
|
# When this value is left empty, the default-dispatcher will be used.
|
||||||
|
dispatcher = ""
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -3,20 +3,34 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import scala.concurrent.duration.FiniteDuration
|
import akka.actor._
|
||||||
import akka.actor.ActorRefFactory
|
import akka.stream.impl.{ ActorBasedFlowMaterializer, Ast, FlowNameCounter, StreamSupervisor }
|
||||||
import akka.stream.impl.ActorBasedFlowMaterializer
|
import com.typesafe.config.Config
|
||||||
import akka.stream.impl.Ast
|
|
||||||
import org.reactivestreams.{ Publisher, Subscriber }
|
import org.reactivestreams.{ Publisher, Subscriber }
|
||||||
import scala.concurrent.duration._
|
|
||||||
import akka.actor.Deploy
|
|
||||||
import akka.actor.ExtendedActorSystem
|
|
||||||
import akka.actor.ActorContext
|
|
||||||
import akka.stream.impl.StreamSupervisor
|
|
||||||
import akka.stream.impl.FlowNameCounter
|
|
||||||
|
|
||||||
object FlowMaterializer {
|
object FlowMaterializer {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Scala API: Creates a FlowMaterializer which will execute every step of a transformation
|
||||||
|
* pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.ActorRefFactory]]
|
||||||
|
* (which can be either an [[akka.actor.ActorSystem]] or an [[akka.actor.ActorContext]])
|
||||||
|
* will be used to create these actors, therefore it is *forbidden* to pass this object
|
||||||
|
* to another actor if the factory is an ActorContext.
|
||||||
|
*
|
||||||
|
* The materializer's [[akka.stream.MaterializerSettings]] will be obtained from the
|
||||||
|
* configuration of the `context`'s underlying [[akka.actor.ActorSystem]].
|
||||||
|
*
|
||||||
|
* The `namePrefix` is used as the first part of the names of the actors running
|
||||||
|
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
||||||
|
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
||||||
|
*/
|
||||||
|
def apply(materializerSettings: Option[MaterializerSettings] = None, namePrefix: Option[String] = None)(implicit context: ActorRefFactory): FlowMaterializer = {
|
||||||
|
val system = actorSystemOf(context)
|
||||||
|
|
||||||
|
val settings = materializerSettings getOrElse MaterializerSettings(system)
|
||||||
|
apply(settings, namePrefix.getOrElse("flow"))(context)
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scala API: Creates a FlowMaterializer which will execute every step of a transformation
|
* Scala API: Creates a FlowMaterializer which will execute every step of a transformation
|
||||||
* pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.ActorRefFactory]]
|
* pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.ActorRefFactory]]
|
||||||
|
|
@ -28,22 +42,43 @@ object FlowMaterializer {
|
||||||
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
||||||
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
||||||
*/
|
*/
|
||||||
def apply(settings: MaterializerSettings, namePrefix: Option[String] = None)(implicit context: ActorRefFactory): FlowMaterializer = {
|
def apply(materializerSettings: MaterializerSettings, namePrefix: String)(implicit context: ActorRefFactory): FlowMaterializer = {
|
||||||
val system = context match {
|
val system = actorSystemOf(context)
|
||||||
case s: ExtendedActorSystem ⇒ s
|
|
||||||
case c: ActorContext ⇒ c.system
|
|
||||||
case null ⇒ throw new IllegalArgumentException("ActorRefFactory context must be defined")
|
|
||||||
case _ ⇒ throw new IllegalArgumentException(s"ActorRefFactory context must be a ActorSystem or ActorContext, " +
|
|
||||||
"got [${_contex.getClass.getName}]")
|
|
||||||
}
|
|
||||||
|
|
||||||
new ActorBasedFlowMaterializer(
|
new ActorBasedFlowMaterializer(
|
||||||
settings,
|
materializerSettings,
|
||||||
context.actorOf(StreamSupervisor.props(settings).withDispatcher(settings.dispatcher)),
|
context.actorOf(StreamSupervisor.props(materializerSettings).withDispatcher(materializerSettings.dispatcher)),
|
||||||
FlowNameCounter(system).counter,
|
FlowNameCounter(system).counter,
|
||||||
namePrefix.getOrElse("flow"))
|
namePrefix)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Scala API: Creates a FlowMaterializer which will execute every step of a transformation
|
||||||
|
* pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.ActorRefFactory]]
|
||||||
|
* (which can be either an [[akka.actor.ActorSystem]] or an [[akka.actor.ActorContext]])
|
||||||
|
* will be used to create these actors, therefore it is *forbidden* to pass this object
|
||||||
|
* to another actor if the factory is an ActorContext.
|
||||||
|
*
|
||||||
|
* The `namePrefix` is used as the first part of the names of the actors running
|
||||||
|
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
||||||
|
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
||||||
|
*/
|
||||||
|
def apply(materializerSettings: MaterializerSettings)(implicit context: ActorRefFactory): FlowMaterializer =
|
||||||
|
apply(Some(materializerSettings), None)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Java API: Creates a FlowMaterializer which will execute every step of a transformation
|
||||||
|
* pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.ActorRefFactory]]
|
||||||
|
* (which can be either an [[akka.actor.ActorSystem]] or an [[akka.actor.ActorContext]])
|
||||||
|
* will be used to create these actors, therefore it is *forbidden* to pass this object
|
||||||
|
* to another actor if the factory is an ActorContext.
|
||||||
|
*
|
||||||
|
* Defaults the actor name prefix used to name actors running the processing steps to `"flow"`.
|
||||||
|
* The actor names are built up of `namePrefix-flowNumber-flowStepNumber-stepName`.
|
||||||
|
*/
|
||||||
|
def create(context: ActorRefFactory): FlowMaterializer =
|
||||||
|
apply()(context)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Creates a FlowMaterializer which will execute every step of a transformation
|
* Java API: Creates a FlowMaterializer which will execute every step of a transformation
|
||||||
* pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.ActorRefFactory]]
|
* pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.ActorRefFactory]]
|
||||||
|
|
@ -52,7 +87,33 @@ object FlowMaterializer {
|
||||||
* to another actor if the factory is an ActorContext.
|
* to another actor if the factory is an ActorContext.
|
||||||
*/
|
*/
|
||||||
def create(settings: MaterializerSettings, context: ActorRefFactory): FlowMaterializer =
|
def create(settings: MaterializerSettings, context: ActorRefFactory): FlowMaterializer =
|
||||||
apply(settings)(context)
|
apply(Option(settings), None)(context)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Java API: Creates a FlowMaterializer which will execute every step of a transformation
|
||||||
|
* pipeline within its own [[akka.actor.Actor]]. The required [[akka.actor.ActorRefFactory]]
|
||||||
|
* (which can be either an [[akka.actor.ActorSystem]] or an [[akka.actor.ActorContext]])
|
||||||
|
* will be used to create these actors, therefore it is *forbidden* to pass this object
|
||||||
|
* to another actor if the factory is an ActorContext.
|
||||||
|
*
|
||||||
|
* The `namePrefix` is used as the first part of the names of the actors running
|
||||||
|
* the processing steps. The default `namePrefix` is `"flow"`. The actor names are built up of
|
||||||
|
* `namePrefix-flowNumber-flowStepNumber-stepName`.
|
||||||
|
*/
|
||||||
|
def create(settings: MaterializerSettings, context: ActorRefFactory, namePrefix: String): FlowMaterializer =
|
||||||
|
apply(Option(settings), Option(namePrefix))(context)
|
||||||
|
|
||||||
|
private def actorSystemOf(context: ActorRefFactory): ActorSystem = {
|
||||||
|
val system = context match {
|
||||||
|
case s: ExtendedActorSystem ⇒ s
|
||||||
|
case c: ActorContext ⇒ c.system
|
||||||
|
case null ⇒ throw new IllegalArgumentException("ActorRefFactory context must be defined")
|
||||||
|
case _ ⇒
|
||||||
|
throw new IllegalArgumentException(s"ActorRefFactory context must be a ActorSystem or ActorContext, got [${context.getClass.getName}]")
|
||||||
|
}
|
||||||
|
system
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -89,13 +150,46 @@ abstract class FlowMaterializer(val settings: MaterializerSettings) {
|
||||||
}
|
}
|
||||||
|
|
||||||
object MaterializerSettings {
|
object MaterializerSettings {
|
||||||
private val defaultSettings = new MaterializerSettings
|
|
||||||
/**
|
/**
|
||||||
* Java API: Default settings.
|
* Create [[MaterializerSettings]].
|
||||||
* Refine the settings using [[MaterializerSettings#withBuffer]],
|
*
|
||||||
|
* You can refine the configuration based settings using [[MaterializerSettings#withBuffer]],
|
||||||
* [[MaterializerSettings#withFanOut]], [[MaterializerSettings#withSubscriptionTimeout]]
|
* [[MaterializerSettings#withFanOut]], [[MaterializerSettings#withSubscriptionTimeout]]
|
||||||
*/
|
*/
|
||||||
def create(): MaterializerSettings = defaultSettings
|
def apply(system: ActorSystem): MaterializerSettings =
|
||||||
|
apply(system.settings.config.getConfig("akka.stream.materializer"))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create [[MaterializerSettings]].
|
||||||
|
*
|
||||||
|
* You can refine the configuration based settings using [[MaterializerSettings#withBuffer]],
|
||||||
|
* [[MaterializerSettings#withFanOut]], [[MaterializerSettings#withSubscriptionTimeout]]
|
||||||
|
*/
|
||||||
|
def apply(config: Config): MaterializerSettings =
|
||||||
|
MaterializerSettings(
|
||||||
|
config.getInt("initial-input-buffer-size"),
|
||||||
|
config.getInt("max-input-buffer-size"),
|
||||||
|
config.getInt("initial-fan-out-buffer-size"),
|
||||||
|
config.getInt("max-fan-out-buffer-size"),
|
||||||
|
config.getString("dispatcher"))
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Java API
|
||||||
|
*
|
||||||
|
* You can refine the configuration based settings using [[MaterializerSettings#withBuffer]],
|
||||||
|
* [[MaterializerSettings#withFanOut]], [[MaterializerSettings#withSubscriptionTimeout]]
|
||||||
|
*/
|
||||||
|
def create(system: ActorSystem): MaterializerSettings =
|
||||||
|
apply(system)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Java API
|
||||||
|
*
|
||||||
|
* You can refine the configuration based settings using [[MaterializerSettings#withBuffer]],
|
||||||
|
* [[MaterializerSettings#withFanOut]], [[MaterializerSettings#withSubscriptionTimeout]]
|
||||||
|
*/
|
||||||
|
def create(config: Config): MaterializerSettings =
|
||||||
|
apply(config)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -104,33 +198,33 @@ object MaterializerSettings {
|
||||||
*
|
*
|
||||||
* This will likely be replaced in the future by auto-tuning these values at runtime.
|
* This will likely be replaced in the future by auto-tuning these values at runtime.
|
||||||
*/
|
*/
|
||||||
case class MaterializerSettings(
|
final case class MaterializerSettings(
|
||||||
initialFanOutBufferSize: Int = 4,
|
initialInputBufferSize: Int,
|
||||||
maxFanOutBufferSize: Int = 16,
|
maxInputBufferSize: Int,
|
||||||
initialInputBufferSize: Int = 4,
|
initialFanOutBufferSize: Int,
|
||||||
maximumInputBufferSize: Int = 16,
|
maxFanOutBufferSize: Int,
|
||||||
dispatcher: String = Deploy.NoDispatcherGiven) {
|
dispatcher: String) {
|
||||||
|
|
||||||
private def isPowerOfTwo(n: Integer): Boolean = (n & (n - 1)) == 0
|
|
||||||
require(initialFanOutBufferSize > 0, "initialFanOutBufferSize must be > 0")
|
|
||||||
require(maxFanOutBufferSize > 0, "maxFanOutBufferSize must be > 0")
|
|
||||||
require(initialFanOutBufferSize <= maxFanOutBufferSize,
|
|
||||||
s"initialFanOutBufferSize($initialFanOutBufferSize) must be <= maxFanOutBufferSize($maxFanOutBufferSize)")
|
|
||||||
|
|
||||||
require(initialInputBufferSize > 0, "initialInputBufferSize must be > 0")
|
require(initialInputBufferSize > 0, "initialInputBufferSize must be > 0")
|
||||||
require(isPowerOfTwo(initialInputBufferSize), "initialInputBufferSize must be a power of two")
|
|
||||||
require(maximumInputBufferSize > 0, "maximumInputBufferSize must be > 0")
|
|
||||||
require(isPowerOfTwo(maximumInputBufferSize), "initialInputBufferSize must be a power of two")
|
|
||||||
require(initialInputBufferSize <= maximumInputBufferSize,
|
|
||||||
s"initialInputBufferSize($initialInputBufferSize) must be <= maximumInputBufferSize($maximumInputBufferSize)")
|
|
||||||
|
|
||||||
def withBuffer(initialInputBufferSize: Int, maximumInputBufferSize: Int): MaterializerSettings =
|
require(maxInputBufferSize > 0, "maxInputBufferSize must be > 0")
|
||||||
copy(initialInputBufferSize = initialInputBufferSize, maximumInputBufferSize = maximumInputBufferSize)
|
require(isPowerOfTwo(maxInputBufferSize), "maxInputBufferSize must be a power of two")
|
||||||
|
require(initialInputBufferSize <= maxInputBufferSize, s"initialInputBufferSize($initialInputBufferSize) must be <= maxInputBufferSize($maxInputBufferSize)")
|
||||||
|
|
||||||
def withFanOut(initialFanOutBufferSize: Int, maxFanOutBufferSize: Int): MaterializerSettings =
|
require(initialFanOutBufferSize > 0, "initialFanOutBufferSize must be > 0")
|
||||||
copy(initialFanOutBufferSize = initialFanOutBufferSize, maxFanOutBufferSize = maxFanOutBufferSize)
|
|
||||||
|
|
||||||
def withDispatcher(dispatcher: String): MaterializerSettings = copy(dispatcher = dispatcher)
|
require(maxFanOutBufferSize > 0, "maxFanOutBufferSize must be > 0")
|
||||||
|
require(isPowerOfTwo(maxFanOutBufferSize), "maxFanOutBufferSize must be a power of two")
|
||||||
|
require(initialFanOutBufferSize <= maxFanOutBufferSize, s"initialFanOutBufferSize($initialFanOutBufferSize) must be <= maxFanOutBufferSize($maxFanOutBufferSize)")
|
||||||
|
|
||||||
|
def withInputBuffer(initialSize: Int, maxSize: Int): MaterializerSettings =
|
||||||
|
copy(initialInputBufferSize = initialSize, maxInputBufferSize = maxSize)
|
||||||
|
|
||||||
|
def withFanOutBuffer(initialSize: Int, maxSize: Int): MaterializerSettings =
|
||||||
|
copy(initialFanOutBufferSize = initialSize, maxFanOutBufferSize = maxSize)
|
||||||
|
|
||||||
|
def withDispatcher(dispatcher: String): MaterializerSettings =
|
||||||
|
copy(dispatcher = dispatcher)
|
||||||
|
|
||||||
|
private def isPowerOfTwo(n: Integer): Boolean = (n & (n - 1)) == 0
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -49,7 +49,7 @@ private[akka] case class FlowImpl[I, O](publisherNode: Ast.PublisherNode[I], ops
|
||||||
}
|
}
|
||||||
|
|
||||||
override def consume()(implicit materializer: FlowMaterializer): Unit =
|
override def consume()(implicit materializer: FlowMaterializer): Unit =
|
||||||
produceTo(new BlackholeSubscriber[Any](materializer.settings.maximumInputBufferSize))
|
produceTo(new BlackholeSubscriber[Any](materializer.settings.maxInputBufferSize))
|
||||||
|
|
||||||
override def onComplete(callback: Try[Unit] ⇒ Unit)(implicit materializer: FlowMaterializer): Unit =
|
override def onComplete(callback: Try[Unit] ⇒ Unit)(implicit materializer: FlowMaterializer): Unit =
|
||||||
transform("onComplete", () ⇒ new Transformer[O, Unit] {
|
transform("onComplete", () ⇒ new Transformer[O, Unit] {
|
||||||
|
|
@ -93,7 +93,7 @@ private[akka] case class DuctImpl[In, Out](ops: List[Ast.AstNode]) extends Duct[
|
||||||
materializer.ductProduceTo(subscriber, ops)
|
materializer.ductProduceTo(subscriber, ops)
|
||||||
|
|
||||||
override def consume()(implicit materializer: FlowMaterializer): Subscriber[In] =
|
override def consume()(implicit materializer: FlowMaterializer): Subscriber[In] =
|
||||||
produceTo(new BlackholeSubscriber[Any](materializer.settings.maximumInputBufferSize))
|
produceTo(new BlackholeSubscriber[Any](materializer.settings.maxInputBufferSize))
|
||||||
|
|
||||||
override def onComplete(callback: Try[Unit] ⇒ Unit)(implicit materializer: FlowMaterializer): Subscriber[In] =
|
override def onComplete(callback: Try[Unit] ⇒ Unit)(implicit materializer: FlowMaterializer): Subscriber[In] =
|
||||||
transform("onComplete", () ⇒ new Transformer[Out, Unit] {
|
transform("onComplete", () ⇒ new Transformer[Out, Unit] {
|
||||||
|
|
|
||||||
|
|
@ -95,7 +95,7 @@ private[akka] class IterablePublisher(iterable: immutable.Iterable[Any], setting
|
||||||
else {
|
else {
|
||||||
val iterator = iterable.iterator
|
val iterator = iterable.iterator
|
||||||
val worker = context.watch(context.actorOf(IterablePublisherWorker.props(iterator, subscriber,
|
val worker = context.watch(context.actorOf(IterablePublisherWorker.props(iterator, subscriber,
|
||||||
settings.maximumInputBufferSize).withDispatcher(context.props.dispatcher)))
|
settings.maxInputBufferSize).withDispatcher(context.props.dispatcher)))
|
||||||
val subscription = new BasicActorSubscription(worker)
|
val subscription = new BasicActorSubscription(worker)
|
||||||
subscribers += subscriber
|
subscribers += subscriber
|
||||||
workers = workers.updated(worker, subscriber)
|
workers = workers.updated(worker, subscriber)
|
||||||
|
|
|
||||||
|
|
@ -48,17 +48,24 @@ object StreamTcp extends ExtensionId[StreamTcpExt] with ExtensionIdProvider {
|
||||||
* message.
|
* message.
|
||||||
*
|
*
|
||||||
* @param remoteAddress the address to connect to
|
* @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 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 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 connectTimeout the desired timeout for connection establishment, infinite means "no timeout"
|
||||||
* @param idleTimeout the desired idle timeout on the connection, infinite means "no timeout"
|
* @param idleTimeout the desired idle timeout on the connection, infinite means "no timeout"
|
||||||
*/
|
*/
|
||||||
case class Connect(settings: MaterializerSettings,
|
case class Connect(remoteAddress: InetSocketAddress,
|
||||||
remoteAddress: InetSocketAddress,
|
|
||||||
localAddress: Option[InetSocketAddress] = None,
|
localAddress: Option[InetSocketAddress] = None,
|
||||||
|
materializerSettings: Option[MaterializerSettings] = None,
|
||||||
options: immutable.Traversable[SocketOption] = Nil,
|
options: immutable.Traversable[SocketOption] = Nil,
|
||||||
connectTimeout: Duration = Duration.Inf,
|
connectTimeout: Duration = Duration.Inf,
|
||||||
idleTimeout: Duration = Duration.Inf) {
|
idleTimeout: Duration = Duration.Inf) {
|
||||||
|
/**
|
||||||
|
* Java API
|
||||||
|
*/
|
||||||
|
def withMaterializerSettings(materializerSettings: MaterializerSettings): Connect =
|
||||||
|
copy(materializerSettings = Option(materializerSettings))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API
|
* Java API
|
||||||
|
|
@ -92,14 +99,16 @@ object StreamTcp extends ExtensionId[StreamTcpExt] with ExtensionIdProvider {
|
||||||
* the Bind message, then the [[StreamTcp.TcpServerBinding]] message should be inspected to find
|
* the Bind message, then the [[StreamTcp.TcpServerBinding]] message should be inspected to find
|
||||||
* the actual port which was bound to.
|
* 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 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
|
* @param backlog the number of unaccepted connections the O/S
|
||||||
* kernel will hold for this port before refusing connections.
|
* 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 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"
|
* @param idleTimeout the desired idle timeout on the accepted connections, infinite means "no timeout"
|
||||||
*/
|
*/
|
||||||
case class Bind(settings: MaterializerSettings,
|
case class Bind(localAddress: InetSocketAddress,
|
||||||
localAddress: InetSocketAddress,
|
settings: Option[MaterializerSettings] = None,
|
||||||
backlog: Int = 100,
|
backlog: Int = 100,
|
||||||
options: immutable.Traversable[SocketOption] = Nil,
|
options: immutable.Traversable[SocketOption] = Nil,
|
||||||
idleTimeout: Duration = Duration.Inf) {
|
idleTimeout: Duration = Duration.Inf) {
|
||||||
|
|
@ -146,15 +155,14 @@ object StreamTcpMessage {
|
||||||
options: java.lang.Iterable[SocketOption],
|
options: java.lang.Iterable[SocketOption],
|
||||||
connectTimeout: Duration,
|
connectTimeout: Duration,
|
||||||
idleTimeout: Duration): StreamTcp.Connect =
|
idleTimeout: Duration): StreamTcp.Connect =
|
||||||
StreamTcp.Connect(settings, remoteAddress, Option(localAddress), Util.immutableSeq(options),
|
StreamTcp.Connect(remoteAddress, Option(localAddress), Option(settings), Util.immutableSeq(options), connectTimeout, idleTimeout)
|
||||||
connectTimeout, idleTimeout)
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Message to Connect to the given `remoteAddress` without binding to a local address and without
|
* Java API: Message to Connect to the given `remoteAddress` without binding to a local address and without
|
||||||
* specifying options.
|
* specifying options.
|
||||||
*/
|
*/
|
||||||
def connect(settings: MaterializerSettings, remoteAddress: InetSocketAddress): StreamTcp.Connect =
|
def connect(settings: MaterializerSettings, remoteAddress: InetSocketAddress): StreamTcp.Connect =
|
||||||
StreamTcp.Connect(settings, remoteAddress)
|
StreamTcp.Connect(remoteAddress, materializerSettings = Option(settings))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: The Bind message is send to the StreamTcp manager actor, which is obtained via
|
* Java API: The Bind message is send to the StreamTcp manager actor, which is obtained via
|
||||||
|
|
@ -163,6 +171,8 @@ object StreamTcpMessage {
|
||||||
* the Bind message, then the [[StreamTcp.TcpServerBinding]] message should be inspected to find
|
* the Bind message, then the [[StreamTcp.TcpServerBinding]] message should be inspected to find
|
||||||
* the actual port which was bound to.
|
* 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 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
|
* @param backlog the number of unaccepted connections the O/S
|
||||||
* kernel will hold for this port before refusing connections.
|
* kernel will hold for this port before refusing connections.
|
||||||
|
|
@ -174,14 +184,13 @@ object StreamTcpMessage {
|
||||||
backlog: Int,
|
backlog: Int,
|
||||||
options: java.lang.Iterable[SocketOption],
|
options: java.lang.Iterable[SocketOption],
|
||||||
idleTimeout: Duration): StreamTcp.Bind =
|
idleTimeout: Duration): StreamTcp.Bind =
|
||||||
StreamTcp.Bind(settings, localAddress, backlog, Util.immutableSeq(options), idleTimeout)
|
StreamTcp.Bind(localAddress, Option(settings), backlog, Util.immutableSeq(options), idleTimeout)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Message to open a listening socket without specifying options.
|
* Java API: Message to open a listening socket without specifying options.
|
||||||
*/
|
*/
|
||||||
def bind(settings: MaterializerSettings,
|
def bind(settings: MaterializerSettings, localAddress: InetSocketAddress): StreamTcp.Bind =
|
||||||
localAddress: InetSocketAddress): StreamTcp.Bind =
|
StreamTcp.Bind(localAddress, Option(settings))
|
||||||
StreamTcp.Bind(settings, localAddress)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -211,22 +220,26 @@ private[akka] class StreamTcpManager extends Actor {
|
||||||
}
|
}
|
||||||
|
|
||||||
def receive: Receive = {
|
def receive: Receive = {
|
||||||
case StreamTcp.Connect(settings, remoteAddress, localAddress, options, connectTimeout, idleTimeout) ⇒
|
case StreamTcp.Connect(remoteAddress, localAddress, maybeMaterializerSettings, options, connectTimeout, idleTimeout) ⇒
|
||||||
val connTimeout = connectTimeout match {
|
val connTimeout = connectTimeout match {
|
||||||
case x: FiniteDuration ⇒ Some(x)
|
case x: FiniteDuration ⇒ Some(x)
|
||||||
case _ ⇒ None
|
case _ ⇒ None
|
||||||
}
|
}
|
||||||
|
val materializerSettings = maybeMaterializerSettings getOrElse MaterializerSettings(context.system)
|
||||||
|
|
||||||
val processorActor = context.actorOf(TcpStreamActor.outboundProps(
|
val processorActor = context.actorOf(TcpStreamActor.outboundProps(
|
||||||
Tcp.Connect(remoteAddress, localAddress, options, connTimeout, pullMode = true),
|
Tcp.Connect(remoteAddress, localAddress, options, connTimeout, pullMode = true),
|
||||||
requester = sender(),
|
requester = sender(),
|
||||||
settings), name = encName("client", remoteAddress))
|
settings = materializerSettings), name = encName("client", remoteAddress))
|
||||||
processorActor ! ExposedProcessor(ActorProcessor[ByteString, ByteString](processorActor))
|
processorActor ! ExposedProcessor(ActorProcessor[ByteString, ByteString](processorActor))
|
||||||
|
|
||||||
case StreamTcp.Bind(settings, localAddress, backlog, options, idleTimeout) ⇒
|
case StreamTcp.Bind(localAddress, maybeMaterializerSettings, backlog, options, idleTimeout) ⇒
|
||||||
|
val materializerSettings = maybeMaterializerSettings getOrElse MaterializerSettings(context.system)
|
||||||
|
|
||||||
val publisherActor = context.actorOf(TcpListenStreamActor.props(
|
val publisherActor = context.actorOf(TcpListenStreamActor.props(
|
||||||
Tcp.Bind(context.system.deadLetters, localAddress, backlog, options, pullMode = true),
|
Tcp.Bind(context.system.deadLetters, localAddress, backlog, options, pullMode = true),
|
||||||
requester = sender(),
|
requester = sender(),
|
||||||
settings), name = encName("server", localAddress))
|
materializerSettings), name = encName("server", localAddress))
|
||||||
publisherActor ! ExposedPublisher(ActorPublisher[Any](publisherActor))
|
publisherActor ! ExposedPublisher(ActorPublisher[Any](publisherActor))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -19,15 +19,16 @@ import scala.util.control.NoStackTrace
|
||||||
private[akka] object TcpListenStreamActor {
|
private[akka] object TcpListenStreamActor {
|
||||||
class TcpListenStreamException(msg: String) extends RuntimeException(msg) with NoStackTrace
|
class TcpListenStreamException(msg: String) extends RuntimeException(msg) with NoStackTrace
|
||||||
|
|
||||||
def props(bindCmd: Tcp.Bind, requester: ActorRef, settings: MaterializerSettings): Props =
|
def props(bindCmd: Tcp.Bind, requester: ActorRef, settings: MaterializerSettings): Props = {
|
||||||
Props(new TcpListenStreamActor(bindCmd, requester, settings)).withDispatcher(settings.dispatcher)
|
Props(new TcpListenStreamActor(bindCmd, requester, settings))
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] class TcpListenStreamActor(bindCmd: Tcp.Bind, requester: ActorRef, val settings: MaterializerSettings) extends Actor
|
private[akka] class TcpListenStreamActor(bindCmd: Tcp.Bind, requester: ActorRef, settings: MaterializerSettings) extends Actor
|
||||||
with Pump with Stash {
|
with Pump with Stash {
|
||||||
import akka.stream.io.TcpListenStreamActor._
|
import akka.stream.io.TcpListenStreamActor._
|
||||||
import context.system
|
import context.system
|
||||||
|
|
|
||||||
|
|
@ -15,11 +15,12 @@ import akka.stream.MaterializerSettings
|
||||||
* [[akka.stream.MaterializerSettings]] for the `FlowMaterializer`.
|
* [[akka.stream.MaterializerSettings]] for the `FlowMaterializer`.
|
||||||
*/
|
*/
|
||||||
trait ImplicitFlowMaterializer { this: Actor ⇒
|
trait ImplicitFlowMaterializer { this: Actor ⇒
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Subclass may override this to define custom
|
* Subclass may override this to define custom
|
||||||
* [[akka.stream.MaterializerSettings]] for the `FlowMaterializer`.
|
* [[akka.stream.MaterializerSettings]] for the `FlowMaterializer`.
|
||||||
*/
|
*/
|
||||||
def flowMaterializerSettings: MaterializerSettings = MaterializerSettings()
|
def flowMaterializerSettings: MaterializerSettings = MaterializerSettings(context.system)
|
||||||
|
|
||||||
final implicit val flowMaterializer: FlowMaterializer = FlowMaterializer(flowMaterializerSettings)
|
final implicit val flowMaterializer: FlowMaterializer = FlowMaterializer(Some(flowMaterializerSettings))
|
||||||
}
|
}
|
||||||
|
|
@ -201,7 +201,7 @@ final case class RunnableFlow[-In, +Out](input: Input[In], output: Output[Out],
|
||||||
|
|
||||||
// FIXME
|
// FIXME
|
||||||
def run()(implicit materializer: FlowMaterializer): Unit =
|
def run()(implicit materializer: FlowMaterializer): Unit =
|
||||||
produceTo(new BlackholeSubscriber[Any](materializer.settings.maximumInputBufferSize))
|
produceTo(new BlackholeSubscriber[Any](materializer.settings.maxInputBufferSize))
|
||||||
|
|
||||||
// FIXME replace with run and input/output factories
|
// FIXME replace with run and input/output factories
|
||||||
def toPublisher[U >: Out]()(implicit materializer: FlowMaterializer): Publisher[U] =
|
def toPublisher[U >: Out]()(implicit materializer: FlowMaterializer): Publisher[U] =
|
||||||
|
|
|
||||||
|
|
@ -41,7 +41,7 @@ public class ActorPublisherTest {
|
||||||
|
|
||||||
final ActorSystem system = actorSystemResource.getSystem();
|
final ActorSystem system = actorSystemResource.getSystem();
|
||||||
|
|
||||||
final MaterializerSettings settings = MaterializerSettings.create().withDispatcher("akka.test.stream-dispatcher");
|
final MaterializerSettings settings = MaterializerSettings.create(system.settings().config()).withDispatcher("akka.test.stream-dispatcher");
|
||||||
final FlowMaterializer materializer = FlowMaterializer.create(settings, system);
|
final FlowMaterializer materializer = FlowMaterializer.create(settings, system);
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
||||||
|
|
@ -58,7 +58,7 @@ public class ActorSubscriberTest {
|
||||||
|
|
||||||
final ActorSystem system = actorSystemResource.getSystem();
|
final ActorSystem system = actorSystemResource.getSystem();
|
||||||
|
|
||||||
final MaterializerSettings settings = MaterializerSettings.create().withDispatcher("akka.test.stream-dispatcher");
|
final MaterializerSettings settings = MaterializerSettings.create(system.settings().config()).withDispatcher("akka.test.stream-dispatcher");
|
||||||
final FlowMaterializer materializer = FlowMaterializer.create(settings, system);
|
final FlowMaterializer materializer = FlowMaterializer.create(settings, system);
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
||||||
|
|
@ -29,7 +29,7 @@ public class DuctTest {
|
||||||
|
|
||||||
final ActorSystem system = actorSystemResource.getSystem();
|
final ActorSystem system = actorSystemResource.getSystem();
|
||||||
|
|
||||||
final MaterializerSettings settings = MaterializerSettings.create().withDispatcher("akka.test.stream-dispatcher");
|
final MaterializerSettings settings = MaterializerSettings.create(system.settings().config()).withDispatcher("akka.test.stream-dispatcher");
|
||||||
final FlowMaterializer materializer = FlowMaterializer.create(settings, system);
|
final FlowMaterializer materializer = FlowMaterializer.create(settings, system);
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
||||||
|
|
@ -30,7 +30,7 @@ public class FlowTest {
|
||||||
|
|
||||||
final ActorSystem system = actorSystemResource.getSystem();
|
final ActorSystem system = actorSystemResource.getSystem();
|
||||||
|
|
||||||
final MaterializerSettings settings = MaterializerSettings.create().withDispatcher("akka.test.stream-dispatcher");
|
final MaterializerSettings settings = MaterializerSettings.create(system.settings().config()).withDispatcher("akka.test.stream-dispatcher");
|
||||||
final FlowMaterializer materializer = FlowMaterializer.create(settings, system);
|
final FlowMaterializer materializer = FlowMaterializer.create(settings, system);
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
||||||
|
|
@ -1,7 +1,6 @@
|
||||||
# The StreamTestDefaultMailbox verifies that stream actors are using
|
# The StreamTestDefaultMailbox verifies that stream actors are using the dispatcher defined in MaterializerSettings.
|
||||||
# the dispatcher defined in MaterializerSettings. All tests should use
|
#
|
||||||
# MaterializerSettings(dispatcher = "akka.test.stream-dispatcher")
|
# All stream tests should use the dedicated `akka.test.stream-dispatcher` or disable this validation by defining:
|
||||||
# or disable this check by defining
|
|
||||||
# akka.actor.default-mailbox.mailbox-type = "akka.dispatch.UnboundedMailbox"
|
# akka.actor.default-mailbox.mailbox-type = "akka.dispatch.UnboundedMailbox"
|
||||||
akka.actor.default-mailbox.mailbox-type = "akka.stream.testkit.StreamTestDefaultMailbox"
|
akka.actor.default-mailbox.mailbox-type = "akka.stream.testkit.StreamTestDefaultMailbox"
|
||||||
|
|
||||||
|
|
@ -16,3 +15,9 @@ akka.test.stream-dispatcher {
|
||||||
}
|
}
|
||||||
mailbox-requirement = "akka.dispatch.UnboundedMessageQueueSemantics"
|
mailbox-requirement = "akka.dispatch.UnboundedMessageQueueSemantics"
|
||||||
}
|
}
|
||||||
|
|
||||||
|
akka.stream {
|
||||||
|
materializer {
|
||||||
|
dispatcher = "akka.test.stream-dispatcher"
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
|
||||||
|
|
@ -7,7 +7,6 @@ import org.reactivestreams.Publisher
|
||||||
|
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import akka.persistence.PersistentActor
|
import akka.persistence.PersistentActor
|
||||||
import akka.persistence.stream.PersistentFlow
|
|
||||||
import akka.stream._
|
import akka.stream._
|
||||||
import akka.stream.scaladsl._
|
import akka.stream.scaladsl._
|
||||||
|
|
||||||
|
|
@ -37,7 +36,7 @@ object PersistentPublisherExample extends App {
|
||||||
val p1 = system.actorOf(Props(classOf[ExamplePersistentActor], "p1"))
|
val p1 = system.actorOf(Props(classOf[ExamplePersistentActor], "p1"))
|
||||||
val p2 = system.actorOf(Props(classOf[ExamplePersistentActor], "p2"))
|
val p2 = system.actorOf(Props(classOf[ExamplePersistentActor], "p2"))
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings())
|
implicit val materializer = FlowMaterializer()
|
||||||
|
|
||||||
// 1 view-backed publisher and 2 subscribers:
|
// 1 view-backed publisher and 2 subscribers:
|
||||||
val publisher1: Publisher[Any] = PersistentFlow.fromPersistentActor("p1").toPublisher()
|
val publisher1: Publisher[Any] = PersistentFlow.fromPersistentActor("p1").toPublisher()
|
||||||
|
|
|
||||||
|
|
@ -34,7 +34,7 @@ class PersistentPublisherSpec extends AkkaSpec(PersistenceSpec.config("leveldb",
|
||||||
val numMessages = 10
|
val numMessages = 10
|
||||||
|
|
||||||
val publisherSettings = PersistentPublisherSettings(idle = Some(100.millis))
|
val publisherSettings = PersistentPublisherSettings(idle = Some(100.millis))
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(dispatcher = "akka.test.stream-dispatcher"))
|
implicit val materializer = FlowMaterializer()
|
||||||
|
|
||||||
var persistentActor1: ActorRef = _
|
var persistentActor1: ActorRef = _
|
||||||
var persistentActor2: ActorRef = _
|
var persistentActor2: ActorRef = _
|
||||||
|
|
|
||||||
|
|
@ -22,7 +22,7 @@ object DuctSpec {
|
||||||
class DuctSpec extends AkkaSpec {
|
class DuctSpec extends AkkaSpec {
|
||||||
import DuctSpec._
|
import DuctSpec._
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(dispatcher = "akka.test.stream-dispatcher"))
|
implicit val materializer = FlowMaterializer()
|
||||||
|
|
||||||
"A Duct" must {
|
"A Duct" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -11,12 +11,11 @@ import akka.stream.testkit.StreamTestKit
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowBroadcastSpec extends AkkaSpec {
|
class FlowBroadcastSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||||
maximumInputBufferSize = 16,
|
.withFanOutBuffer(initialSize = 1, maxSize = 16)
|
||||||
initialFanOutBufferSize = 1,
|
|
||||||
maxFanOutBufferSize = 16,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"A broadcast" must {
|
"A broadcast" must {
|
||||||
|
|
||||||
|
|
@ -3,20 +3,19 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
|
|
||||||
import akka.stream.scaladsl.Flow
|
import akka.stream.scaladsl.Flow
|
||||||
|
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
|
||||||
|
|
||||||
import scala.concurrent.Await
|
import scala.concurrent.Await
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import OverflowStrategy._
|
|
||||||
|
|
||||||
class FlowBufferSpec extends AkkaSpec {
|
class FlowBufferSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 1,
|
.withInputBuffer(initialSize = 1, maxSize = 1)
|
||||||
maximumInputBufferSize = 1,
|
.withFanOutBuffer(initialSize = 1, maxSize = 1)
|
||||||
initialFanOutBufferSize = 1,
|
|
||||||
maxFanOutBufferSize = 1,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"Buffer" must {
|
"Buffer" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -3,13 +3,13 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import akka.stream.testkit.AkkaSpec
|
import akka.stream.testkit.{ AkkaSpec, ScriptedTest }
|
||||||
import akka.stream.testkit.{ StreamTestKit, ScriptedTest }
|
|
||||||
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
||||||
|
|
||||||
class FlowCollectSpec extends AkkaSpec with ScriptedTest {
|
class FlowCollectSpec extends AkkaSpec with ScriptedTest {
|
||||||
|
|
||||||
val settings = MaterializerSettings(dispatcher = "akka.test.stream-dispatcher")
|
val settings = MaterializerSettings(system)
|
||||||
|
|
||||||
"A Collect" must {
|
"A Collect" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -12,12 +12,11 @@ import scala.util.control.NoStackTrace
|
||||||
|
|
||||||
class FlowConcatAllSpec extends AkkaSpec {
|
class FlowConcatAllSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 2)
|
||||||
maximumInputBufferSize = 2,
|
.withFanOutBuffer(initialSize = 2, maxSize = 2)
|
||||||
initialFanOutBufferSize = 2,
|
|
||||||
maxFanOutBufferSize = 2,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"ConcatAll" must {
|
"ConcatAll" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -11,12 +11,11 @@ import scala.concurrent.duration._
|
||||||
|
|
||||||
class FlowConflateSpec extends AkkaSpec {
|
class FlowConflateSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 2)
|
||||||
maximumInputBufferSize = 2,
|
.withFanOutBuffer(initialSize = 2, maxSize = 2)
|
||||||
initialFanOutBufferSize = 2,
|
|
||||||
maxFanOutBufferSize = 2,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"Conflate" must {
|
"Conflate" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -4,13 +4,13 @@
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import akka.stream.scaladsl.Flow
|
import akka.stream.scaladsl.Flow
|
||||||
import akka.stream.testkit.AkkaSpec
|
|
||||||
import akka.testkit.TestProbe
|
import akka.testkit.TestProbe
|
||||||
|
import akka.stream.testkit.AkkaSpec
|
||||||
|
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowDispatcherSpec extends AkkaSpec {
|
class FlowDispatcherSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(dispatcher = "akka.test.stream-dispatcher"))
|
implicit val materializer = FlowMaterializer()
|
||||||
|
|
||||||
"Flow with dispatcher setting" must {
|
"Flow with dispatcher setting" must {
|
||||||
"use the specified dispatcher" in {
|
"use the specified dispatcher" in {
|
||||||
|
|
|
||||||
|
|
@ -3,20 +3,17 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import akka.stream.testkit.AkkaSpec
|
import akka.stream.testkit.{ AkkaSpec, ScriptedTest }
|
||||||
import akka.stream.testkit.ScriptedTest
|
|
||||||
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
||||||
import akka.stream.testkit.StreamTestKit
|
import akka.stream.testkit.StreamTestKit
|
||||||
import akka.stream.scaladsl.Flow
|
import akka.stream.scaladsl.Flow
|
||||||
|
|
||||||
class FlowDropSpec extends AkkaSpec with ScriptedTest {
|
class FlowDropSpec extends AkkaSpec with ScriptedTest {
|
||||||
|
|
||||||
val settings = MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||||
maximumInputBufferSize = 16,
|
.withFanOutBuffer(initialSize = 1, maxSize = 16)
|
||||||
initialFanOutBufferSize = 1,
|
|
||||||
maxFanOutBufferSize = 16,
|
|
||||||
dispatcher = "akka.test.stream-dispatcher")
|
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(settings)
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -11,8 +11,7 @@ import akka.stream.testkit.StreamTestKit
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowDropWithinSpec extends AkkaSpec {
|
class FlowDropWithinSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
implicit val materializer = FlowMaterializer()
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"A DropWithin" must {
|
"A DropWithin" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -11,12 +11,11 @@ import scala.concurrent.duration._
|
||||||
|
|
||||||
class FlowExpandSpec extends AkkaSpec {
|
class FlowExpandSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 2)
|
||||||
maximumInputBufferSize = 2,
|
.withFanOutBuffer(initialSize = 2, maxSize = 2)
|
||||||
initialFanOutBufferSize = 2,
|
|
||||||
maxFanOutBufferSize = 2,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"Expand" must {
|
"Expand" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -3,20 +3,16 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import akka.stream.testkit.AkkaSpec
|
|
||||||
import akka.stream.testkit.{ StreamTestKit, ScriptedTest }
|
|
||||||
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
|
||||||
import akka.stream.scaladsl.Flow
|
import akka.stream.scaladsl.Flow
|
||||||
import akka.stream.impl.ActorBasedFlowMaterializer
|
import akka.stream.testkit.{ AkkaSpec, ScriptedTest, StreamTestKit }
|
||||||
|
|
||||||
|
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
||||||
|
|
||||||
class FlowFilterSpec extends AkkaSpec with ScriptedTest {
|
class FlowFilterSpec extends AkkaSpec with ScriptedTest {
|
||||||
|
|
||||||
val settings = MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||||
maximumInputBufferSize = 16,
|
.withFanOutBuffer(initialSize = 1, maxSize = 16)
|
||||||
initialFanOutBufferSize = 1,
|
|
||||||
maxFanOutBufferSize = 16,
|
|
||||||
dispatcher = "akka.test.stream-dispatcher")
|
|
||||||
|
|
||||||
"A Filter" must {
|
"A Filter" must {
|
||||||
|
|
||||||
|
|
@ -26,12 +22,10 @@ class FlowFilterSpec extends AkkaSpec with ScriptedTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
"not blow up with high request counts" in {
|
"not blow up with high request counts" in {
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 1,
|
.withInputBuffer(initialSize = 1, maxSize = 1)
|
||||||
maximumInputBufferSize = 1,
|
.withFanOutBuffer(initialSize = 1, maxSize = 1)
|
||||||
initialFanOutBufferSize = 1,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
maxFanOutBufferSize = 1,
|
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
val probe = StreamTestKit.SubscriberProbe[Int]()
|
val probe = StreamTestKit.SubscriberProbe[Int]()
|
||||||
Flow(Iterator.fill(1000)(0) ++ List(1)).filter(_ != 0).
|
Flow(Iterator.fill(1000)(0) ++ List(1)).filter(_ != 0).
|
||||||
|
|
|
||||||
|
|
@ -3,18 +3,15 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import akka.stream.testkit.AkkaSpec
|
import akka.stream.testkit.{ AkkaSpec, ScriptedTest }
|
||||||
import akka.stream.testkit.ScriptedTest
|
|
||||||
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
||||||
|
|
||||||
class FlowFoldSpec extends AkkaSpec with ScriptedTest {
|
class FlowFoldSpec extends AkkaSpec with ScriptedTest {
|
||||||
|
|
||||||
val settings = MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||||
maximumInputBufferSize = 16,
|
.withFanOutBuffer(initialSize = 2, maxSize = 16)
|
||||||
initialFanOutBufferSize = 1,
|
|
||||||
maxFanOutBufferSize = 16,
|
|
||||||
dispatcher = "akka.test.stream-dispatcher")
|
|
||||||
|
|
||||||
"A Fold" must {
|
"A Fold" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -3,16 +3,15 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import akka.stream.testkit.AkkaSpec
|
|
||||||
import akka.stream.testkit.ScriptedTest
|
|
||||||
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
|
||||||
import akka.stream.scaladsl.Flow
|
import akka.stream.scaladsl.Flow
|
||||||
import akka.stream.testkit.StreamTestKit
|
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
|
||||||
|
|
||||||
|
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
||||||
import scala.util.control.NoStackTrace
|
import scala.util.control.NoStackTrace
|
||||||
|
|
||||||
class FlowForeachSpec extends AkkaSpec {
|
class FlowForeachSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val mat = FlowMaterializer(MaterializerSettings(dispatcher = "akka.test.stream-dispatcher"))
|
implicit val mat = FlowMaterializer()
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
|
|
||||||
"A Foreach" must {
|
"A Foreach" must {
|
||||||
|
|
|
||||||
|
|
@ -12,7 +12,9 @@ import scala.concurrent.duration._
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowFromFutureSpec extends AkkaSpec {
|
class FlowFromFutureSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(dispatcher = "akka.test.stream-dispatcher"))
|
val settings = MaterializerSettings(system)
|
||||||
|
|
||||||
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
|
|
||||||
"A Flow based on a Future" must {
|
"A Flow based on a Future" must {
|
||||||
"produce one element from already successful Future" in {
|
"produce one element from already successful Future" in {
|
||||||
|
|
|
||||||
|
|
@ -12,12 +12,11 @@ import scala.util.control.NoStackTrace
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowGroupBySpec extends AkkaSpec {
|
class FlowGroupBySpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 2)
|
||||||
maximumInputBufferSize = 2,
|
.withFanOutBuffer(initialSize = 2, maxSize = 2)
|
||||||
initialFanOutBufferSize = 2,
|
|
||||||
maxFanOutBufferSize = 2,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
case class StreamPuppet(p: Publisher[Int]) {
|
case class StreamPuppet(p: Publisher[Int]) {
|
||||||
val probe = StreamTestKit.SubscriberProbe[Int]()
|
val probe = StreamTestKit.SubscriberProbe[Int]()
|
||||||
|
|
|
||||||
|
|
@ -3,19 +3,16 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import akka.stream.testkit.AkkaSpec
|
import akka.stream.testkit.{ AkkaSpec, ScriptedTest }
|
||||||
import akka.stream.testkit.ScriptedTest
|
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
||||||
|
|
||||||
class FlowGroupedSpec extends AkkaSpec with ScriptedTest {
|
class FlowGroupedSpec extends AkkaSpec with ScriptedTest {
|
||||||
|
|
||||||
val settings = MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||||
maximumInputBufferSize = 16,
|
.withFanOutBuffer(initialSize = 1, maxSize = 16)
|
||||||
initialFanOutBufferSize = 1,
|
|
||||||
maxFanOutBufferSize = 16,
|
|
||||||
dispatcher = "akka.test.stream-dispatcher")
|
|
||||||
|
|
||||||
"A Grouped" must {
|
"A Grouped" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -14,8 +14,9 @@ import akka.stream.testkit.ScriptedTest
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
|
class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
|
||||||
|
|
||||||
val settings = MaterializerSettings(dispatcher = "akka.test.stream-dispatcher")
|
val settings = MaterializerSettings(system)
|
||||||
implicit val materializer = FlowMaterializer(settings)
|
|
||||||
|
implicit val materializer = FlowMaterializer()
|
||||||
|
|
||||||
"A GroupedWithin" must {
|
"A GroupedWithin" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -12,9 +12,10 @@ import scala.concurrent.duration._
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowIterableSpec extends AkkaSpec {
|
class FlowIterableSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
maximumInputBufferSize = 512,
|
.withInputBuffer(initialSize = 2, maxSize = 512)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
|
|
||||||
"A Flow based on an iterable" must {
|
"A Flow based on an iterable" must {
|
||||||
"produce elements" in {
|
"produce elements" in {
|
||||||
|
|
|
||||||
|
|
@ -14,12 +14,11 @@ import akka.stream.scaladsl.Flow
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowIteratorSpec extends AkkaSpec {
|
class FlowIteratorSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 2)
|
||||||
maximumInputBufferSize = 2,
|
.withFanOutBuffer(initialSize = 4, maxSize = 4)
|
||||||
initialFanOutBufferSize = 4,
|
|
||||||
maxFanOutBufferSize = 4,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"A Flow based on an iterator" must {
|
"A Flow based on an iterator" must {
|
||||||
"produce elements" in {
|
"produce elements" in {
|
||||||
|
|
|
||||||
|
|
@ -3,17 +3,13 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import akka.stream.testkit.AkkaSpec
|
import akka.stream.testkit.{ AkkaSpec, ScriptedTest }
|
||||||
import akka.stream.testkit.ScriptedTest
|
|
||||||
|
|
||||||
class FlowMapConcatSpec extends AkkaSpec with ScriptedTest {
|
class FlowMapConcatSpec extends AkkaSpec with ScriptedTest {
|
||||||
|
|
||||||
val settings = MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||||
maximumInputBufferSize = 16,
|
.withFanOutBuffer(initialSize = 1, maxSize = 16)
|
||||||
initialFanOutBufferSize = 1,
|
|
||||||
maxFanOutBufferSize = 16,
|
|
||||||
dispatcher = "akka.test.stream-dispatcher")
|
|
||||||
|
|
||||||
"A MapConcat" must {
|
"A MapConcat" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -17,8 +17,7 @@ import scala.concurrent.Await
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowMapFutureSpec extends AkkaSpec {
|
class FlowMapFutureSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
implicit val materializer = FlowMaterializer()
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"A Flow with mapFuture" must {
|
"A Flow with mapFuture" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -3,20 +3,16 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import akka.stream.testkit.AkkaSpec
|
|
||||||
import akka.stream.testkit.{ StreamTestKit, ScriptedTest }
|
|
||||||
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
|
||||||
import akka.stream.scaladsl.Flow
|
import akka.stream.scaladsl.Flow
|
||||||
import akka.stream.impl.ActorBasedFlowMaterializer
|
import akka.stream.testkit.{ AkkaSpec, ScriptedTest, StreamTestKit }
|
||||||
|
|
||||||
|
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
||||||
|
|
||||||
class FlowMapSpec extends AkkaSpec with ScriptedTest {
|
class FlowMapSpec extends AkkaSpec with ScriptedTest {
|
||||||
|
|
||||||
val settings = MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||||
maximumInputBufferSize = 16,
|
.withFanOutBuffer(initialSize = 1, maxSize = 16)
|
||||||
initialFanOutBufferSize = 1,
|
|
||||||
maxFanOutBufferSize = 16,
|
|
||||||
dispatcher = "akka.test.stream-dispatcher")
|
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(settings)
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -3,28 +3,23 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import akka.stream.testkit.AkkaSpec
|
|
||||||
import akka.stream.testkit.ScriptedTest
|
|
||||||
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
|
||||||
import akka.stream.testkit.StreamTestKit
|
|
||||||
import scala.concurrent.Await
|
|
||||||
import scala.concurrent.duration._
|
|
||||||
import scala.util.Failure
|
|
||||||
import akka.stream.scaladsl.Flow
|
import akka.stream.scaladsl.Flow
|
||||||
|
import akka.stream.testkit.{ AkkaSpec, ScriptedTest, StreamTestKit }
|
||||||
import akka.testkit.TestProbe
|
import akka.testkit.TestProbe
|
||||||
import scala.util.Try
|
|
||||||
import scala.util.Success
|
import scala.concurrent.duration._
|
||||||
|
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
||||||
import scala.util.control.NoStackTrace
|
import scala.util.control.NoStackTrace
|
||||||
|
import scala.util.{ Failure, Success }
|
||||||
|
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest {
|
class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||||
maximumInputBufferSize = 16,
|
.withFanOutBuffer(initialSize = 1, maxSize = 16)
|
||||||
initialFanOutBufferSize = 1,
|
|
||||||
maxFanOutBufferSize = 16,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"A Flow with onComplete" must {
|
"A Flow with onComplete" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -14,12 +14,11 @@ import scala.util.control.NoStackTrace
|
||||||
|
|
||||||
class FlowPrefixAndTailSpec extends AkkaSpec {
|
class FlowPrefixAndTailSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 2)
|
||||||
maximumInputBufferSize = 2,
|
.withFanOutBuffer(initialSize = 2, maxSize = 2)
|
||||||
initialFanOutBufferSize = 2,
|
|
||||||
maxFanOutBufferSize = 2,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"PrefixAndTail" must {
|
"PrefixAndTail" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -9,7 +9,11 @@ import akka.stream.testkit.StreamTestKit
|
||||||
|
|
||||||
class FlowProduceToSubscriberSpec extends AkkaSpec {
|
class FlowProduceToSubscriberSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(dispatcher = "akka.test.stream-dispatcher"))
|
val settings = MaterializerSettings(system)
|
||||||
|
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||||
|
.withFanOutBuffer(initialSize = 1, maxSize = 16)
|
||||||
|
|
||||||
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
|
|
||||||
"A Flow with toPublisher" must {
|
"A Flow with toPublisher" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -3,14 +3,15 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import scala.collection.immutable
|
import akka.stream.scaladsl.Flow
|
||||||
import scala.concurrent.duration._
|
|
||||||
import akka.stream.testkit.{ AkkaSpec, ChainSetup, StreamTestKit }
|
import akka.stream.testkit.{ AkkaSpec, ChainSetup, StreamTestKit }
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import akka.stream.scaladsl.Flow
|
|
||||||
import org.reactivestreams.Publisher
|
import org.reactivestreams.Publisher
|
||||||
|
|
||||||
|
import scala.collection.immutable
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
object FlowSpec {
|
object FlowSpec {
|
||||||
class Fruit
|
class Fruit
|
||||||
class Apple extends Fruit
|
class Apple extends Fruit
|
||||||
|
|
@ -19,14 +20,11 @@ object FlowSpec {
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.receive=off\nakka.loglevel=INFO")) {
|
class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.receive=off\nakka.loglevel=INFO")) {
|
||||||
import FlowSpec._
|
import FlowSpec._
|
||||||
import system.dispatcher
|
|
||||||
|
|
||||||
val settings = MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||||
maximumInputBufferSize = 16,
|
.withFanOutBuffer(initialSize = 1, maxSize = 16)
|
||||||
initialFanOutBufferSize = 1,
|
|
||||||
maxFanOutBufferSize = 16,
|
|
||||||
dispatcher = "akka.test.stream-dispatcher")
|
|
||||||
implicit val mat = FlowMaterializer(settings)
|
implicit val mat = FlowMaterializer(settings)
|
||||||
|
|
||||||
val identity: Flow[Any] ⇒ Flow[Any] = in ⇒ in.map(e ⇒ e)
|
val identity: Flow[Any] ⇒ Flow[Any] = in ⇒ in.map(e ⇒ e)
|
||||||
|
|
@ -36,7 +34,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
||||||
|
|
||||||
for ((name, op) ← List("identity" -> identity, "identity2" -> identity2); n ← List(1, 2, 4)) {
|
for ((name, op) ← List("identity" -> identity, "identity2" -> identity2); n ← List(1, 2, 4)) {
|
||||||
s"requests initial elements from upstream ($name, $n)" in {
|
s"requests initial elements from upstream ($name, $n)" in {
|
||||||
new ChainSetup(op, settings.copy(initialInputBufferSize = n)) {
|
new ChainSetup(op, settings.withInputBuffer(initialSize = n, maxSize = settings.maxInputBufferSize)) {
|
||||||
upstream.expectRequest(upstreamSubscription, settings.initialInputBufferSize)
|
upstream.expectRequest(upstreamSubscription, settings.initialInputBufferSize)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -89,7 +87,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
||||||
}
|
}
|
||||||
|
|
||||||
"single subscriber cancels subscription while receiving data" in {
|
"single subscriber cancels subscription while receiving data" in {
|
||||||
new ChainSetup(identity, settings.copy(initialInputBufferSize = 1)) {
|
new ChainSetup(identity, settings.withInputBuffer(initialSize = 1, maxSize = settings.maxInputBufferSize)) {
|
||||||
downstreamSubscription.request(5)
|
downstreamSubscription.request(5)
|
||||||
upstreamSubscription.expectRequest(1)
|
upstreamSubscription.expectRequest(1)
|
||||||
upstreamSubscription.sendNext("test")
|
upstreamSubscription.sendNext("test")
|
||||||
|
|
@ -109,7 +107,11 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
||||||
|
|
||||||
"A Flow with multiple subscribers (FanOutBox)" must {
|
"A Flow with multiple subscribers (FanOutBox)" must {
|
||||||
"adapt speed to the currently slowest subscriber" in {
|
"adapt speed to the currently slowest subscriber" in {
|
||||||
new ChainSetup(identity, settings.copy(initialInputBufferSize = 1, maxFanOutBufferSize = 1)) {
|
val tweakedSettings = settings
|
||||||
|
.withInputBuffer(initialSize = 1, maxSize = settings.maxInputBufferSize)
|
||||||
|
.withFanOutBuffer(initialSize = settings.initialFanOutBufferSize, maxSize = 1)
|
||||||
|
|
||||||
|
new ChainSetup(identity, tweakedSettings) {
|
||||||
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
|
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
|
||||||
publisher.subscribe(downstream2)
|
publisher.subscribe(downstream2)
|
||||||
val downstream2Subscription = downstream2.expectSubscription()
|
val downstream2Subscription = downstream2.expectSubscription()
|
||||||
|
|
@ -135,7 +137,11 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
||||||
}
|
}
|
||||||
|
|
||||||
"support slow subscriber with fan-out 2" in {
|
"support slow subscriber with fan-out 2" in {
|
||||||
new ChainSetup(identity, settings.copy(initialInputBufferSize = 1, initialFanOutBufferSize = 2, maxFanOutBufferSize = 2)) {
|
val tweakedSettings = settings
|
||||||
|
.withInputBuffer(initialSize = 1, maxSize = settings.maxInputBufferSize)
|
||||||
|
.withFanOutBuffer(initialSize = 2, maxSize = 2)
|
||||||
|
|
||||||
|
new ChainSetup(identity, tweakedSettings) {
|
||||||
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
|
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
|
||||||
publisher.subscribe(downstream2)
|
publisher.subscribe(downstream2)
|
||||||
val downstream2Subscription = downstream2.expectSubscription()
|
val downstream2Subscription = downstream2.expectSubscription()
|
||||||
|
|
@ -174,7 +180,11 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
||||||
}
|
}
|
||||||
|
|
||||||
"incoming subscriber while elements were requested before" in {
|
"incoming subscriber while elements were requested before" in {
|
||||||
new ChainSetup(identity, settings.copy(initialInputBufferSize = 1, maxFanOutBufferSize = 1)) {
|
val tweakedSettings = settings
|
||||||
|
.withInputBuffer(initialSize = 1, maxSize = settings.maxInputBufferSize)
|
||||||
|
.withFanOutBuffer(initialSize = settings.initialFanOutBufferSize, maxSize = 1)
|
||||||
|
|
||||||
|
new ChainSetup(identity, tweakedSettings) {
|
||||||
downstreamSubscription.request(5)
|
downstreamSubscription.request(5)
|
||||||
upstream.expectRequest(upstreamSubscription, 1)
|
upstream.expectRequest(upstreamSubscription, 1)
|
||||||
upstreamSubscription.sendNext("a1")
|
upstreamSubscription.sendNext("a1")
|
||||||
|
|
@ -211,7 +221,11 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
||||||
}
|
}
|
||||||
|
|
||||||
"blocking subscriber cancels subscription" in {
|
"blocking subscriber cancels subscription" in {
|
||||||
new ChainSetup(identity, settings.copy(initialInputBufferSize = 1, maxFanOutBufferSize = 1)) {
|
val tweakedSettings = settings
|
||||||
|
.withInputBuffer(initialSize = 1, maxSize = settings.maxInputBufferSize)
|
||||||
|
.withFanOutBuffer(initialSize = settings.initialFanOutBufferSize, maxSize = 1)
|
||||||
|
|
||||||
|
new ChainSetup(identity, tweakedSettings) {
|
||||||
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
|
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
|
||||||
publisher.subscribe(downstream2)
|
publisher.subscribe(downstream2)
|
||||||
val downstream2Subscription = downstream2.expectSubscription()
|
val downstream2Subscription = downstream2.expectSubscription()
|
||||||
|
|
@ -246,7 +260,11 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
||||||
}
|
}
|
||||||
|
|
||||||
"after initial upstream was completed future subscribers' onComplete should be called instead of onSubscribed" in {
|
"after initial upstream was completed future subscribers' onComplete should be called instead of onSubscribed" in {
|
||||||
new ChainSetup(identity, settings.copy(initialInputBufferSize = 1, maxFanOutBufferSize = 1)) {
|
val tweakedSettings = settings
|
||||||
|
.withInputBuffer(initialSize = 1, maxSize = settings.maxInputBufferSize)
|
||||||
|
.withFanOutBuffer(initialSize = settings.initialFanOutBufferSize, maxSize = 1)
|
||||||
|
|
||||||
|
new ChainSetup(identity, tweakedSettings) {
|
||||||
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
|
val downstream2 = StreamTestKit.SubscriberProbe[Any]()
|
||||||
// don't link it just yet
|
// don't link it just yet
|
||||||
|
|
||||||
|
|
@ -285,7 +303,11 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
||||||
}
|
}
|
||||||
|
|
||||||
"after initial upstream reported an error future subscribers' onError should be called instead of onSubscribed" in {
|
"after initial upstream reported an error future subscribers' onError should be called instead of onSubscribed" in {
|
||||||
new ChainSetup[Int, String](_.map(_ ⇒ throw TestException), settings.copy(initialInputBufferSize = 1, maxFanOutBufferSize = 1)) {
|
val tweakedSettings = settings
|
||||||
|
.withInputBuffer(initialSize = 1, maxSize = settings.maxInputBufferSize)
|
||||||
|
.withFanOutBuffer(initialSize = settings.initialFanOutBufferSize, maxSize = 1)
|
||||||
|
|
||||||
|
new ChainSetup[Int, String](_.map(_ ⇒ throw TestException), tweakedSettings) {
|
||||||
downstreamSubscription.request(1)
|
downstreamSubscription.request(1)
|
||||||
upstreamSubscription.expectRequest(1)
|
upstreamSubscription.expectRequest(1)
|
||||||
|
|
||||||
|
|
@ -303,7 +325,11 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
||||||
}
|
}
|
||||||
|
|
||||||
"when all subscriptions were cancelled future subscribers' onError should be called" in {
|
"when all subscriptions were cancelled future subscribers' onError should be called" in {
|
||||||
new ChainSetup(identity, settings.copy(initialInputBufferSize = 1)) {
|
val tweakedSettings = settings
|
||||||
|
.withInputBuffer(initialSize = 1, maxSize = settings.maxInputBufferSize)
|
||||||
|
.withFanOutBuffer(initialSize = settings.initialFanOutBufferSize, maxSize = settings.maxFanOutBufferSize)
|
||||||
|
|
||||||
|
new ChainSetup(identity, tweakedSettings) {
|
||||||
upstreamSubscription.expectRequest(1)
|
upstreamSubscription.expectRequest(1)
|
||||||
downstreamSubscription.cancel()
|
downstreamSubscription.cancel()
|
||||||
upstreamSubscription.expectCancellation()
|
upstreamSubscription.expectCancellation()
|
||||||
|
|
|
||||||
|
|
@ -12,12 +12,11 @@ import akka.stream.scaladsl.Flow
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowSplitWhenSpec extends AkkaSpec {
|
class FlowSplitWhenSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 2)
|
||||||
maximumInputBufferSize = 2,
|
.withFanOutBuffer(initialSize = 2, maxSize = 2)
|
||||||
initialFanOutBufferSize = 2,
|
|
||||||
maxFanOutBufferSize = 2,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
case class StreamPuppet(p: Publisher[Int]) {
|
case class StreamPuppet(p: Publisher[Int]) {
|
||||||
val probe = StreamTestKit.SubscriberProbe[Int]()
|
val probe = StreamTestKit.SubscriberProbe[Int]()
|
||||||
|
|
|
||||||
|
|
@ -3,22 +3,21 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
|
import akka.stream.actor.ActorSubscriberMessage.OnComplete
|
||||||
|
import akka.stream.actor.ActorSubscriberMessage.OnNext
|
||||||
|
import akka.stream.impl.RequestMore
|
||||||
|
import akka.stream.scaladsl.Flow
|
||||||
import akka.stream.testkit.AkkaSpec
|
import akka.stream.testkit.AkkaSpec
|
||||||
import akka.stream.testkit.ScriptedTest
|
import akka.stream.testkit.ScriptedTest
|
||||||
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
|
||||||
import akka.stream.actor.ActorSubscriberMessage.{ OnNext, OnComplete }
|
|
||||||
import akka.stream.impl.RequestMore
|
|
||||||
import akka.stream.testkit.StreamTestKit
|
import akka.stream.testkit.StreamTestKit
|
||||||
import akka.stream.scaladsl.Flow
|
|
||||||
|
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
||||||
|
|
||||||
class FlowTakeSpec extends AkkaSpec with ScriptedTest {
|
class FlowTakeSpec extends AkkaSpec with ScriptedTest {
|
||||||
|
|
||||||
val settings = MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||||
maximumInputBufferSize = 16,
|
.withFanOutBuffer(initialSize = 1, maxSize = 16)
|
||||||
initialFanOutBufferSize = 1,
|
|
||||||
maxFanOutBufferSize = 16,
|
|
||||||
dispatcher = "akka.test.stream-dispatcher")
|
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(settings)
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -11,8 +11,7 @@ import akka.stream.scaladsl.Flow
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowTakeWithinSpec extends AkkaSpec {
|
class FlowTakeWithinSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
implicit val materializer = FlowMaterializer()
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"A TakeWithin" must {
|
"A TakeWithin" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -12,7 +12,7 @@ import scala.util.control.NoStackTrace
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowTimerTransformerSpec extends AkkaSpec {
|
class FlowTimerTransformerSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(dispatcher = "akka.test.stream-dispatcher"))
|
implicit val materializer = FlowMaterializer()
|
||||||
|
|
||||||
"A Flow with TimerTransformer operations" must {
|
"A Flow with TimerTransformer operations" must {
|
||||||
"produce scheduled ticks as expected" in {
|
"produce scheduled ticks as expected" in {
|
||||||
|
|
|
||||||
|
|
@ -3,23 +3,21 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import akka.stream.testkit.AkkaSpec
|
import akka.stream.scaladsl.Flow
|
||||||
import akka.stream.testkit.ScriptedTest
|
import akka.stream.testkit.{ AkkaSpec, ScriptedTest, StreamTestKit }
|
||||||
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
|
||||||
import akka.stream.testkit.StreamTestKit
|
|
||||||
import scala.concurrent.Await
|
import scala.concurrent.Await
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
|
||||||
import scala.util.Failure
|
import scala.util.Failure
|
||||||
import akka.stream.scaladsl.Flow
|
|
||||||
|
|
||||||
class FlowToFutureSpec extends AkkaSpec with ScriptedTest {
|
class FlowToFutureSpec extends AkkaSpec with ScriptedTest {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||||
maximumInputBufferSize = 16,
|
.withFanOutBuffer(initialSize = 1, maxSize = 16)
|
||||||
initialFanOutBufferSize = 1,
|
|
||||||
maxFanOutBufferSize = 16,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"A Flow with toFuture" must {
|
"A Flow with toFuture" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -31,12 +31,11 @@ object FlowTransformRecoverSpec {
|
||||||
class FlowTransformRecoverSpec extends AkkaSpec {
|
class FlowTransformRecoverSpec extends AkkaSpec {
|
||||||
import FlowTransformRecoverSpec._
|
import FlowTransformRecoverSpec._
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 2)
|
||||||
maximumInputBufferSize = 2,
|
.withFanOutBuffer(initialSize = 2, maxSize = 2)
|
||||||
initialFanOutBufferSize = 2,
|
|
||||||
maxFanOutBufferSize = 2,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"A Flow with transformRecover operations" must {
|
"A Flow with transformRecover operations" must {
|
||||||
"produce one-to-one transformation as expected" in {
|
"produce one-to-one transformation as expected" in {
|
||||||
|
|
|
||||||
|
|
@ -15,12 +15,11 @@ import scala.util.control.NoStackTrace
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.receive=off\nakka.loglevel=INFO")) {
|
class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.receive=off\nakka.loglevel=INFO")) {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 2)
|
||||||
maximumInputBufferSize = 2,
|
.withFanOutBuffer(initialSize = 2, maxSize = 2)
|
||||||
initialFanOutBufferSize = 2,
|
|
||||||
maxFanOutBufferSize = 2,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"A Flow with transform operations" must {
|
"A Flow with transform operations" must {
|
||||||
"produce one-to-one transformation as expected" in {
|
"produce one-to-one transformation as expected" in {
|
||||||
|
|
|
||||||
|
|
@ -3,26 +3,26 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream
|
package akka.stream
|
||||||
|
|
||||||
import akka.actor.Actor
|
import akka.actor.{ Actor, Props }
|
||||||
import akka.actor.Props
|
|
||||||
import akka.pattern.pipe
|
import akka.pattern.pipe
|
||||||
import akka.stream.scaladsl.ImplicitFlowMaterializer
|
import akka.stream.scaladsl.{ Flow, ImplicitFlowMaterializer }
|
||||||
import akka.stream.scaladsl.Flow
|
|
||||||
import akka.stream.testkit.AkkaSpec
|
import akka.stream.testkit.AkkaSpec
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
|
|
||||||
object ImplicitFlowMaterializerSpec {
|
object ImplicitFlowMaterializerSpec {
|
||||||
class SomeActor(input: List[String]) extends Actor with ImplicitFlowMaterializer {
|
class SomeActor(input: List[String]) extends Actor with ImplicitFlowMaterializer {
|
||||||
|
|
||||||
override def flowMaterializerSettings = MaterializerSettings(dispatcher = "akka.test.stream-dispatcher")
|
override def flowMaterializerSettings = MaterializerSettings(context.system)
|
||||||
|
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||||
|
.withFanOutBuffer(initialSize = 1, maxSize = 16)
|
||||||
|
|
||||||
val flow = Flow(input).map(_.toUpperCase()).fold("")(_ + _)
|
val flow = Flow(input).map(_.toUpperCase()).fold("")(_ + _)
|
||||||
|
|
||||||
def receive = {
|
def receive = {
|
||||||
case "run" ⇒
|
case "run" ⇒
|
||||||
// toFuture takes an implicit FlowMaterializer parameter, which is provided by ImplicitFlowMaterializer
|
// toFuture takes an implicit FlowMaterializer parameter, which is provided by ImplicitFlowMaterializer
|
||||||
val futureResult = flow.toFuture()
|
|
||||||
import context.dispatcher
|
import context.dispatcher
|
||||||
|
val futureResult = flow.toFuture()
|
||||||
futureResult pipeTo sender()
|
futureResult pipeTo sender()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -30,7 +30,7 @@ object ImplicitFlowMaterializerSpec {
|
||||||
|
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class ImplicitFlowMaterializerSpec extends AkkaSpec with ImplicitSender {
|
class ImplicitFlowMaterializerSpec extends AkkaSpec with ImplicitSender {
|
||||||
import ImplicitFlowMaterializerSpec._
|
import akka.stream.ImplicitFlowMaterializerSpec._
|
||||||
|
|
||||||
"An ImplicitFlowMaterializer" must {
|
"An ImplicitFlowMaterializer" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -12,8 +12,7 @@ import scala.util.control.NoStackTrace
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class TickPublisherSpec extends AkkaSpec {
|
class TickPublisherSpec extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
implicit val materializer = FlowMaterializer()
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"A Flow based on tick publisher" must {
|
"A Flow based on tick publisher" must {
|
||||||
"produce ticks" in {
|
"produce ticks" in {
|
||||||
|
|
|
||||||
|
|
@ -11,12 +11,11 @@ import scala.util.control.NoStackTrace
|
||||||
|
|
||||||
abstract class TwoStreamsSetup extends AkkaSpec {
|
abstract class TwoStreamsSetup extends AkkaSpec {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 2)
|
||||||
maximumInputBufferSize = 2,
|
.withFanOutBuffer(initialSize = 2, maxSize = 2)
|
||||||
initialFanOutBufferSize = 2,
|
|
||||||
maxFanOutBufferSize = 2,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
case class TE(message: String) extends RuntimeException(message) with NoStackTrace
|
case class TE(message: String) extends RuntimeException(message) with NoStackTrace
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -3,20 +3,15 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream.actor
|
package akka.stream.actor
|
||||||
|
|
||||||
|
import akka.actor.{ ActorRef, PoisonPill, Props }
|
||||||
|
import akka.stream.{ MaterializerSettings, FlowMaterializer }
|
||||||
|
import akka.stream.scaladsl.Flow
|
||||||
|
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
|
||||||
|
import akka.testkit.TestEvent.Mute
|
||||||
|
import akka.testkit.{ EventFilter, ImplicitSender, TestProbe }
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.util.control.NoStackTrace
|
import scala.util.control.NoStackTrace
|
||||||
import akka.actor.ActorRef
|
|
||||||
import akka.actor.PoisonPill
|
|
||||||
import akka.actor.Props
|
|
||||||
import akka.stream.FlowMaterializer
|
|
||||||
import akka.stream.MaterializerSettings
|
|
||||||
import akka.stream.scaladsl.Flow
|
|
||||||
import akka.stream.testkit.AkkaSpec
|
|
||||||
import akka.stream.testkit.StreamTestKit
|
|
||||||
import akka.testkit.EventFilter
|
|
||||||
import akka.testkit.ImplicitSender
|
|
||||||
import akka.testkit.TestEvent.Mute
|
|
||||||
import akka.testkit.TestProbe
|
|
||||||
|
|
||||||
object ActorPublisherSpec {
|
object ActorPublisherSpec {
|
||||||
|
|
||||||
|
|
@ -30,7 +25,6 @@ object ActorPublisherSpec {
|
||||||
case object Complete
|
case object Complete
|
||||||
|
|
||||||
class TestPublisher(probe: ActorRef) extends ActorPublisher[String] {
|
class TestPublisher(probe: ActorRef) extends ActorPublisher[String] {
|
||||||
import ActorPublisher._
|
|
||||||
import ActorPublisherMessage._
|
import ActorPublisherMessage._
|
||||||
|
|
||||||
def receive = {
|
def receive = {
|
||||||
|
|
@ -90,7 +84,6 @@ object ActorPublisherSpec {
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class ActorPublisherSpec extends AkkaSpec with ImplicitSender {
|
class ActorPublisherSpec extends AkkaSpec with ImplicitSender {
|
||||||
import ActorPublisherSpec._
|
import ActorPublisherSpec._
|
||||||
import ActorPublisher._
|
|
||||||
|
|
||||||
system.eventStream.publish(Mute(EventFilter[IllegalStateException]()))
|
system.eventStream.publish(Mute(EventFilter[IllegalStateException]()))
|
||||||
|
|
||||||
|
|
@ -225,7 +218,7 @@ class ActorPublisherSpec extends AkkaSpec with ImplicitSender {
|
||||||
}
|
}
|
||||||
|
|
||||||
"work together with Flow and ActorSubscriber" in {
|
"work together with Flow and ActorSubscriber" in {
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(dispatcher = "akka.test.stream-dispatcher"))
|
implicit val materializer = FlowMaterializer()
|
||||||
val probe = TestProbe()
|
val probe = TestProbe()
|
||||||
val snd = system.actorOf(senderProps)
|
val snd = system.actorOf(senderProps)
|
||||||
val rcv = system.actorOf(receiverProps(probe.ref))
|
val rcv = system.actorOf(receiverProps(probe.ref))
|
||||||
|
|
|
||||||
|
|
@ -3,18 +3,14 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream.actor
|
package akka.stream.actor
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import akka.actor.{ Actor, ActorRef, Props }
|
||||||
import akka.actor.ActorRef
|
import akka.routing.{ ActorRefRoutee, RoundRobinRoutingLogic, Router }
|
||||||
import akka.actor.Props
|
|
||||||
import akka.stream.FlowMaterializer
|
|
||||||
import akka.stream.MaterializerSettings
|
|
||||||
import akka.stream.scaladsl.Flow
|
import akka.stream.scaladsl.Flow
|
||||||
import akka.stream.testkit.AkkaSpec
|
import akka.stream.testkit.AkkaSpec
|
||||||
import akka.actor.Actor
|
import akka.stream.{ FlowMaterializer, MaterializerSettings }
|
||||||
import akka.routing.ActorRefRoutee
|
|
||||||
import akka.routing.Router
|
|
||||||
import akka.routing.RoundRobinRoutingLogic
|
|
||||||
import akka.testkit.ImplicitSender
|
import akka.testkit.ImplicitSender
|
||||||
|
|
||||||
|
import scala.concurrent.duration._
|
||||||
import scala.util.control.NoStackTrace
|
import scala.util.control.NoStackTrace
|
||||||
|
|
||||||
object ActorSubscriberSpec {
|
object ActorSubscriberSpec {
|
||||||
|
|
@ -96,10 +92,10 @@ object ActorSubscriberSpec {
|
||||||
|
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class ActorSubscriberSpec extends AkkaSpec with ImplicitSender {
|
class ActorSubscriberSpec extends AkkaSpec with ImplicitSender {
|
||||||
import ActorSubscriberSpec._
|
|
||||||
import ActorSubscriberMessage._
|
import ActorSubscriberMessage._
|
||||||
|
import ActorSubscriberSpec._
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(dispatcher = "akka.test.stream-dispatcher"))
|
implicit val materializer = FlowMaterializer()
|
||||||
|
|
||||||
"An ActorSubscriber" must {
|
"An ActorSubscriber" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -3,24 +3,19 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream.extra
|
package akka.stream.extra
|
||||||
|
|
||||||
import akka.stream.testkit.{ StreamTestKit, ScriptedTest, AkkaSpec }
|
import akka.stream.{ MaterializerSettings, FlowMaterializer }
|
||||||
import akka.stream.{ FlowMaterializer, MaterializerSettings }
|
import akka.stream.scaladsl.{ Duct, Flow }
|
||||||
|
import akka.stream.testkit.{ AkkaSpec, ScriptedTest, StreamTestKit }
|
||||||
import akka.testkit.TestProbe
|
import akka.testkit.TestProbe
|
||||||
import akka.stream.scaladsl.{ Flow, Duct }
|
import org.reactivestreams.{ Publisher, Subscriber }
|
||||||
import org.reactivestreams.{ Subscriber, Publisher }
|
|
||||||
|
|
||||||
class FlowTimedSpec extends AkkaSpec with ScriptedTest {
|
class FlowTimedSpec extends AkkaSpec with ScriptedTest {
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
val settings = MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 16)
|
||||||
maximumInputBufferSize = 16,
|
.withFanOutBuffer(initialSize = 1, maxSize = 16)
|
||||||
initialFanOutBufferSize = 1,
|
|
||||||
maxFanOutBufferSize = 16,
|
|
||||||
dispatcher = "akka.test.stream-dispatcher")
|
|
||||||
|
|
||||||
lazy val metricsConfig = system.settings.config
|
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(settings)
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -109,12 +109,9 @@ object TcpFlowSpec {
|
||||||
class TcpFlowSpec extends AkkaSpec {
|
class TcpFlowSpec extends AkkaSpec {
|
||||||
import TcpFlowSpec._
|
import TcpFlowSpec._
|
||||||
|
|
||||||
val settings = MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 4,
|
.withInputBuffer(initialSize = 4, maxSize = 4)
|
||||||
maximumInputBufferSize = 4,
|
.withFanOutBuffer(initialSize = 2, maxSize = 2)
|
||||||
initialFanOutBufferSize = 2,
|
|
||||||
maxFanOutBufferSize = 2,
|
|
||||||
dispatcher = "akka.test.stream-dispatcher")
|
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(settings)
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
|
|
||||||
|
|
@ -182,7 +179,7 @@ class TcpFlowSpec extends AkkaSpec {
|
||||||
|
|
||||||
def connect(server: Server): (Processor[ByteString, ByteString], ServerConnection) = {
|
def connect(server: Server): (Processor[ByteString, ByteString], ServerConnection) = {
|
||||||
val tcpProbe = TestProbe()
|
val tcpProbe = TestProbe()
|
||||||
tcpProbe.send(IO(StreamTcp), StreamTcp.Connect(settings, server.address))
|
tcpProbe.send(IO(StreamTcp), StreamTcp.Connect(server.address, materializerSettings = Some(settings)))
|
||||||
val client = server.waitAccept()
|
val client = server.waitAccept()
|
||||||
val outgoingConnection = tcpProbe.expectMsgType[StreamTcp.OutgoingTcpConnection]
|
val outgoingConnection = tcpProbe.expectMsgType[StreamTcp.OutgoingTcpConnection]
|
||||||
|
|
||||||
|
|
@ -191,13 +188,13 @@ class TcpFlowSpec extends AkkaSpec {
|
||||||
|
|
||||||
def connect(serverAddress: InetSocketAddress): StreamTcp.OutgoingTcpConnection = {
|
def connect(serverAddress: InetSocketAddress): StreamTcp.OutgoingTcpConnection = {
|
||||||
val connectProbe = TestProbe()
|
val connectProbe = TestProbe()
|
||||||
connectProbe.send(IO(StreamTcp), StreamTcp.Connect(settings, serverAddress))
|
connectProbe.send(IO(StreamTcp), StreamTcp.Connect(serverAddress, materializerSettings = Some(settings)))
|
||||||
connectProbe.expectMsgType[StreamTcp.OutgoingTcpConnection]
|
connectProbe.expectMsgType[StreamTcp.OutgoingTcpConnection]
|
||||||
}
|
}
|
||||||
|
|
||||||
def bind(serverAddress: InetSocketAddress = temporaryServerAddress): StreamTcp.TcpServerBinding = {
|
def bind(serverAddress: InetSocketAddress = temporaryServerAddress): StreamTcp.TcpServerBinding = {
|
||||||
val bindProbe = TestProbe()
|
val bindProbe = TestProbe()
|
||||||
bindProbe.send(IO(StreamTcp), StreamTcp.Bind(settings, serverAddress))
|
bindProbe.send(IO(StreamTcp), StreamTcp.Bind(serverAddress, Some(settings)))
|
||||||
bindProbe.expectMsgType[StreamTcp.TcpServerBinding]
|
bindProbe.expectMsgType[StreamTcp.TcpServerBinding]
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -3,11 +3,11 @@
|
||||||
*/
|
*/
|
||||||
package akka.stream.scaladsl2
|
package akka.stream.scaladsl2
|
||||||
|
|
||||||
import org.scalatest.{ Matchers, WordSpec }
|
import akka.stream.MaterializerSettings
|
||||||
|
import akka.stream.testkit.AkkaSpec
|
||||||
|
|
||||||
import scala.collection.immutable.Seq
|
import scala.collection.immutable.Seq
|
||||||
import scala.concurrent.Future
|
import scala.concurrent.Future
|
||||||
import akka.stream.testkit.AkkaSpec
|
|
||||||
import akka.stream.MaterializerSettings
|
|
||||||
|
|
||||||
class FlowSpec extends AkkaSpec {
|
class FlowSpec extends AkkaSpec {
|
||||||
|
|
||||||
|
|
@ -16,7 +16,7 @@ class FlowSpec extends AkkaSpec {
|
||||||
|
|
||||||
import scala.concurrent.ExecutionContext.Implicits.global
|
import scala.concurrent.ExecutionContext.Implicits.global
|
||||||
val intFut = FutureIn(Future { 3 })
|
val intFut = FutureIn(Future { 3 })
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(dispatcher = "akka.test.stream-dispatcher"))
|
implicit val materializer = FlowMaterializer(MaterializerSettings(system))
|
||||||
|
|
||||||
"ProcessorFlow" should {
|
"ProcessorFlow" should {
|
||||||
"go through all states" in {
|
"go through all states" in {
|
||||||
|
|
|
||||||
|
|
@ -15,12 +15,11 @@ import akka.stream.MaterializerSettings
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.receive=off\nakka.loglevel=INFO")) {
|
class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.receive=off\nakka.loglevel=INFO")) {
|
||||||
|
|
||||||
implicit val materializer = FlowMaterializer(MaterializerSettings(
|
val settings = MaterializerSettings(system)
|
||||||
initialInputBufferSize = 2,
|
.withInputBuffer(initialSize = 2, maxSize = 2)
|
||||||
maximumInputBufferSize = 2,
|
.withFanOutBuffer(initialSize = 2, maxSize = 2)
|
||||||
initialFanOutBufferSize = 2,
|
|
||||||
maxFanOutBufferSize = 2,
|
implicit val materializer = FlowMaterializer(settings)
|
||||||
dispatcher = "akka.test.stream-dispatcher"))
|
|
||||||
|
|
||||||
"A Flow with transform operations" must {
|
"A Flow with transform operations" must {
|
||||||
"produce one-to-one transformation as expected" in {
|
"produce one-to-one transformation as expected" in {
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue