Merge pull request #16730 from akka/wip-15851-ActorBasedFlowMaterializer-patriknw

!str #15851 Rename FlowMaterializer and settings
This commit is contained in:
Patrik Nordwall 2015-01-29 10:30:08 +01:00
commit 580ba8c484
146 changed files with 601 additions and 595 deletions

View file

@ -19,12 +19,12 @@ Here's the data model we'll be working with throughout the quickstart examples:
Transforming and consuming simple streams Transforming and consuming simple streams
----------------------------------------- -----------------------------------------
In order to prepare our environment by creating an :class:`ActorSystem` and :class:`FlowMaterializer`, In order to prepare our environment by creating an :class:`ActorSystem` and :class:`ActorFlowMaterializer`,
which will be responsible for materializing and running the streams we are about to create: which will be responsible for materializing and running the streams we are about to create:
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/TwitterStreamQuickstartDocTest.java#materializer-setup .. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/TwitterStreamQuickstartDocTest.java#materializer-setup
The :class:`FlowMaterializer` can optionally take :class:`MaterializerSettings` which can be used to define The :class:`ActorFlowMaterializer` can optionally take :class:`ActorFlowMaterializerSettings` which can be used to define
materialization properties, such as default buffer sizes (see also :ref:`stream-buffering-explained-scala`), the dispatcher to materialization properties, such as default buffer sizes (see also :ref:`stream-buffering-explained-scala`), the dispatcher to
be used by the pipeline etc. These can be overridden on an element-by-element basis or for an entire section, but this be used by the pipeline etc. These can be overridden on an element-by-element basis or for an entire section, but this
will be discussed in depth in :ref:`stream-section-configuration`. will be discussed in depth in :ref:`stream-section-configuration`.
@ -55,8 +55,8 @@ or by using the shorthand version (which are defined only for the most popular s
.. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/TwitterStreamQuickstartDocTest.java#authors-foreach-println .. includecode:: ../../../akka-samples/akka-docs-java-lambda/src/test/java/docs/stream/TwitterStreamQuickstartDocTest.java#authors-foreach-println
Materializing and running a stream always requires a :class:`FlowMaterializer` to be in implicit scope (or passed in explicitly, Materializing and running a stream always requires a :class:`FlowMaterializer` to be passed in explicitly,
like this: ``.run(mat)``). like this: ``.run(mat)``.
Flattening sequences in streams Flattening sequences in streams
------------------------------- -------------------------------
@ -141,7 +141,7 @@ First, we prepare the :class:`FoldSink` which will be used to sum all ``Integer`
Next we connect the ``tweets`` stream though a ``map`` step which converts each tweet into the number ``1``, Next we connect the ``tweets`` stream though a ``map`` step which converts each tweet into the number ``1``,
finally we connect the flow ``to`` the previously prepared Sink. Notice that this step does *not* yet materialize the finally we connect the flow ``to`` the previously prepared Sink. Notice that this step does *not* yet materialize the
processing pipeline, it merely prepares the description of the Flow, which is now connected to a Sink, and therefore can processing pipeline, it merely prepares the description of the Flow, which is now connected to a Sink, and therefore can
be ``run()``, as indicated by its type: :class:`RunnableFlow`. Next we call ``run()`` which uses the implicit :class:`FlowMaterializer` be ``run()``, as indicated by its type: :class:`RunnableFlow`. Next we call ``run()`` which uses the implicit :class:`ActorFlowMaterializer`
to materialize and run the flow. The value returned by calling ``run()`` on a ``RunnableFlow`` or ``FlowGraph`` is ``MaterializedMap``, to materialize and run the flow. The value returned by calling ``run()`` on a ``RunnableFlow`` or ``FlowGraph`` is ``MaterializedMap``,
which can be used to retrieve materialized values from the running stream. which can be used to retrieve materialized values from the running stream.

View file

@ -16,10 +16,10 @@ class HttpServerExampleSpec
"binding example" in { "binding example" in {
//#bind-example //#bind-example
import akka.http.Http import akka.http.Http
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
implicit val system = ActorSystem() implicit val system = ActorSystem()
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
val serverBinding = Http(system).bind(interface = "localhost", port = 8080) val serverBinding = Http(system).bind(interface = "localhost", port = 8080)
serverBinding.connections.runForeach { connection => // foreach materializes the source serverBinding.connections.runForeach { connection => // foreach materializes the source
@ -30,10 +30,10 @@ class HttpServerExampleSpec
"full-server-example" in { "full-server-example" in {
import akka.http.Http import akka.http.Http
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
implicit val system = ActorSystem() implicit val system = ActorSystem()
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
val serverBinding = Http(system).bind(interface = "localhost", port = 8080) val serverBinding = Http(system).bind(interface = "localhost", port = 8080)

View file

@ -6,7 +6,7 @@ package docs.http.server
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.http.server.Route import akka.http.server.Route
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
object MyHandler { object MyHandler {
//# example-1 //# example-1
@ -27,7 +27,7 @@ object MyHandler {
object MyApp { object MyApp {
implicit val system = ActorSystem() implicit val system = ActorSystem()
import system.dispatcher import system.dispatcher
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
def handler = Route.handlerFlow(`<my-route-definition>`) def handler = Route.handlerFlow(`<my-route-definition>`)
} }

View file

@ -5,7 +5,7 @@
package docs.http.server package docs.http.server
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.http.server.{ Route, MissingCookieRejection } import akka.http.server.{ Route, MissingCookieRejection }
@ -26,7 +26,7 @@ object MyRejectionHandler {
object MyApp { object MyApp {
implicit val system = ActorSystem() implicit val system = ActorSystem()
import system.dispatcher import system.dispatcher
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
def handler = Route.handlerFlow(`<my-route-definition>`) def handler = Route.handlerFlow(`<my-route-definition>`)
} }

View file

@ -5,7 +5,7 @@ package docs.stream
import scala.annotation.tailrec import scala.annotation.tailrec
import akka.actor.Props import akka.actor.Props
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.actor.ActorPublisher import akka.stream.actor.ActorPublisher
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
@ -70,7 +70,7 @@ object ActorPublisherDocSpec {
class ActorPublisherDocSpec extends AkkaSpec { class ActorPublisherDocSpec extends AkkaSpec {
import ActorPublisherDocSpec._ import ActorPublisherDocSpec._
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
"illustrate usage of ActorPublisher" in { "illustrate usage of ActorPublisher" in {
def println(s: String): Unit = def println(s: String): Unit =

View file

@ -9,7 +9,7 @@ import akka.actor.Props
import akka.routing.ActorRefRoutee import akka.routing.ActorRefRoutee
import akka.routing.RoundRobinRoutingLogic import akka.routing.RoundRobinRoutingLogic
import akka.routing.Router import akka.routing.Router
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.actor.ActorSubscriber import akka.stream.actor.ActorSubscriber
import akka.stream.actor.ActorSubscriberMessage import akka.stream.actor.ActorSubscriberMessage
import akka.stream.actor.MaxInFlightRequestStrategy import akka.stream.actor.MaxInFlightRequestStrategy
@ -72,7 +72,7 @@ object ActorSubscriberDocSpec {
class ActorSubscriberDocSpec extends AkkaSpec { class ActorSubscriberDocSpec extends AkkaSpec {
import ActorSubscriberDocSpec._ import ActorSubscriberDocSpec._
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
"illustrate usage of ActorSubscriber" in { "illustrate usage of ActorSubscriber" in {
val replyTo = testActor val replyTo = testActor

View file

@ -3,7 +3,7 @@
*/ */
package docs.stream package docs.stream
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
@ -15,7 +15,7 @@ import scala.util.control.NoStackTrace
class FlexiDocSpec extends AkkaSpec { class FlexiDocSpec extends AkkaSpec {
implicit val ec = system.dispatcher implicit val ec = system.dispatcher
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
"implement zip using readall" in { "implement zip using readall" in {
//#fleximerge-zip-readall //#fleximerge-zip-readall

View file

@ -14,10 +14,10 @@ class FlowDocSpec extends AkkaSpec {
implicit val ec = system.dispatcher implicit val ec = system.dispatcher
//#imports //#imports
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
//#imports //#imports
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
"source is immutable" in { "source is immutable" in {
//#source-immutable //#source-immutable

View file

@ -3,7 +3,7 @@
*/ */
package docs.stream package docs.stream
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Broadcast import akka.stream.scaladsl.Broadcast
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Flow
import akka.stream.scaladsl.FlowGraph import akka.stream.scaladsl.FlowGraph
@ -22,7 +22,7 @@ class FlowGraphDocSpec extends AkkaSpec {
implicit val ec = system.dispatcher implicit val ec = system.dispatcher
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
"build simple graph" in { "build simple graph" in {
//format: OFF //format: OFF

View file

@ -1,6 +1,6 @@
package docs.stream package docs.stream
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.{ RunnableFlow, Sink, Source, Flow } import akka.stream.scaladsl.{ RunnableFlow, Sink, Source, Flow }
import akka.stream.stage.PushPullStage import akka.stream.stage.PushPullStage
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
@ -14,7 +14,7 @@ class FlowStagesSpec extends AkkaSpec {
import akka.stream.stage._ import akka.stream.stage._
//#import-stage //#import-stage
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
"stages demo" must { "stages demo" must {

View file

@ -1,12 +1,12 @@
package docs.stream package docs.stream
import akka.stream.{ OverflowStrategy, FlowMaterializer } import akka.stream.{ OverflowStrategy, ActorFlowMaterializer }
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
class GraphCyclesSpec extends AkkaSpec { class GraphCyclesSpec extends AkkaSpec {
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
"Cycle demonstration" must { "Cycle demonstration" must {
val source = Source(() => Iterator.from(0)) val source = Source(() => Iterator.from(0))

View file

@ -7,7 +7,7 @@ import scala.concurrent.duration._
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import java.util.Date import java.util.Date
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import scala.concurrent.Future import scala.concurrent.Future
import akka.stream.scaladsl.RunnableFlow import akka.stream.scaladsl.RunnableFlow
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
@ -20,7 +20,7 @@ import akka.pattern.ask
import akka.util.Timeout import akka.util.Timeout
import akka.stream.scaladsl.OperationAttributes import akka.stream.scaladsl.OperationAttributes
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
object IntegrationDocSpec { object IntegrationDocSpec {
@ -117,7 +117,7 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
import TwitterStreamQuickstartDocSpec._ import TwitterStreamQuickstartDocSpec._
import IntegrationDocSpec._ import IntegrationDocSpec._
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
"calling external service with mapAsync" in { "calling external service with mapAsync" in {
val probe = TestProbe() val probe = TestProbe()
@ -301,8 +301,8 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
implicit val blockingExecutionContext = system.dispatchers.lookup("blocking-dispatcher") implicit val blockingExecutionContext = system.dispatchers.lookup("blocking-dispatcher")
val service = new SometimesSlowService val service = new SometimesSlowService
implicit val mat = FlowMaterializer( implicit val mat = ActorFlowMaterializer(
MaterializerSettings(system).withInputBuffer(initialSize = 4, maxSize = 4)) ActorFlowMaterializerSettings(system).withInputBuffer(initialSize = 4, maxSize = 4))
Source(List("a", "B", "C", "D", "e", "F", "g", "H", "i", "J")) Source(List("a", "B", "C", "D", "e", "F", "g", "H", "i", "J"))
.map(elem => { println(s"before: $elem"); elem }) .map(elem => { println(s"before: $elem"); elem })
@ -333,8 +333,8 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
implicit val blockingExecutionContext = system.dispatchers.lookup("blocking-dispatcher") implicit val blockingExecutionContext = system.dispatchers.lookup("blocking-dispatcher")
val service = new SometimesSlowService val service = new SometimesSlowService
implicit val mat = FlowMaterializer( implicit val mat = ActorFlowMaterializer(
MaterializerSettings(system).withInputBuffer(initialSize = 4, maxSize = 4)) ActorFlowMaterializerSettings(system).withInputBuffer(initialSize = 4, maxSize = 4))
Source(List("a", "B", "C", "D", "e", "F", "g", "H", "i", "J")) Source(List("a", "B", "C", "D", "e", "F", "g", "H", "i", "J"))
.map(elem => { println(s"before: $elem"); elem }) .map(elem => { println(s"before: $elem"); elem })

View file

@ -3,7 +3,7 @@
*/ */
package docs.stream package docs.stream
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Flow
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
@ -13,7 +13,7 @@ import akka.stream.scaladsl.Source
class ReactiveStreamsDocSpec extends AkkaSpec { class ReactiveStreamsDocSpec extends AkkaSpec {
import TwitterStreamQuickstartDocSpec._ import TwitterStreamQuickstartDocSpec._
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
//#imports //#imports
import org.reactivestreams.Publisher import org.reactivestreams.Publisher

View file

@ -1,11 +1,11 @@
package docs.stream package docs.stream
import akka.stream.{ OverflowStrategy, MaterializerSettings, FlowMaterializer } import akka.stream.{ OverflowStrategy, ActorFlowMaterializerSettings, ActorFlowMaterializer }
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
class StreamBuffersRateSpec extends AkkaSpec { class StreamBuffersRateSpec extends AkkaSpec {
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
"Demonstrate pipelining" in { "Demonstrate pipelining" in {
def println(s: Any) = () def println(s: Any) = ()
@ -20,8 +20,8 @@ class StreamBuffersRateSpec extends AkkaSpec {
"Demonstrate buffer sizes" in { "Demonstrate buffer sizes" in {
//#materializer-buffer //#materializer-buffer
val materializer = FlowMaterializer( val materializer = ActorFlowMaterializer(
MaterializerSettings(system) ActorFlowMaterializerSettings(system)
.withInputBuffer( .withInputBuffer(
initialSize = 64, initialSize = 64,
maxSize = 64)) maxSize = 64))

View file

@ -3,7 +3,7 @@
*/ */
package docs.stream package docs.stream
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Broadcast import akka.stream.scaladsl.Broadcast
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Flow
import akka.stream.scaladsl.FlowGraph import akka.stream.scaladsl.FlowGraph
@ -25,7 +25,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
implicit val ec = system.dispatcher implicit val ec = system.dispatcher
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
"build with open ports" in { "build with open ports" in {
// format: OFF // format: OFF

View file

@ -21,12 +21,12 @@ class StreamTcpDocSpec extends AkkaSpec {
implicit val ec = system.dispatcher implicit val ec = system.dispatcher
//#setup //#setup
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.StreamTcp import akka.stream.scaladsl.StreamTcp
import akka.stream.scaladsl.StreamTcp._ import akka.stream.scaladsl.StreamTcp._
implicit val sys = ActorSystem("stream-tcp-system") implicit val sys = ActorSystem("stream-tcp-system")
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
//#setup //#setup
val localhost = new InetSocketAddress("127.0.0.1", 8888) val localhost = new InetSocketAddress("127.0.0.1", 8888)

View file

@ -6,7 +6,7 @@ package docs.stream
//#imports //#imports
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.OverflowStrategy import akka.stream.OverflowStrategy
import akka.stream.scaladsl.Broadcast import akka.stream.scaladsl.Broadcast
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Flow
@ -66,11 +66,11 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
trait Example1 { trait Example1 {
//#materializer-setup //#materializer-setup
implicit val system = ActorSystem("reactive-tweets") implicit val system = ActorSystem("reactive-tweets")
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
//#materializer-setup //#materializer-setup
} }
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
"filter and map" in { "filter and map" in {
//#authors-filter-map //#authors-filter-map

View file

@ -1,6 +1,6 @@
package docs.stream.cookbook package docs.stream.cookbook
import akka.stream.{ MaterializerSettings, FlowMaterializer } import akka.stream.{ ActorFlowMaterializerSettings, ActorFlowMaterializer }
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.testkit.StreamTestKit import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.{ SubscriberProbe, PublisherProbe } import akka.stream.testkit.StreamTestKit.{ SubscriberProbe, PublisherProbe }
@ -11,7 +11,7 @@ import scala.concurrent.duration._
class RecipeCollectingMetrics extends RecipeSpec { class RecipeCollectingMetrics extends RecipeSpec {
import HoldOps._ import HoldOps._
implicit val m2 = FlowMaterializer(MaterializerSettings(system).withInputBuffer(1, 1)) implicit val m2 = ActorFlowMaterializer(ActorFlowMaterializerSettings(system).withInputBuffer(1, 1))
"Recipe for periodically collecting metrics" must { "Recipe for periodically collecting metrics" must {

View file

@ -1,11 +1,11 @@
package docs.stream.cookbook package docs.stream.cookbook
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
trait RecipeSpec extends AkkaSpec { trait RecipeSpec extends AkkaSpec {
implicit val m = FlowMaterializer() implicit val m = ActorFlowMaterializer()
type Message = String type Message = String
type Trigger = Unit type Trigger = Unit
type Job = String type Job = String

View file

@ -190,7 +190,7 @@ well-known sinks, such as ``runForeach(el => )`` (being an alias to ``runWith(Si
Materialization is currently performed synchronously on the materializing thread. Materialization is currently performed synchronously on the materializing thread.
Tha actual stream processing is handled by :ref:`Actors actor-scala` started up during the streams materialization, Tha actual stream processing is handled by :ref:`Actors actor-scala` started up during the streams materialization,
which will be running on the thread pools they have been configured to run on - which defaults to the dispatcher set in which will be running on the thread pools they have been configured to run on - which defaults to the dispatcher set in
:class:`MaterializationSettings` while constructing the :class:`FlowMaterializer`. :class:`MaterializationSettings` while constructing the :class:`ActorFlowMaterializer`.
.. note:: .. note::
Reusing *instances* of linear computation stages (Source, Sink, Flow) inside FlowGraphs is legal, Reusing *instances* of linear computation stages (Source, Sink, Flow) inside FlowGraphs is legal,

View file

@ -254,7 +254,7 @@ is ``completed`` before ``g``, but still emitted afterwards.
The numbers in parenthesis illustrates how many calls that are in progress at The numbers in parenthesis illustrates how many calls that are in progress at
the same time. Here the downstream demand and thereby the number of concurrent the same time. Here the downstream demand and thereby the number of concurrent
calls are limited by the buffer size (4) of the :class:`MaterializerSettings`. calls are limited by the buffer size (4) of the :class:`ActorFlowMaterializerSettings`.
Here is how we can use the same service with ``mapAsyncUnordered``: Here is how we can use the same service with ``mapAsyncUnordered``:
@ -310,7 +310,7 @@ Note that ``after`` lines are not in the same order as the ``before`` lines. For
The numbers in parenthesis illustrates how many calls that are in progress at The numbers in parenthesis illustrates how many calls that are in progress at
the same time. Here the downstream demand and thereby the number of concurrent the same time. Here the downstream demand and thereby the number of concurrent
calls are limited by the buffer size (4) of the :class:`MaterializerSettings`. calls are limited by the buffer size (4) of the :class:`ActorFlowMaterializerSettings`.
.. _reactive-streams-integration-scala: .. _reactive-streams-integration-scala:

View file

@ -19,12 +19,12 @@ Here's the data model we'll be working with throughout the quickstart examples:
Transforming and consuming simple streams Transforming and consuming simple streams
----------------------------------------- -----------------------------------------
In order to prepare our environment by creating an :class:`ActorSystem` and :class:`FlowMaterializer`, In order to prepare our environment by creating an :class:`ActorSystem` and :class:`ActorFlowMaterializer`,
which will be responsible for materializing and running the streams we are about to create: which will be responsible for materializing and running the streams we are about to create:
.. includecode:: code/docs/stream/TwitterStreamQuickstartDocSpec.scala#materializer-setup .. includecode:: code/docs/stream/TwitterStreamQuickstartDocSpec.scala#materializer-setup
The :class:`FlowMaterializer` can optionally take :class:`MaterializerSettings` which can be used to define The :class:`ActorFlowMaterializer` can optionally take :class:`ActorFlowMaterializerSettings` which can be used to define
materialization properties, such as default buffer sizes (see also :ref:`stream-buffering-explained-scala`), the dispatcher to materialization properties, such as default buffer sizes (see also :ref:`stream-buffering-explained-scala`), the dispatcher to
be used by the pipeline etc. These can be overridden on an element-by-element basis or for an entire section, but this be used by the pipeline etc. These can be overridden on an element-by-element basis or for an entire section, but this
will be discussed in depth in :ref:`stream-section-configuration`. will be discussed in depth in :ref:`stream-section-configuration`.
@ -146,7 +146,7 @@ First, we prepare the :class:`FoldSink` which will be used to sum all ``Int`` el
Next we connect the ``tweets`` stream though a ``map`` step which converts each tweet into the number ``1``, Next we connect the ``tweets`` stream though a ``map`` step which converts each tweet into the number ``1``,
finally we connect the flow ``to`` the previously prepared Sink. Notice that this step does *not* yet materialize the finally we connect the flow ``to`` the previously prepared Sink. Notice that this step does *not* yet materialize the
processing pipeline, it merely prepares the description of the Flow, which is now connected to a Sink, and therefore can processing pipeline, it merely prepares the description of the Flow, which is now connected to a Sink, and therefore can
be ``run()``, as indicated by its type: :class:`RunnableFlow`. Next we call ``run()`` which uses the implicit :class:`FlowMaterializer` be ``run()``, as indicated by its type: :class:`RunnableFlow`. Next we call ``run()`` which uses the implicit :class:`ActorFlowMaterializer`
to materialize and run the flow. The value returned by calling ``run()`` on a ``RunnableFlow`` or ``FlowGraph`` is ``MaterializedMap``, to materialize and run the flow. The value returned by calling ``run()`` on a ``RunnableFlow`` or ``FlowGraph`` is ``MaterializedMap``,
which can be used to retrieve materialized values from the running stream. which can be used to retrieve materialized values from the running stream.

View file

@ -62,7 +62,7 @@ to a level that throughput requirements of the application require. Default buff
akka.stream.materializer.max-input-buffer-size = 16 akka.stream.materializer.max-input-buffer-size = 16
Alternatively they can be set by passing a :class:`MaterializerSettings` to the materializer: Alternatively they can be set by passing a :class:`ActorFlowMaterializerSettings` to the materializer:
.. includecode:: code/docs/stream/StreamBuffersRateSpec.scala#materializer-buffer .. includecode:: code/docs/stream/StreamBuffersRateSpec.scala#materializer-buffer

View file

@ -6,7 +6,6 @@ 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(){}

View file

@ -5,7 +5,7 @@
package akka.http.engine.rendering package akka.http.engine.rendering
import akka.parboiled2.CharUtils import akka.parboiled2.CharUtils
import akka.stream.impl.ActorBasedFlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.util.ByteString import akka.util.ByteString
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.stream.scaladsl._ import akka.stream.scaladsl._
@ -34,7 +34,7 @@ private object RenderSupport {
// allows us to not take a FlowMaterializer but delegate the cancellation to the point when the whole stream // allows us to not take a FlowMaterializer but delegate the cancellation to the point when the whole stream
// materializes // materializes
private case class CancelSecond[T](first: Source[T], second: Source[T]) extends SimpleActorFlowSource[T] { private case class CancelSecond[T](first: Source[T], second: Source[T]) extends SimpleActorFlowSource[T] {
override def attach(flowSubscriber: Subscriber[T], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = { override def attach(flowSubscriber: Subscriber[T], materializer: ActorFlowMaterializer, flowName: String): Unit = {
first.to(Sink(flowSubscriber)).run()(materializer) first.to(Sink(flowSubscriber)).run()(materializer)
second.to(Sink.cancelled).run()(materializer) second.to(Sink.cancelled).run()(materializer)
} }

View file

@ -5,7 +5,6 @@
package akka.http.model.japi package akka.http.model.japi
import akka.http.model import akka.http.model
import akka.stream.MaterializerSettings
/** /**
* INTERNAL API * INTERNAL API

View file

@ -14,6 +14,7 @@ import scala.concurrent.{ ExecutionContext, Future }
import scala.util.Try import scala.util.Try
import akka.actor.Props import akka.actor.Props
import akka.http.model.RequestEntity import akka.http.model.RequestEntity
import akka.stream.ActorFlowMaterializer
import akka.stream.FlowMaterializer import akka.stream.FlowMaterializer
import akka.stream.impl.Ast.AstNode import akka.stream.impl.Ast.AstNode
import akka.stream.impl.Ast.StageFactory import akka.stream.impl.Ast.StageFactory
@ -162,7 +163,7 @@ private[http] object StreamUtils {
def fromInputStreamSource(inputStream: InputStream, defaultChunkSize: Int = 65536): Source[ByteString] = { def fromInputStreamSource(inputStream: InputStream, defaultChunkSize: Int = 65536): Source[ByteString] = {
import akka.stream.impl._ import akka.stream.impl._
def props(materializer: ActorBasedFlowMaterializer): Props = { def props(materializer: ActorFlowMaterializer): Props = {
val iterator = new Iterator[ByteString] { val iterator = new Iterator[ByteString] {
var finished = false var finished = false
def hasNext: Boolean = !finished def hasNext: Boolean = !finished
@ -182,11 +183,11 @@ private[http] object StreamUtils {
} }
new AtomicBoolean(false) with SimpleActorFlowSource[ByteString] { new AtomicBoolean(false) with SimpleActorFlowSource[ByteString] {
override def attach(flowSubscriber: Subscriber[ByteString], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = override def attach(flowSubscriber: Subscriber[ByteString], materializer: ActorFlowMaterializer, flowName: String): Unit =
create(materializer, flowName)._1.subscribe(flowSubscriber) create(materializer, flowName)._1.subscribe(flowSubscriber)
override def isActive: Boolean = true override def isActive: Boolean = true
override def create(materializer: ActorBasedFlowMaterializer, flowName: String): (Publisher[ByteString], Unit) = override def create(materializer: ActorFlowMaterializer, flowName: String): (Publisher[ByteString], Unit) =
if (!getAndSet(true)) { if (!getAndSet(true)) {
val ref = materializer.actorOf(props(materializer), name = s"$flowName-0-InputStream-source") val ref = materializer.actorOf(props(materializer), name = s"$flowName-0-InputStream-source")
val publisher = ActorPublisher[ByteString](ref) val publisher = ActorPublisher[ByteString](ref)
@ -204,10 +205,10 @@ private[http] object StreamUtils {
import akka.stream.impl._ import akka.stream.impl._
val original = other.asInstanceOf[ActorFlowSource[T]] val original = other.asInstanceOf[ActorFlowSource[T]]
new AtomicBoolean(false) with SimpleActorFlowSource[T] { new AtomicBoolean(false) with SimpleActorFlowSource[T] {
override def attach(flowSubscriber: Subscriber[T], materializer: ActorBasedFlowMaterializer, flowName: String): Unit = override def attach(flowSubscriber: Subscriber[T], materializer: ActorFlowMaterializer, flowName: String): Unit =
create(materializer, flowName)._1.subscribe(flowSubscriber) create(materializer, flowName)._1.subscribe(flowSubscriber)
override def isActive: Boolean = true override def isActive: Boolean = true
override def create(materializer: ActorBasedFlowMaterializer, flowName: String): (Publisher[T], Unit) = override def create(materializer: ActorFlowMaterializer, flowName: String): (Publisher[T], Unit) =
if (!getAndSet(true)) (original.create(materializer, flowName)._1, ()) if (!getAndSet(true)) (original.create(materializer, flowName)._1, ())
else (ErrorPublisher(new IllegalStateException("One time source can only be instantiated once"), "failed").asInstanceOf[Publisher[T]], ()) else (ErrorPublisher(new IllegalStateException("One time source can only be instantiated once"), "failed").asInstanceOf[Publisher[T]], ())
} }

View file

@ -14,7 +14,7 @@ import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.scaladsl.StreamTcp import akka.stream.scaladsl.StreamTcp
import akka.stream.BindFailedException import akka.stream.BindFailedException
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.StreamTestKit import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.{ PublisherProbe, SubscriberProbe } import akka.stream.testkit.StreamTestKit.{ PublisherProbe, SubscriberProbe }
import akka.stream.scaladsl._ import akka.stream.scaladsl._
@ -34,7 +34,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
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
"The low-level HTTP infrastructure" should { "The low-level HTTP infrastructure" should {

View file

@ -7,7 +7,7 @@ package akka.http
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import scala.util.{ Failure, Success } import scala.util.{ Failure, Success }
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.{ Sink, Source } import akka.stream.scaladsl.{ Sink, Source }
import akka.http.model._ import akka.http.model._
@ -17,7 +17,7 @@ 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)
implicit val fm = FlowMaterializer() implicit val fm = ActorFlowMaterializer()
import system.dispatcher import system.dispatcher
val host = "spray.io" val host = "spray.io"

View file

@ -6,7 +6,7 @@ package akka.http
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.http.model._ import akka.http.model._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Flow
import com.typesafe.config.{ ConfigFactory, Config } import com.typesafe.config.{ ConfigFactory, Config }
import HttpMethods._ import HttpMethods._
@ -17,7 +17,7 @@ object TestServer extends App {
akka.log-dead-letters = off akka.log-dead-letters = off
""") """)
implicit val system = ActorSystem("ServerTest", testConf) implicit val system = ActorSystem("ServerTest", testConf)
implicit val fm = FlowMaterializer() implicit val fm = ActorFlowMaterializer()
val binding = Http().bind(interface = "localhost", port = 8080) val binding = Http().bind(interface = "localhost", port = 8080)

View file

@ -8,7 +8,7 @@ import java.net.InetSocketAddress
import org.scalatest.Inside import org.scalatest.Inside
import akka.util.ByteString import akka.util.ByteString
import akka.event.NoLogging import akka.event.NoLogging
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.http.model.HttpEntity._ import akka.http.model.HttpEntity._
@ -18,7 +18,7 @@ import akka.http.model.headers._
import akka.http.util._ import akka.http.util._
class HttpClientSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF") with Inside { class HttpClientSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF") with Inside {
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
"The client implementation" should { "The client implementation" should {

View file

@ -12,7 +12,7 @@ import org.scalatest.matchers.Matcher
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.scaladsl.OperationAttributes._ import akka.stream.scaladsl.OperationAttributes._
import akka.stream.FlattenStrategy import akka.stream.FlattenStrategy
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.util.ByteString import akka.util.ByteString
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.http.util._ import akka.http.util._
@ -37,7 +37,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
import system.dispatcher import system.dispatcher
val BOLT = HttpMethod.custom("BOLT", safe = false, idempotent = true, entityAccepted = true) val BOLT = HttpMethod.custom("BOLT", safe = false, idempotent = true, entityAccepted = true)
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
"The request parsing logic should" - { "The request parsing logic should" - {
"properly parse a request" - { "properly parse a request" - {

View file

@ -12,7 +12,7 @@ import org.scalatest.matchers.Matcher
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.scaladsl.OperationAttributes._ import akka.stream.scaladsl.OperationAttributes._
import akka.stream.FlattenStrategy import akka.stream.FlattenStrategy
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.util.ByteString import akka.util.ByteString
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.http.util._ import akka.http.util._
@ -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
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
val ServerOnTheMove = StatusCodes.custom(331, "Server on the move") val ServerOnTheMove = StatusCodes.custom(331, "Server on the move")
"The response parsing logic should" - { "The response parsing logic should" - {

View file

@ -17,7 +17,7 @@ import akka.http.model.headers._
import akka.http.util._ import akka.http.util._
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.scaladsl.OperationAttributes._ import akka.stream.scaladsl.OperationAttributes._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.impl.SynchronousIterablePublisher import akka.stream.impl.SynchronousIterablePublisher
import HttpEntity._ import HttpEntity._
import HttpMethods._ import HttpMethods._
@ -29,7 +29,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
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
"The request preparation logic should" - { "The request preparation logic should" - {
"properly render an unchunked" - { "properly render an unchunked" - {

View file

@ -17,7 +17,7 @@ import akka.http.util._
import akka.util.ByteString import akka.util.ByteString
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.scaladsl.OperationAttributes._ import akka.stream.scaladsl.OperationAttributes._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import HttpEntity._ import HttpEntity._
class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll { class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
@ -28,7 +28,7 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll
import system.dispatcher import system.dispatcher
val ServerOnTheMove = StatusCodes.custom(330, "Server on the move") val ServerOnTheMove = StatusCodes.custom(330, "Server on the move")
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
"The response preparation logic should properly render" - { "The response preparation logic should properly render" - {
"a response with no body," - { "a response with no body," - {

View file

@ -11,7 +11,7 @@ import org.scalatest.Inside
import akka.event.NoLogging import akka.event.NoLogging
import akka.util.ByteString import akka.util.ByteString
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.http.model._ import akka.http.model._
import akka.http.util._ import akka.http.util._
@ -21,7 +21,7 @@ import MediaTypes._
import HttpMethods._ import HttpMethods._
class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF") with Inside { class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF") with Inside {
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
"The server implementation" should { "The server implementation" should {

View file

@ -15,7 +15,7 @@ import org.scalatest.matchers.Matcher
import akka.util.ByteString import akka.util.ByteString
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.http.model.HttpEntity._ import akka.http.model.HttpEntity._
import akka.http.util.StreamUtils import akka.http.util.StreamUtils
@ -32,7 +32,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
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
override def afterAll() = system.shutdown() override def afterAll() = system.shutdown()
"HttpEntity" - { "HttpEntity" - {

View file

@ -8,7 +8,7 @@ import com.typesafe.config.{ Config, ConfigFactory }
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import org.scalatest.{ BeforeAndAfterAll, Inside, Matchers, WordSpec } import org.scalatest.{ BeforeAndAfterAll, Inside, Matchers, WordSpec }
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.util.ByteString import akka.util.ByteString
import akka.actor.ActorSystem import akka.actor.ActorSystem
@ -21,7 +21,7 @@ class MultipartSpec extends WordSpec with Matchers with Inside with BeforeAndAft
akka.loglevel = WARNING""") akka.loglevel = WARNING""")
implicit val system = ActorSystem(getClass.getSimpleName, testConf) implicit val system = ActorSystem(getClass.getSimpleName, testConf)
import system.dispatcher import system.dispatcher
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
override def afterAll() = system.shutdown() override def afterAll() = system.shutdown()
"Multipart.General" should { "Multipart.General" should {

View file

@ -14,7 +14,7 @@ import akka.http.model.parser.HeaderParser
import akka.http.model._ import akka.http.model._
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import headers._ import headers._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
/** /**
* Integration test for external HTTP libraries that are built on top of * Integration test for external HTTP libraries that are built on top of
@ -43,7 +43,7 @@ class HttpModelIntegrationSpec extends WordSpec with Matchers with BeforeAndAfte
override def afterAll() = system.shutdown() override def afterAll() = system.shutdown()
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
"External HTTP libraries" should { "External HTTP libraries" should {

View file

@ -11,6 +11,7 @@ import scala.concurrent.duration._
import scala.util.DynamicVariable import scala.util.DynamicVariable
import scala.reflect.ClassTag import scala.reflect.ClassTag
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.FlowMaterializer import akka.stream.FlowMaterializer
import akka.http.client.RequestBuilding import akka.http.client.RequestBuilding
import akka.http.util.FastFuture import akka.http.util.FastFuture
@ -41,7 +42,7 @@ trait RouteTest extends RequestBuilding with RouteTestResultComponent with Marsh
} }
implicit val system = createActorSystem() implicit val system = createActorSystem()
implicit def executor = system.dispatcher implicit def executor = system.dispatcher
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
def cleanUp(): Unit = system.shutdown() def cleanUp(): Unit = system.shutdown()
@ -141,4 +142,4 @@ trait RouteTest extends RequestBuilding with RouteTestResultComponent with Marsh
} }
} }
//FIXME: trait Specs2RouteTest extends RouteTest with Specs2Interface //FIXME: trait Specs2RouteTest extends RouteTest with Specs2Interface

View file

@ -8,14 +8,14 @@ import scala.concurrent.duration._
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec } import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
import org.scalatest.concurrent.ScalaFutures import org.scalatest.concurrent.ScalaFutures
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.http.unmarshalling.Unmarshal import akka.http.unmarshalling.Unmarshal
import akka.http.marshalling.Marshal import akka.http.marshalling.Marshal
import akka.http.model._ import akka.http.model._
class FormDataSpec extends WordSpec with Matchers with ScalaFutures with BeforeAndAfterAll { class FormDataSpec extends WordSpec with Matchers with ScalaFutures with BeforeAndAfterAll {
implicit val system = ActorSystem(getClass.getSimpleName) implicit val system = ActorSystem(getClass.getSimpleName)
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
import system.dispatcher import system.dispatcher
val formData = FormData(Map("surname" -> "Smith", "age" -> "42")) val formData = FormData(Map("surname" -> "Smith", "age" -> "42"))

View file

@ -9,7 +9,7 @@ import org.scalatest.{ Suite, BeforeAndAfterAll, Matchers }
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.util.ByteString import akka.util.ByteString
trait CodecSpecSupport extends Matchers with BeforeAndAfterAll { self: Suite trait CodecSpecSupport extends Matchers with BeforeAndAfterAll { self: Suite
@ -70,10 +70,10 @@ voluptua. At vero eos et accusam et justo duo dolores et ea rebum. Stet clita ka
est Lorem ipsum dolor sit amet. Lorem ipsum dolor sit amet, consetetur sadipscing elitr, sed diam nonumy e""".replace("\r\n", "\n") est Lorem ipsum dolor sit amet. Lorem ipsum dolor sit amet, consetetur sadipscing elitr, sed diam nonumy e""".replace("\r\n", "\n")
implicit val system = ActorSystem(getClass.getSimpleName) implicit val system = ActorSystem(getClass.getSimpleName)
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
override def afterAll() = { override def afterAll() = {
system.shutdown() system.shutdown()
system.awaitTermination(10.seconds) system.awaitTermination(10.seconds)
} }
} }

View file

@ -10,7 +10,7 @@ import akka.http.marshallers.xml.ScalaXmlSupport._
import scala.collection.immutable.ListMap import scala.collection.immutable.ListMap
import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers } import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers }
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.http.util._ import akka.http.util._
import akka.http.model._ import akka.http.model._
@ -20,7 +20,7 @@ import MediaTypes._
class MarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll with MultipartMarshallers with MarshallingTestUtils { class MarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll with MultipartMarshallers with MarshallingTestUtils {
implicit val system = ActorSystem(getClass.getSimpleName) implicit val system = ActorSystem(getClass.getSimpleName)
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
import system.dispatcher import system.dispatcher
"The PredefinedToEntityMarshallers." - { "The PredefinedToEntityMarshallers." - {

View file

@ -8,7 +8,7 @@ import akka.http.marshallers.xml.ScalaXmlSupport
import akka.http.server.directives.AuthenticationDirectives._ import akka.http.server.directives.AuthenticationDirectives._
import com.typesafe.config.{ ConfigFactory, Config } import com.typesafe.config.{ ConfigFactory, Config }
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.http.Http import akka.http.Http
object TestServer extends App { object TestServer extends App {
@ -17,7 +17,7 @@ object TestServer 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 system.dispatcher
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
import ScalaXmlSupport._ import ScalaXmlSupport._
import Directives._ import Directives._

View file

@ -12,7 +12,7 @@ import scala.concurrent.{ Future, Await }
import org.scalatest.matchers.Matcher import org.scalatest.matchers.Matcher
import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers } import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers }
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.http.model._ import akka.http.model._
import akka.http.util._ import akka.http.util._
@ -22,7 +22,7 @@ import FastFuture._
class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll with ScalatestUtils { class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll with ScalatestUtils {
implicit val system = ActorSystem(getClass.getSimpleName) implicit val system = ActorSystem(getClass.getSimpleName)
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
import system.dispatcher import system.dispatcher
"The PredefinedFromEntityUnmarshallers." - { "The PredefinedFromEntityUnmarshallers." - {

View file

@ -7,7 +7,7 @@ import akka.event.Logging
import scala.collection.{ mutable, immutable } import scala.collection.{ mutable, immutable }
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
@ -40,7 +40,7 @@ abstract class AkkaIdentityProcessorVerification[T](val system: ActorSystem, env
override def createErrorStatePublisher(): Publisher[T] = override def createErrorStatePublisher(): Publisher[T] =
StreamTestKit.errorPublisher(new Exception("Unable to serve subscribers right now!")) StreamTestKit.errorPublisher(new Exception("Unable to serve subscribers right now!"))
def createSimpleIntPublisher(elements: Long)(implicit mat: FlowMaterializer): Publisher[Int] = { def createSimpleIntPublisher(elements: Long)(implicit mat: ActorFlowMaterializer): Publisher[Int] = {
val iterable: immutable.Iterable[Int] = val iterable: immutable.Iterable[Int] =
if (elements == Long.MaxValue) 1 to Int.MaxValue if (elements == Long.MaxValue) 1 to Int.MaxValue
else 0 until elements.toInt else 0 until elements.toInt

View file

@ -8,8 +8,8 @@ import akka.event.Logging
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit import akka.stream.testkit.StreamTestKit
import org.reactivestreams.Publisher import org.reactivestreams.Publisher
@ -33,7 +33,7 @@ abstract class AkkaPublisherVerification[T](val system: ActorSystem, env: TestEn
this(false) this(false)
} }
implicit val materializer = FlowMaterializer(MaterializerSettings(system).copy(maxInputBufferSize = 512))(system) implicit val materializer = ActorFlowMaterializer(ActorFlowMaterializerSettings(system).copy(maxInputBufferSize = 512))(system)
@AfterClass @AfterClass
def shutdownActorSystem(): Unit = { def shutdownActorSystem(): Unit = {

View file

@ -8,8 +8,8 @@ import akka.event.Logging
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
@ -57,7 +57,7 @@ abstract class AkkaSubscriberWhiteboxVerification[T](val system: ActorSystem, en
trait AkkaSubscriberVerificationLike { trait AkkaSubscriberVerificationLike {
implicit def system: ActorSystem implicit def system: ActorSystem
implicit val materializer = FlowMaterializer(MaterializerSettings(system)) implicit val materializer = ActorFlowMaterializer(ActorFlowMaterializerSettings(system))
def createSimpleIntPublisher(elements: Long): Publisher[Int] = { def createSimpleIntPublisher(elements: Long): Publisher[Int] = {
val iterable: immutable.Iterable[Int] = val iterable: immutable.Iterable[Int] =

View file

@ -4,10 +4,10 @@
package akka.stream.tck package akka.stream.tck
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import akka.stream.impl.{ Ast, ActorBasedFlowMaterializer } import akka.stream.impl.{ Ast, ActorFlowMaterializerImpl }
import akka.stream.scaladsl.MaterializedMap import akka.stream.scaladsl.MaterializedMap
import akka.stream.scaladsl.OperationAttributes._ import akka.stream.scaladsl.OperationAttributes._
import akka.stream.{ FlowMaterializer, MaterializerSettings } import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings }
import org.reactivestreams.{ Publisher, Processor } import org.reactivestreams.{ Publisher, Processor }
import akka.stream.impl.fusing.Map import akka.stream.impl.fusing.Map
@ -18,21 +18,21 @@ class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] {
val processorCounter = new AtomicInteger val processorCounter = new AtomicInteger
override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = { override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = maxBufferSize / 2, maxSize = maxBufferSize) .withInputBuffer(initialSize = maxBufferSize / 2, maxSize = maxBufferSize)
implicit val materializer = FlowMaterializer(settings)(system) implicit val materializer = ActorFlowMaterializer(settings)(system)
val flowName = getClass.getSimpleName + "-" + processorCounter.incrementAndGet() val flowName = getClass.getSimpleName + "-" + processorCounter.incrementAndGet()
val (processor, _ns) = materializer.asInstanceOf[ActorBasedFlowMaterializer].processorForNode( val (processor, _ns) = materializer.asInstanceOf[ActorFlowMaterializerImpl].processorForNode(
Ast.Fused(List(Map[Int, Int](identity)), name("identity")), flowName, 1) Ast.Fused(List(Map[Int, Int](identity)), name("identity")), flowName, 1)
processor.asInstanceOf[Processor[Int, Int]] processor.asInstanceOf[Processor[Int, Int]]
} }
override def createHelperPublisher(elements: Long): Publisher[Int] = { override def createHelperPublisher(elements: Long): Publisher[Int] = {
implicit val mat = FlowMaterializer()(system) implicit val mat = ActorFlowMaterializer()(system)
createSimpleIntPublisher(elements)(mat) createSimpleIntPublisher(elements)(mat)
} }

View file

@ -4,10 +4,10 @@
package akka.stream.tck package akka.stream.tck
import akka.stream.scaladsl.OperationAttributes._ import akka.stream.scaladsl.OperationAttributes._
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.impl.ActorBasedFlowMaterializer import akka.stream.impl.ActorFlowMaterializerImpl
import akka.stream.impl.Ast import akka.stream.impl.Ast
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import akka.stream.scaladsl.MaterializedMap import akka.stream.scaladsl.MaterializedMap
import org.reactivestreams.Processor import org.reactivestreams.Processor
@ -22,10 +22,10 @@ class TransformProcessorTest extends AkkaIdentityProcessorVerification[Int] {
val processorCounter = new AtomicInteger val processorCounter = new AtomicInteger
override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = { override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = maxBufferSize / 2, maxSize = maxBufferSize) .withInputBuffer(initialSize = maxBufferSize / 2, maxSize = maxBufferSize)
implicit val materializer = FlowMaterializer(settings)(system) implicit val materializer = ActorFlowMaterializer(settings)(system)
val flowName = getClass.getSimpleName + "-" + processorCounter.incrementAndGet() val flowName = getClass.getSimpleName + "-" + processorCounter.incrementAndGet()
@ -34,14 +34,14 @@ class TransformProcessorTest extends AkkaIdentityProcessorVerification[Int] {
override def onPush(in: Any, ctx: Context[Any]) = ctx.push(in) override def onPush(in: Any, ctx: Context[Any]) = ctx.push(in)
} }
val (processor, _) = materializer.asInstanceOf[ActorBasedFlowMaterializer].processorForNode( val (processor, _) = materializer.asInstanceOf[ActorFlowMaterializerImpl].processorForNode(
Ast.StageFactory(mkStage, name("transform")), flowName, 1) Ast.StageFactory(mkStage, name("transform")), flowName, 1)
processor.asInstanceOf[Processor[Int, Int]] processor.asInstanceOf[Processor[Int, Int]]
} }
override def createHelperPublisher(elements: Long): Publisher[Int] = { override def createHelperPublisher(elements: Long): Publisher[Int] = {
implicit val mat = FlowMaterializer()(system) implicit val mat = ActorFlowMaterializer()(system)
createSimpleIntPublisher(elements)(mat) createSimpleIntPublisher(elements)(mat)
} }

View file

@ -1,11 +1,11 @@
# The StreamTestDefaultMailbox verifies that stream actors are using the dispatcher defined in MaterializerSettings. # The StreamTestDefaultMailbox verifies that stream actors are using the dispatcher defined in ActorFlowMaterializerSettings.
# #
# All stream tests should use the dedicated `akka.test.stream-dispatcher` or disable this validation by defining: # All stream tests should use the dedicated `akka.test.stream-dispatcher` or disable this validation 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"
# Dispatcher for stream actors. Specified in tests with # Dispatcher for stream actors. Specified in tests with
# MaterializerSettings(dispatcher = "akka.test.stream-dispatcher") # ActorFlowMaterializerSettings(dispatcher = "akka.test.stream-dispatcher")
akka.test.stream-dispatcher { akka.test.stream-dispatcher {
type = Dispatcher type = Dispatcher
executor = "fork-join-executor" executor = "fork-join-executor"

View file

@ -1,21 +1,21 @@
package akka.stream.testkit package akka.stream.testkit
import akka.actor.{ ActorRefFactory, ActorSystem } import akka.actor.{ ActorRefFactory, ActorSystem }
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import org.reactivestreams.Publisher import org.reactivestreams.Publisher
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
class ChainSetup[In, Out]( class ChainSetup[In, Out](
stream: Flow[In, In] Flow[In, Out], stream: Flow[In, In] Flow[In, Out],
val settings: MaterializerSettings, val settings: ActorFlowMaterializerSettings,
materializer: FlowMaterializer, materializer: ActorFlowMaterializer,
toPublisher: (Source[Out], FlowMaterializer) Publisher[Out])(implicit val system: ActorSystem) { toPublisher: (Source[Out], ActorFlowMaterializer) Publisher[Out])(implicit val system: ActorSystem) {
def this(stream: Flow[In, In] Flow[In, Out], settings: MaterializerSettings, toPublisher: (Source[Out], FlowMaterializer) Publisher[Out])(implicit system: ActorSystem) = def this(stream: Flow[In, In] Flow[In, Out], settings: ActorFlowMaterializerSettings, toPublisher: (Source[Out], ActorFlowMaterializer) Publisher[Out])(implicit system: ActorSystem) =
this(stream, settings, FlowMaterializer(settings)(system), toPublisher)(system) this(stream, settings, ActorFlowMaterializer(settings)(system), toPublisher)(system)
def this(stream: Flow[In, In] Flow[In, Out], settings: MaterializerSettings, materializerCreator: (MaterializerSettings, ActorRefFactory) FlowMaterializer, toPublisher: (Source[Out], FlowMaterializer) Publisher[Out])(implicit system: ActorSystem) = def this(stream: Flow[In, In] Flow[In, Out], settings: ActorFlowMaterializerSettings, materializerCreator: (ActorFlowMaterializerSettings, ActorRefFactory) ActorFlowMaterializer, toPublisher: (Source[Out], ActorFlowMaterializer) Publisher[Out])(implicit system: ActorSystem) =
this(stream, settings, materializerCreator(settings, system), toPublisher)(system) this(stream, settings, materializerCreator(settings, system), toPublisher)(system)
val upstream = StreamTestKit.PublisherProbe[In]() val upstream = StreamTestKit.PublisherProbe[In]()

View file

@ -4,7 +4,7 @@
package akka.stream.testkit package akka.stream.testkit
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.scaladsl.{ Sink, Source, Flow } import akka.stream.scaladsl.{ Sink, Source, Flow }
import akka.stream.testkit.StreamTestKit._ import akka.stream.testkit.StreamTestKit._
import org.reactivestreams.Publisher import org.reactivestreams.Publisher
@ -12,13 +12,13 @@ import org.scalatest.Matchers
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
trait ScriptedTest extends Matchers { trait ScriptedTest extends Matchers {
class ScriptException(msg: String) extends RuntimeException(msg) class ScriptException(msg: String) extends RuntimeException(msg)
def toPublisher[In, Out]: (Source[Out], FlowMaterializer) Publisher[Out] = def toPublisher[In, Out]: (Source[Out], ActorFlowMaterializer) Publisher[Out] =
(f, m) f.runWith(Sink.publisher)(m) (f, m) f.runWith(Sink.publisher)(m)
object Script { object Script {
@ -82,7 +82,7 @@ trait ScriptedTest extends Matchers {
class ScriptRunner[In, Out]( class ScriptRunner[In, Out](
op: Flow[In, In] Flow[In, Out], op: Flow[In, In] Flow[In, Out],
settings: MaterializerSettings, settings: ActorFlowMaterializerSettings,
script: Script[In, Out], script: Script[In, Out],
maximumOverrun: Int, maximumOverrun: Int,
maximumRequest: Int, maximumRequest: Int,
@ -190,7 +190,7 @@ trait ScriptedTest extends Matchers {
} }
def runScript[In, Out](script: Script[In, Out], settings: MaterializerSettings, maximumOverrun: Int = 3, maximumRequest: Int = 3, maximumBuffer: Int = 3)( def runScript[In, Out](script: Script[In, Out], settings: ActorFlowMaterializerSettings, maximumOverrun: Int = 3, maximumRequest: Int = 3, maximumBuffer: Int = 3)(
op: Flow[In, In] Flow[In, Out])(implicit system: ActorSystem): Unit = { op: Flow[In, In] Flow[In, Out])(implicit system: ActorSystem): Unit = {
new ScriptRunner(op, settings, script, maximumOverrun, maximumRequest, maximumBuffer).run() new ScriptRunner(op, settings, script, maximumOverrun, maximumRequest, maximumBuffer).run()
} }

View file

@ -14,7 +14,7 @@ import akka.actor.Actor
/** /**
* INTERNAL API * INTERNAL API
* This mailbox is only used in tests to verify that stream actors are using * This mailbox is only used in tests to verify that stream actors are using
* the dispatcher defined in MaterializerSettings. * the dispatcher defined in ActorFlowMaterializerSettings.
*/ */
private[akka] final case class StreamTestDefaultMailbox() extends MailboxType with ProducesMessageQueue[UnboundedMailbox.MessageQueue] { private[akka] final case class StreamTestDefaultMailbox() extends MailboxType with ProducesMessageQueue[UnboundedMailbox.MessageQueue] {
@ -31,7 +31,7 @@ private[akka] final case class StreamTestDefaultMailbox() extends MailboxType wi
s"$r with actor class [${actorClass.getName}] must not run on default dispatcher in tests. " + s"$r with actor class [${actorClass.getName}] must not run on default dispatcher in tests. " +
"Did you forget to define `props.withDispatcher` when creating the actor? " + "Did you forget to define `props.withDispatcher` when creating the actor? " +
"Or did you forget to configure the `akka.stream.materializer` setting accordingly or force the " + "Or did you forget to configure the `akka.stream.materializer` setting accordingly or force the " +
"""dispatcher using `MaterializerSettings(sys).withDispatcher("akka.test.stream-dispatcher")` in the test?""") """dispatcher using `ActorFlowMaterializerSettings(sys).withDispatcher("akka.test.stream-dispatcher")` in the test?""")
} catch { } catch {
// this logging should not be needed when issue #15947 has been fixed // this logging should not be needed when issue #15947 has been fixed
case e: AssertionError case e: AssertionError

View file

@ -1,18 +1,18 @@
package akka.stream.testkit package akka.stream.testkit
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import org.reactivestreams.Publisher import org.reactivestreams.Publisher
import scala.collection.immutable import scala.collection.immutable
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
abstract class TwoStreamsSetup extends AkkaSpec { abstract class TwoStreamsSetup extends AkkaSpec {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 2) .withInputBuffer(initialSize = 2, maxSize = 2)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
val TestException = new RuntimeException("test") with NoStackTrace val TestException = new RuntimeException("test") with NoStackTrace

View file

@ -9,11 +9,11 @@ import akka.stream.javadsl.AkkaJUnitActorSystemResource;
public abstract class StreamTest { public abstract class StreamTest {
final protected ActorSystem system; final protected ActorSystem system;
final protected FlowMaterializer materializer; final protected ActorFlowMaterializer materializer;
protected StreamTest(AkkaJUnitActorSystemResource actorSystemResource) { protected StreamTest(AkkaJUnitActorSystemResource actorSystemResource) {
system = actorSystemResource.getSystem(); system = actorSystemResource.getSystem();
MaterializerSettings settings = MaterializerSettings.create(system); ActorFlowMaterializerSettings settings = ActorFlowMaterializerSettings.create(system);
materializer = FlowMaterializer.create(settings, system); materializer = ActorFlowMaterializer.create(settings, system);
} }
} }

View file

@ -14,7 +14,7 @@ import static org.junit.Assert.assertEquals;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.reactivestreams.Publisher; import org.reactivestreams.Publisher;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.stream.FlowMaterializer; import akka.stream.ActorFlowMaterializer;
import akka.stream.testkit.AkkaSpec; import akka.stream.testkit.AkkaSpec;
import akka.stream.javadsl.FlexiMerge; import akka.stream.javadsl.FlexiMerge;
import scala.concurrent.Await; import scala.concurrent.Await;
@ -30,7 +30,7 @@ public class FlexiMergeTest {
final ActorSystem system = actorSystemResource.getSystem(); final ActorSystem system = actorSystemResource.getSystem();
final FlowMaterializer materializer = FlowMaterializer.create(system); final ActorFlowMaterializer materializer = ActorFlowMaterializer.create(system);
final Source<String> in1 = Source.from(Arrays.asList("a", "b", "c", "d")); final Source<String> in1 = Source.from(Arrays.asList("a", "b", "c", "d"));
final Source<String> in2 = Source.from(Arrays.asList("e", "f")); final Source<String> in2 = Source.from(Arrays.asList("e", "f"));

View file

@ -12,7 +12,7 @@ import static org.junit.Assert.assertEquals;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.stream.FlowMaterializer; import akka.stream.ActorFlowMaterializer;
import akka.stream.testkit.AkkaSpec; import akka.stream.testkit.AkkaSpec;
import akka.stream.javadsl.FlexiRoute; import akka.stream.javadsl.FlexiRoute;
import akka.japi.Pair; import akka.japi.Pair;
@ -29,7 +29,7 @@ public class FlexiRouteTest {
final ActorSystem system = actorSystemResource.getSystem(); final ActorSystem system = actorSystemResource.getSystem();
final FlowMaterializer materializer = FlowMaterializer.create(system); final ActorFlowMaterializer materializer = ActorFlowMaterializer.create(system);
final Source<String> in = Source.from(Arrays.asList("a", "b", "c", "d", "e")); final Source<String> in = Source.from(Arrays.asList("a", "b", "c", "d", "e"));

View file

@ -50,7 +50,7 @@ class DslConsistencySpec extends WordSpec with Matchers {
jFlowGraphClass Set("graph", "cyclesAllowed"), jFlowGraphClass Set("graph", "cyclesAllowed"),
jPartialFlowGraphClass Set("graph", "cyclesAllowed", "disconnectedAllowed")) jPartialFlowGraphClass Set("graph", "cyclesAllowed", "disconnectedAllowed"))
def materializing(m: Method): Boolean = m.getParameterTypes.contains(classOf[FlowMaterializer]) def materializing(m: Method): Boolean = m.getParameterTypes.contains(classOf[ActorFlowMaterializer])
def assertHasMethod(c: Class[_], name: String): Unit = { def assertHasMethod(c: Class[_], name: String): Unit = {
// include class name to get better error message // include class name to get better error message

View file

@ -11,7 +11,7 @@ import akka.stream.scaladsl.Flow
import akka.stream.scaladsl.FlowGraph import akka.stream.scaladsl.FlowGraph
import akka.stream.scaladsl.FlowGraphImplicits import akka.stream.scaladsl.FlowGraphImplicits
import akka.stream.scaladsl.Merge import akka.stream.scaladsl.Merge
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
@ -256,7 +256,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() implicit val materializer = ActorFlowMaterializer()
val probe = TestProbe() val probe = TestProbe()
val source = Source[Int](senderProps) val source = Source[Int](senderProps)
@ -285,7 +285,7 @@ class ActorPublisherSpec extends AkkaSpec with ImplicitSender {
} }
"work in a FlowGraph" in { "work in a FlowGraph" in {
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
val probe1 = TestProbe() val probe1 = TestProbe()
val probe2 = TestProbe() val probe2 = TestProbe()
@ -325,7 +325,7 @@ class ActorPublisherSpec extends AkkaSpec with ImplicitSender {
} }
"be able to define a subscription-timeout, after which it should shut down" in { "be able to define a subscription-timeout, after which it should shut down" in {
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
val timeout = 150.millis val timeout = 150.millis
val a = system.actorOf(timeoutingProps(testActor, timeout)) val a = system.actorOf(timeoutingProps(testActor, timeout))
val pub = ActorPublisher(a) val pub = ActorPublisher(a)
@ -345,7 +345,7 @@ class ActorPublisherSpec extends AkkaSpec with ImplicitSender {
} }
"be able to define a subscription-timeout, which is cancelled by the first incoming Subscriber" in { "be able to define a subscription-timeout, which is cancelled by the first incoming Subscriber" in {
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
val timeout = 500.millis val timeout = 500.millis
val sub = StreamTestKit.SubscriberProbe[Int]() val sub = StreamTestKit.SubscriberProbe[Int]()

View file

@ -5,7 +5,7 @@ package akka.stream.actor
import akka.actor.{ Actor, ActorRef, Props } import akka.actor.{ Actor, ActorRef, Props }
import akka.routing.{ ActorRefRoutee, RoundRobinRoutingLogic, Router } import akka.routing.{ ActorRefRoutee, RoundRobinRoutingLogic, Router }
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
@ -95,7 +95,7 @@ class ActorSubscriberSpec extends AkkaSpec with ImplicitSender {
import ActorSubscriberMessage._ import ActorSubscriberMessage._
import ActorSubscriberSpec._ import ActorSubscriberSpec._
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
"An ActorSubscriber" must { "An ActorSubscriber" must {

View file

@ -3,7 +3,7 @@
*/ */
package akka.stream.extra package akka.stream.extra
import akka.stream.{ MaterializerSettings, FlowMaterializer } import akka.stream.{ ActorFlowMaterializerSettings, ActorFlowMaterializer }
import akka.stream.scaladsl.{ Source, Flow } import akka.stream.scaladsl.{ Source, Flow }
import akka.stream.scaladsl.Sink import akka.stream.scaladsl.Sink
import akka.stream.testkit.{ AkkaSpec, ScriptedTest, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, ScriptedTest, StreamTestKit }
@ -14,10 +14,10 @@ class FlowTimedSpec extends AkkaSpec with ScriptedTest {
import scala.concurrent.duration._ import scala.concurrent.duration._
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 16) .withInputBuffer(initialSize = 2, maxSize = 16)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
"Timed Source" must { "Timed Source" must {

View file

@ -6,7 +6,7 @@ package akka.stream.io
import akka.actor.{ Actor, ActorRef, Props } import akka.actor.{ Actor, ActorRef, Props }
import akka.io.{ IO, Tcp } import akka.io.{ IO, Tcp }
import akka.stream.testkit.StreamTestKit import akka.stream.testkit.StreamTestKit
import akka.stream.{ FlowMaterializer, MaterializerSettings } import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings }
import akka.testkit.{ TestKitBase, TestProbe } import akka.testkit.{ TestKitBase, TestProbe }
import akka.util.ByteString import akka.util.ByteString
import java.net.InetSocketAddress import java.net.InetSocketAddress
@ -102,10 +102,10 @@ object TcpHelper {
trait TcpHelper { this: TestKitBase trait TcpHelper { this: TestKitBase
import akka.stream.io.TcpHelper._ import akka.stream.io.TcpHelper._
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 4, maxSize = 4) .withInputBuffer(initialSize = 4, maxSize = 4)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
class Server(val address: InetSocketAddress = temporaryServerAddress()) { class Server(val address: InetSocketAddress = temporaryServerAddress()) {
val serverProbe = TestProbe() val serverProbe = TestProbe()

View file

@ -4,16 +4,16 @@
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import org.reactivestreams.Subscriber import org.reactivestreams.Subscriber
import org.scalatest.Matchers import org.scalatest.Matchers
class FlowAppendSpec extends AkkaSpec with River { class FlowAppendSpec extends AkkaSpec with River {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
"Flow" should { "Flow" should {
"append Flow" in riverOf[String] { subscriber "append Flow" in riverOf[String] { subscriber

View file

@ -7,18 +7,18 @@ import scala.concurrent.Await
import scala.concurrent.Future import scala.concurrent.Future
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.OverflowStrategy import akka.stream.OverflowStrategy
import akka.stream.OverflowStrategy.Error.BufferOverflowException import akka.stream.OverflowStrategy.Error.BufferOverflowException
import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
class FlowBufferSpec extends AkkaSpec { class FlowBufferSpec extends AkkaSpec {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 1, maxSize = 1) .withInputBuffer(initialSize = 1, maxSize = 1)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
"Buffer" must { "Buffer" must {

View file

@ -7,13 +7,13 @@ import java.io.{ File, FileInputStream }
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current random } import scala.concurrent.forkjoin.ThreadLocalRandom.{ current random }
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.ScriptedTest import akka.stream.testkit.ScriptedTest
class FlowCollectSpec extends AkkaSpec with ScriptedTest { class FlowCollectSpec extends AkkaSpec with ScriptedTest {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
"A Collect" must { "A Collect" must {

View file

@ -6,8 +6,8 @@ package akka.stream.scaladsl
import scala.collection.immutable.Seq import scala.collection.immutable.Seq
import scala.concurrent.Future import scala.concurrent.Future
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
class FlowCompileSpec extends AkkaSpec { class FlowCompileSpec extends AkkaSpec {
@ -17,7 +17,7 @@ class FlowCompileSpec extends AkkaSpec {
import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.ExecutionContext.Implicits.global
val intFut = Source(Future { 3 }) val intFut = Source(Future { 3 })
implicit val materializer = FlowMaterializer(MaterializerSettings(system)) implicit val materializer = ActorFlowMaterializer(ActorFlowMaterializerSettings(system))
"Flow" should { "Flow" should {
"should not run" in { "should not run" in {

View file

@ -6,16 +6,16 @@ package akka.stream.scaladsl
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.FlattenStrategy import akka.stream.FlattenStrategy
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ StreamTestKit, AkkaSpec } import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
class FlowConcatAllSpec extends AkkaSpec { class FlowConcatAllSpec extends AkkaSpec {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 2) .withInputBuffer(initialSize = 2, maxSize = 2)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
"ConcatAll" must { "ConcatAll" must {

View file

@ -6,15 +6,15 @@ package akka.stream.scaladsl
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.stream.{ OverflowStrategy, FlowMaterializer, MaterializerSettings } import akka.stream.{ OverflowStrategy, ActorFlowMaterializer, ActorFlowMaterializerSettings }
import akka.stream.testkit.{ StreamTestKit, AkkaSpec } import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
class FlowConflateSpec extends AkkaSpec { class FlowConflateSpec extends AkkaSpec {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 2) .withInputBuffer(initialSize = 2, maxSize = 2)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
"Conflate" must { "Conflate" must {

View file

@ -5,16 +5,16 @@ package akka.stream.scaladsl
import akka.testkit.TestProbe import akka.testkit.TestProbe
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
class FlowDispatcherSpec extends AkkaSpec("my-dispatcher = ${akka.test.stream-dispatcher}") { class FlowDispatcherSpec extends AkkaSpec("my-dispatcher = ${akka.test.stream-dispatcher}") {
val defaultSettings = MaterializerSettings(system) val defaultSettings = ActorFlowMaterializerSettings(system)
def testDispatcher(settings: MaterializerSettings = defaultSettings, dispatcher: String = "akka.test.stream-dispatcher") = { def testDispatcher(settings: ActorFlowMaterializerSettings = defaultSettings, dispatcher: String = "akka.test.stream-dispatcher") = {
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
val probe = TestProbe() val probe = TestProbe()
val p = Source(List(1, 2, 3)).map(i val p = Source(List(1, 2, 3)).map(i

View file

@ -5,18 +5,18 @@ package akka.stream.scaladsl
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current random } import scala.concurrent.forkjoin.ThreadLocalRandom.{ current random }
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.ScriptedTest import akka.stream.testkit.ScriptedTest
import akka.stream.testkit.StreamTestKit import akka.stream.testkit.StreamTestKit
class FlowDropSpec extends AkkaSpec with ScriptedTest { class FlowDropSpec extends AkkaSpec with ScriptedTest {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 16) .withInputBuffer(initialSize = 2, maxSize = 16)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
"A Drop" must { "A Drop" must {

View file

@ -5,13 +5,13 @@ package akka.stream.scaladsl
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit import akka.stream.testkit.StreamTestKit
class FlowDropWithinSpec extends AkkaSpec { class FlowDropWithinSpec extends AkkaSpec {
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
"A DropWithin" must { "A DropWithin" must {

View file

@ -7,17 +7,17 @@ import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ StreamTestKit, AkkaSpec } import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
class FlowExpandSpec extends AkkaSpec { class FlowExpandSpec extends AkkaSpec {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 2) .withInputBuffer(initialSize = 2, maxSize = 2)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
"Expand" must { "Expand" must {

View file

@ -5,14 +5,14 @@ package akka.stream.scaladsl
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current random } import scala.concurrent.forkjoin.ThreadLocalRandom.{ current random }
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.ScriptedTest import akka.stream.testkit.ScriptedTest
class FlowFilterSpec extends AkkaSpec with ScriptedTest { class FlowFilterSpec extends AkkaSpec with ScriptedTest {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 16) .withInputBuffer(initialSize = 2, maxSize = 16)
"A Filter" must { "A Filter" must {
@ -23,9 +23,9 @@ class FlowFilterSpec extends AkkaSpec with ScriptedTest {
} }
"not blow up with high request counts" in { "not blow up with high request counts" in {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 1, maxSize = 1) .withInputBuffer(initialSize = 1, maxSize = 1)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
val probe = StreamTestKit.SubscriberProbe[Int]() val probe = StreamTestKit.SubscriberProbe[Int]()
Source(List.fill(1000)(0) ::: List(1)).filter(_ != 0).runWith(Sink(probe)) Source(List.fill(1000)(0) ::: List(1)).filter(_ != 0).runWith(Sink(probe))

View file

@ -6,12 +6,12 @@ package akka.stream.scaladsl
import scala.concurrent.Await import scala.concurrent.Await
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.{ OverflowStrategy, FlowMaterializer } import akka.stream.{ OverflowStrategy, ActorFlowMaterializer }
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
class FlowFoldSpec extends AkkaSpec with DefaultTimeout { class FlowFoldSpec extends AkkaSpec with DefaultTimeout {
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
"A Fold" must { "A Fold" must {

View file

@ -5,12 +5,12 @@ package akka.stream.scaladsl
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
class FlowForeachSpec extends AkkaSpec { class FlowForeachSpec extends AkkaSpec {
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
import system.dispatcher import system.dispatcher
"A Foreach" must { "A Foreach" must {

View file

@ -7,16 +7,16 @@ import scala.concurrent.{ Future, Promise }
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
class FlowFromFutureSpec extends AkkaSpec { class FlowFromFutureSpec extends AkkaSpec {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(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 {

View file

@ -4,7 +4,7 @@
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.scaladsl.OperationAttributes._ import akka.stream.scaladsl.OperationAttributes._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.OverflowStrategy import akka.stream.OverflowStrategy
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit.{ PublisherProbe, SubscriberProbe } import akka.stream.testkit.StreamTestKit.{ PublisherProbe, SubscriberProbe }
@ -18,7 +18,7 @@ object FlowGraphCompileSpec {
class FlowGraphCompileSpec extends AkkaSpec { class FlowGraphCompileSpec extends AkkaSpec {
import FlowGraphCompileSpec._ import FlowGraphCompileSpec._
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
def op[In, Out]: () PushStage[In, Out] = { () def op[In, Out]: () PushStage[In, Out] = { ()
new PushStage[In, Out] { new PushStage[In, Out] {

View file

@ -3,7 +3,7 @@
*/ */
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import scala.concurrent.Await import scala.concurrent.Await
@ -13,7 +13,7 @@ import scala.concurrent.duration._
class FlowGraphInitSpec extends AkkaSpec { class FlowGraphInitSpec extends AkkaSpec {
import system.dispatcher import system.dispatcher
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
"Initialization of FlowGraph" should { "Initialization of FlowGraph" should {
"be thread safe" in { "be thread safe" in {

View file

@ -6,17 +6,17 @@ package akka.stream.scaladsl
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit._ import akka.stream.testkit._
import org.reactivestreams.Publisher import org.reactivestreams.Publisher
class FlowGroupBySpec extends AkkaSpec { class FlowGroupBySpec extends AkkaSpec {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 2) .withInputBuffer(initialSize = 2, maxSize = 2)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
case class StreamPuppet(p: Publisher[Int]) { case class StreamPuppet(p: Publisher[Int]) {
val probe = StreamTestKit.SubscriberProbe[Int]() val probe = StreamTestKit.SubscriberProbe[Int]()

View file

@ -6,13 +6,13 @@ package akka.stream.scaladsl
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current random } import scala.concurrent.forkjoin.ThreadLocalRandom.{ current random }
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.ScriptedTest import akka.stream.testkit.ScriptedTest
class FlowGroupedSpec extends AkkaSpec with ScriptedTest { class FlowGroupedSpec extends AkkaSpec with ScriptedTest {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 16) .withInputBuffer(initialSize = 2, maxSize = 16)
"A Grouped" must { "A Grouped" must {

View file

@ -7,17 +7,17 @@ import scala.collection.immutable
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current random } import scala.concurrent.forkjoin.ThreadLocalRandom.{ current random }
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.ScriptedTest import akka.stream.testkit.ScriptedTest
import akka.stream.testkit.StreamTestKit import akka.stream.testkit.StreamTestKit
class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest { class FlowGroupedWithinSpec extends AkkaSpec with ScriptedTest {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
"A GroupedWithin" must { "A GroupedWithin" must {

View file

@ -6,8 +6,8 @@ package akka.stream.scaladsl
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.OnComplete import akka.stream.testkit.StreamTestKit.OnComplete
@ -28,10 +28,10 @@ class FlowIterableSpec extends AbstractFlowIteratorSpec {
abstract class AbstractFlowIteratorSpec extends AkkaSpec { abstract class AbstractFlowIteratorSpec extends AkkaSpec {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 2) .withInputBuffer(initialSize = 2, maxSize = 2)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
def testName: String def testName: String

View file

@ -3,7 +3,7 @@
*/ */
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.{ FlowMaterializer, MaterializerSettings } import akka.stream.{ ActorFlowMaterializer, ActorFlowMaterializerSettings }
import akka.stream.testkit.{ StreamTestKit, AkkaSpec } import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.concurrent.Await import scala.concurrent.Await
@ -11,10 +11,10 @@ import scala.concurrent.duration._
class FlowJoinSpec extends AkkaSpec(ConfigFactory.parseString("akka.loglevel=INFO")) { class FlowJoinSpec extends AkkaSpec(ConfigFactory.parseString("akka.loglevel=INFO")) {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 16) .withInputBuffer(initialSize = 2, maxSize = 16)
implicit val mat = FlowMaterializer(settings) implicit val mat = ActorFlowMaterializer(settings)
"A Flow using join" must { "A Flow using join" must {
"allow for cycles" in { "allow for cycles" in {

View file

@ -9,7 +9,7 @@ import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom import scala.concurrent.forkjoin.ThreadLocalRandom
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit import akka.stream.testkit.StreamTestKit
import akka.testkit.TestLatch import akka.testkit.TestLatch
@ -17,7 +17,7 @@ import akka.testkit.TestProbe
class FlowMapAsyncSpec extends AkkaSpec { class FlowMapAsyncSpec extends AkkaSpec {
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
"A Flow with mapAsync" must { "A Flow with mapAsync" must {

View file

@ -8,7 +8,7 @@ import scala.concurrent.Future
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit import akka.stream.testkit.StreamTestKit
import akka.testkit.TestLatch import akka.testkit.TestLatch
@ -16,7 +16,7 @@ import akka.testkit.TestProbe
class FlowMapAsyncUnorderedSpec extends AkkaSpec { class FlowMapAsyncUnorderedSpec extends AkkaSpec {
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
"A Flow with mapAsyncUnordered" must { "A Flow with mapAsyncUnordered" must {

View file

@ -4,15 +4,15 @@
package akka.stream.scaladsl package akka.stream.scaladsl
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.ScriptedTest import akka.stream.testkit.ScriptedTest
import akka.stream.testkit.StreamTestKit.SubscriberProbe import akka.stream.testkit.StreamTestKit.SubscriberProbe
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
class FlowMapConcatSpec extends AkkaSpec with ScriptedTest { class FlowMapConcatSpec extends AkkaSpec with ScriptedTest {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 16) .withInputBuffer(initialSize = 2, maxSize = 16)
"A MapConcat" must { "A MapConcat" must {
@ -29,9 +29,9 @@ class FlowMapConcatSpec extends AkkaSpec with ScriptedTest {
} }
"map and concat grouping with slow downstream" in { "map and concat grouping with slow downstream" in {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 2) .withInputBuffer(initialSize = 2, maxSize = 2)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
val s = SubscriberProbe[Int] val s = SubscriberProbe[Int]
val input = (1 to 20).grouped(5).toList val input = (1 to 20).grouped(5).toList
Source(input).mapConcat(identity).map(x { Thread.sleep(10); x }).runWith(Sink(s)) Source(input).mapConcat(identity).map(x { Thread.sleep(10); x }).runWith(Sink(s))

View file

@ -5,17 +5,17 @@ package akka.stream.scaladsl
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current random } import scala.concurrent.forkjoin.ThreadLocalRandom.{ current random }
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.ScriptedTest import akka.stream.testkit.ScriptedTest
class FlowMapSpec extends AkkaSpec with ScriptedTest { class FlowMapSpec extends AkkaSpec with ScriptedTest {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 16) .withInputBuffer(initialSize = 2, maxSize = 16)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
"A Map" must { "A Map" must {

View file

@ -7,8 +7,8 @@ import scala.concurrent.duration._
import scala.util.{ Failure, Success } import scala.util.{ Failure, Success }
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.ScriptedTest import akka.stream.testkit.ScriptedTest
@ -16,10 +16,10 @@ import akka.testkit.TestProbe
class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest { class FlowOnCompleteSpec extends AkkaSpec with ScriptedTest {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 16) .withInputBuffer(initialSize = 2, maxSize = 16)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
"A Flow with onComplete" must { "A Flow with onComplete" must {

View file

@ -8,17 +8,17 @@ import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.StreamTestKit.SubscriberProbe import akka.stream.testkit.StreamTestKit.SubscriberProbe
class FlowPrefixAndTailSpec extends AkkaSpec { class FlowPrefixAndTailSpec extends AkkaSpec {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 2) .withInputBuffer(initialSize = 2, maxSize = 2)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
"PrefixAndTail" must { "PrefixAndTail" must {

View file

@ -9,16 +9,16 @@ import scala.concurrent.forkjoin.ThreadLocalRandom.{ current ⇒ random }
import scala.collection.immutable import scala.collection.immutable
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
class FlowScanSpec extends AkkaSpec { class FlowScanSpec extends AkkaSpec {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 16) .withInputBuffer(initialSize = 2, maxSize = 16)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
"A Scan" must { "A Scan" must {

View file

@ -4,7 +4,7 @@
package akka.stream.scaladsl package akka.stream.scaladsl
import akka.stream.scaladsl.OperationAttributes._ import akka.stream.scaladsl.OperationAttributes._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.testkit.TestProbe import akka.testkit.TestProbe
@ -15,7 +15,7 @@ object FlowSectionSpec {
class FlowSectionSpec extends AkkaSpec(FlowSectionSpec.config) { class FlowSectionSpec extends AkkaSpec(FlowSectionSpec.config) {
implicit val mat = FlowMaterializer() implicit val mat = ActorFlowMaterializer()
"A flow" can { "A flow" can {

View file

@ -9,8 +9,8 @@ import akka.stream.stage.Stage
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor._ import akka.actor._
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.impl._ import akka.stream.impl._
import akka.stream.impl.Ast._ import akka.stream.impl.Ast._
import akka.stream.testkit.{ StreamTestKit, AkkaSpec } import akka.stream.testkit.{ StreamTestKit, AkkaSpec }
@ -32,7 +32,7 @@ object FlowSpec {
case class BrokenMessage(msg: String) case class BrokenMessage(msg: String)
class BrokenActorInterpreter( class BrokenActorInterpreter(
_settings: MaterializerSettings, _settings: ActorFlowMaterializerSettings,
_ops: Seq[Stage[_, _]], _ops: Seq[Stage[_, _]],
brokenMessage: Any) brokenMessage: Any)
extends ActorInterpreter(_settings, _ops) { extends ActorInterpreter(_settings, _ops) {
@ -48,14 +48,13 @@ object FlowSpec {
} }
} }
class BrokenFlowMaterializer( class BrokenActorFlowMaterializer(
settings: MaterializerSettings, settings: ActorFlowMaterializerSettings,
dispatchers: Dispatchers, dispatchers: Dispatchers,
supervisor: ActorRef, supervisor: ActorRef,
flowNameCounter: AtomicLong, flowNameCounter: AtomicLong,
namePrefix: String, namePrefix: String,
optimizations: Optimizations, brokenMessage: Any) extends ActorFlowMaterializerImpl(settings, dispatchers, supervisor, flowNameCounter, namePrefix) {
brokenMessage: Any) extends ActorBasedFlowMaterializer(settings, dispatchers, supervisor, flowNameCounter, namePrefix, optimizations) {
override def processorForNode[In, Out](op: AstNode, flowName: String, n: Int): (Processor[In, Out], MaterializedMap) = { override def processorForNode[In, Out](op: AstNode, flowName: String, n: Int): (Processor[In, Out], MaterializedMap) = {
val props = op match { val props = op match {
@ -78,8 +77,8 @@ object FlowSpec {
} }
def createBrokenFlowMaterializer(settings: MaterializerSettings, brokenMessage: Any)(implicit context: ActorRefFactory): BrokenFlowMaterializer = { def createBrokenActorFlowMaterializer(settings: ActorFlowMaterializerSettings, brokenMessage: Any)(implicit context: ActorRefFactory): BrokenActorFlowMaterializer = {
new BrokenFlowMaterializer( new BrokenActorFlowMaterializer(
settings, settings,
{ {
context match { context match {
@ -93,7 +92,6 @@ object FlowSpec {
context.actorOf(StreamSupervisor.props(settings).withDispatcher(settings.dispatcher)), context.actorOf(StreamSupervisor.props(settings).withDispatcher(settings.dispatcher)),
flowNameCounter, flowNameCounter,
"brokenflow", "brokenflow",
Optimizations.none,
brokenMessage) brokenMessage)
} }
} }
@ -101,17 +99,17 @@ object FlowSpec {
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._
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 16) .withInputBuffer(initialSize = 2, maxSize = 16)
implicit val mat = FlowMaterializer(settings) implicit val mat = ActorFlowMaterializer(settings)
val identity: Flow[Any, Any] Flow[Any, Any] = in in.map(e e) val identity: Flow[Any, Any] Flow[Any, Any] = in in.map(e e)
val identity2: Flow[Any, Any] Flow[Any, Any] = in identity(in) val identity2: Flow[Any, Any] Flow[Any, Any] = in identity(in)
val toPublisher: (Source[Any], FlowMaterializer) Publisher[Any] = val toPublisher: (Source[Any], ActorFlowMaterializer) Publisher[Any] =
(f, m) f.runWith(Sink.publisher)(m) (f, m) f.runWith(Sink.publisher)(m)
def toFanoutPublisher[In, Out](initialBufferSize: Int, maximumBufferSize: Int): (Source[Out], FlowMaterializer) Publisher[Out] = def toFanoutPublisher[In, Out](initialBufferSize: Int, maximumBufferSize: Int): (Source[Out], ActorFlowMaterializer) Publisher[Out] =
(f, m) f.runWith(Sink.fanoutPublisher(initialBufferSize, maximumBufferSize))(m) (f, m) f.runWith(Sink.fanoutPublisher(initialBufferSize, maximumBufferSize))(m)
def materializeIntoSubscriberAndPublisher[In, Out](flow: Flow[In, Out]): (Subscriber[In], Publisher[Out]) = { def materializeIntoSubscriberAndPublisher[In, Out](flow: Flow[In, Out]): (Subscriber[In], Publisher[Out]) = {
@ -572,7 +570,7 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
"A broken Flow" must { "A broken Flow" must {
"cancel upstream and call onError on current and future downstream subscribers if an internal error occurs" in { "cancel upstream and call onError on current and future downstream subscribers if an internal error occurs" in {
new ChainSetup(identity, settings.copy(initialInputBufferSize = 1), (s, f) createBrokenFlowMaterializer(s, "a3")(f), new ChainSetup(identity, settings.copy(initialInputBufferSize = 1), (s, f) createBrokenActorFlowMaterializer(s, "a3")(f),
toFanoutPublisher(initialBufferSize = 1, maximumBufferSize = 16)) { toFanoutPublisher(initialBufferSize = 1, maximumBufferSize = 16)) {
def checkError(sprobe: StreamTestKit.SubscriberProbe[Any]): Unit = { def checkError(sprobe: StreamTestKit.SubscriberProbe[Any]): Unit = {

View file

@ -5,18 +5,18 @@ package akka.stream.scaladsl
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit import akka.stream.testkit.StreamTestKit
import org.reactivestreams.Publisher import org.reactivestreams.Publisher
class FlowSplitWhenSpec extends AkkaSpec { class FlowSplitWhenSpec extends AkkaSpec {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 2) .withInputBuffer(initialSize = 2, maxSize = 2)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
case class StreamPuppet(p: Publisher[Int]) { case class StreamPuppet(p: Publisher[Int]) {
val probe = StreamTestKit.SubscriberProbe[Int]() val probe = StreamTestKit.SubscriberProbe[Int]()

View file

@ -6,8 +6,8 @@ package akka.stream.scaladsl
import scala.collection.immutable.Seq import scala.collection.immutable.Seq
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.testkit.{ EventFilter, TestProbe } import akka.testkit.{ EventFilter, TestProbe }
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
@ -15,10 +15,10 @@ import akka.stream.stage._
class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.receive=off\nakka.loglevel=INFO")) { class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.receive=off\nakka.loglevel=INFO")) {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 2) .withInputBuffer(initialSize = 2, maxSize = 2)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
"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 {

View file

@ -5,8 +5,8 @@ package akka.stream.scaladsl
import scala.concurrent.forkjoin.ThreadLocalRandom.{ current random } import scala.concurrent.forkjoin.ThreadLocalRandom.{ current random }
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.actor.ActorSubscriberMessage.OnComplete import akka.stream.actor.ActorSubscriberMessage.OnComplete
import akka.stream.actor.ActorSubscriberMessage.OnNext import akka.stream.actor.ActorSubscriberMessage.OnNext
import akka.stream.impl.RequestMore import akka.stream.impl.RequestMore
@ -16,10 +16,10 @@ import akka.stream.testkit.StreamTestKit
class FlowTakeSpec extends AkkaSpec with ScriptedTest { class FlowTakeSpec extends AkkaSpec with ScriptedTest {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 16) .withInputBuffer(initialSize = 2, maxSize = 16)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
muteDeadLetters(classOf[OnNext], OnComplete.getClass, classOf[RequestMore])() muteDeadLetters(classOf[OnNext], OnComplete.getClass, classOf[RequestMore])()

View file

@ -5,13 +5,13 @@ package akka.stream.scaladsl
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.testkit.AkkaSpec import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.StreamTestKit import akka.stream.testkit.StreamTestKit
class FlowTakeWithinSpec extends AkkaSpec { class FlowTakeWithinSpec extends AkkaSpec {
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
"A TakeWithin" must { "A TakeWithin" must {

View file

@ -6,14 +6,14 @@ package akka.stream.scaladsl
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.TimerTransformer import akka.stream.TimerTransformer
import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
class FlowTimerTransformerSpec extends AkkaSpec { class FlowTimerTransformerSpec extends AkkaSpec {
implicit val materializer = FlowMaterializer() implicit val materializer = ActorFlowMaterializer()
"A Flow with TimerTransformer operations" must { "A Flow with TimerTransformer operations" must {
"produce scheduled ticks as expected" in { "produce scheduled ticks as expected" in {

View file

@ -8,17 +8,17 @@ import scala.concurrent.Future
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.Failure import scala.util.Failure
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
import akka.stream.testkit.ScriptedTest import akka.stream.testkit.ScriptedTest
class HeadSinkSpec extends AkkaSpec with ScriptedTest { class HeadSinkSpec extends AkkaSpec with ScriptedTest {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 16) .withInputBuffer(initialSize = 2, maxSize = 16)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
"A Flow with Sink.head" must { "A Flow with Sink.head" must {

View file

@ -4,17 +4,17 @@ import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import FlowGraphImplicits._ import FlowGraphImplicits._
import akka.stream.FlowMaterializer import akka.stream.ActorFlowMaterializer
import akka.stream.MaterializerSettings import akka.stream.ActorFlowMaterializerSettings
import akka.stream.testkit.{ AkkaSpec, StreamTestKit } import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
class GraphBalanceSpec extends AkkaSpec { class GraphBalanceSpec extends AkkaSpec {
val settings = MaterializerSettings(system) val settings = ActorFlowMaterializerSettings(system)
.withInputBuffer(initialSize = 2, maxSize = 16) .withInputBuffer(initialSize = 2, maxSize = 16)
implicit val materializer = FlowMaterializer(settings) implicit val materializer = ActorFlowMaterializer(settings)
"A balance" must { "A balance" must {

Some files were not shown because too many files have changed in this diff Show more