!str #15851 Rename FlowMaterializer and settings

* FlowMaterializer is now the actor independent interface
* ActorFlowMaterializer is the actor based interface
* MaterializerSettings renamed to ActorFlowMaterializerSettings
* impl.ActorBasedFlowMaterializer renamed to impl.ActorFlowMaterializerImpl
* Optimizations included in ActorFlowMaterializerSettings
* Note that http is using FlowMaterializer in api, but I suspect that it
  will currently only run with a ActorFlowMaterializer
This commit is contained in:
Patrik Nordwall 2015-01-27 18:29:20 +01:00
parent 5b72928439
commit cd9d503b03
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
-----------------------------------------
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:
.. 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
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`.
@ -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
Materializing and running a stream always requires a :class:`FlowMaterializer` to be in implicit scope (or passed in explicitly,
like this: ``.run(mat)``).
Materializing and running a stream always requires a :class:`FlowMaterializer` to be passed in explicitly,
like this: ``.run(mat)``.
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``,
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
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``,
which can be used to retrieve materialized values from the running stream.

View file

@ -16,10 +16,10 @@ class HttpServerExampleSpec
"binding example" in {
//#bind-example
import akka.http.Http
import akka.stream.FlowMaterializer
import akka.stream.ActorFlowMaterializer
implicit val system = ActorSystem()
implicit val materializer = FlowMaterializer()
implicit val materializer = ActorFlowMaterializer()
val serverBinding = Http(system).bind(interface = "localhost", port = 8080)
serverBinding.connections.foreach { connection => // foreach materializes the source
@ -30,10 +30,10 @@ class HttpServerExampleSpec
"full-server-example" in {
import akka.http.Http
import akka.stream.FlowMaterializer
import akka.stream.ActorFlowMaterializer
implicit val system = ActorSystem()
implicit val materializer = FlowMaterializer()
implicit val materializer = ActorFlowMaterializer()
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.http.server.Route
import akka.stream.FlowMaterializer
import akka.stream.ActorFlowMaterializer
object MyHandler {
//# example-1
@ -27,7 +27,7 @@ object MyHandler {
object MyApp {
implicit val system = ActorSystem()
import system.dispatcher
implicit val materializer = FlowMaterializer()
implicit val materializer = ActorFlowMaterializer()
def handler = Route.handlerFlow(`<my-route-definition>`)
}

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -1,6 +1,6 @@
package docs.stream.cookbook
import akka.stream.{ MaterializerSettings, FlowMaterializer }
import akka.stream.{ ActorFlowMaterializerSettings, ActorFlowMaterializer }
import akka.stream.scaladsl._
import akka.stream.testkit.StreamTestKit
import akka.stream.testkit.StreamTestKit.{ SubscriberProbe, PublisherProbe }
@ -11,7 +11,7 @@ import scala.concurrent.duration._
class RecipeCollectingMetrics extends RecipeSpec {
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 {

View file

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

View file

@ -190,7 +190,7 @@ well-known sinks, such as ``foreach(el => )`` (being an alias to ``runWith(Sink.
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,
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::
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 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``:
@ -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 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:

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
-----------------------------------------
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:
.. 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
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`.
@ -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``,
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
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``,
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
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