!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:
parent
5b72928439
commit
cd9d503b03
146 changed files with 601 additions and 595 deletions
|
|
@ -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.
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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>`)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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>`)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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 =
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
|
|
|
|||
|
|
@ -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 })
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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,
|
||||
|
|
|
|||
|
|
@ -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:
|
||||
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue