!str #18692 javadsl.FlowGraph.Builder.add()

* also make factories more consistent by only offering
  FlowGraph.create()
* also remove secondary (edge-based) FlowGraph.Builder DSL
* also improve naming for conversions from Graph to
  Source/Flow/BidiFlow/Sink
This commit is contained in:
Viktor Klang 2015-10-21 22:45:39 +02:00 committed by Roland Kuhn
parent 0f99a42df9
commit f29d7affbd
120 changed files with 1535 additions and 1897 deletions

View file

@ -44,21 +44,21 @@ object BidiFlowDocSpec {
}
//#codec-impl
val codecVerbose = BidiFlow() { b =>
val codecVerbose = BidiFlow.fromGraph(FlowGraph.create() { b =>
// construct and add the top flow, going outbound
val outbound = b.add(Flow[Message].map(toBytes))
// construct and add the bottom flow, going inbound
val inbound = b.add(Flow[ByteString].map(fromBytes))
// fuse them together into a BidiShape
BidiShape.fromFlows(outbound, inbound)
}
})
// this is the same as the above
val codec = BidiFlow(toBytes _, fromBytes _)
val codec = BidiFlow.fromFunctions(toBytes _, fromBytes _)
//#codec
//#framing
val framing = BidiFlow() { b =>
val framing = BidiFlow.fromGraph(FlowGraph.create() { b =>
implicit val order = ByteOrder.LITTLE_ENDIAN
def addLengthHeader(bytes: ByteString) = {
@ -113,18 +113,18 @@ object BidiFlowDocSpec {
val outbound = b.add(Flow[ByteString].map(addLengthHeader))
val inbound = b.add(Flow[ByteString].transform(() => new FrameParser))
BidiShape.fromFlows(outbound, inbound)
}
})
//#framing
val chopUp = BidiFlow() { b =>
val chopUp = BidiFlow.fromGraph(FlowGraph.create() { b =>
val f = Flow[ByteString].mapConcat(_.map(ByteString(_)))
BidiShape.fromFlows(b.add(f), b.add(f))
}
})
val accumulate = BidiFlow() { b =>
val accumulate = BidiFlow.fromGraph(FlowGraph.create() { b =>
val f = Flow[ByteString].grouped(1000).map(_.fold(ByteString.empty)(_ ++ _))
BidiShape.fromFlows(b.add(f), b.add(f))
}
})
}
class BidiFlowDocSpec extends AkkaSpec with ConversionCheckedTripleEquals {

View file

@ -77,7 +77,7 @@ class CompositionDocSpec extends AkkaSpec {
// format: OFF
//#complex-graph
import FlowGraph.Implicits._
FlowGraph.closed() { implicit builder =>
RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
val A: Outlet[Int] = builder.add(Source.single(0))
val B: UniformFanOutShape[Int, Int] = builder.add(Broadcast[Int](2))
val C: UniformFanInShape[Int, Int] = builder.add(Merge[Int](2))
@ -90,12 +90,14 @@ class CompositionDocSpec extends AkkaSpec {
A ~> B ~> C ~> F
B ~> D ~> E ~> F
E ~> G
}
ClosedShape
})
//#complex-graph
//#complex-graph-alt
import FlowGraph.Implicits._
FlowGraph.closed() { implicit builder =>
RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
val B = builder.add(Broadcast[Int](2))
val C = builder.add(Merge[Int](2))
val E = builder.add(Balance[Int](2))
@ -106,7 +108,8 @@ class CompositionDocSpec extends AkkaSpec {
B.out(1).map(_ + 1) ~> E.in; E.out(0) ~> F.in(1)
E.out(1) ~> Sink.foreach(println)
}
ClosedShape
})
//#complex-graph-alt
// format: ON
}
@ -115,7 +118,7 @@ class CompositionDocSpec extends AkkaSpec {
// format: OFF
//#partial-graph
import FlowGraph.Implicits._
val partial = FlowGraph.partial() { implicit builder =>
val partial = FlowGraph.create() { implicit builder =>
val B = builder.add(Broadcast[Int](2))
val C = builder.add(Merge[Int](2))
val E = builder.add(Balance[Int](2))
@ -137,17 +140,17 @@ class CompositionDocSpec extends AkkaSpec {
//#partial-flow-dsl
// Convert the partial graph of FlowShape to a Flow to get
// access to the fluid DSL (for example to be able to call .filter())
val flow = Flow.wrap(partial)
val flow = Flow.fromGraph(partial)
// Simple way to create a graph backed Source
val source = Source() { implicit builder =>
val source = Source.fromGraph( FlowGraph.create() { implicit builder =>
val merge = builder.add(Merge[Int](2))
Source.single(0) ~> merge
Source(List(2, 3, 4)) ~> merge
// Exposing exactly one output port
merge.out
}
SourceShape(merge.out)
})
// Building a Sink with a nested Flow, using the fluid DSL
val sink = {
@ -164,22 +167,24 @@ class CompositionDocSpec extends AkkaSpec {
"closed graph" in {
//#embed-closed
val closed1 = Source.single(0).to(Sink.foreach(println))
val closed2 = FlowGraph.closed() { implicit builder =>
val closed2 = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
val embeddedClosed: ClosedShape = builder.add(closed1)
}
//
embeddedClosed
})
//#embed-closed
}
"materialized values" in {
//#mat-combine-1
// Materializes to Promise[Unit] (red)
val source: Source[Int, Promise[Unit]] = Source.lazyEmpty[Int]
// Materializes to Promise[Option[Int]] (red)
val source: Source[Int, Promise[Option[Int]]] = Source.maybe[Int]
// Materializes to Unit (black)
val flow1: Flow[Int, Int, Unit] = Flow[Int].take(100)
// Materializes to Promise[Unit] (red)
val nestedSource: Source[Int, Promise[Unit]] =
// Materializes to Promise[Int] (red)
val nestedSource: Source[Int, Promise[Option[Int]]] =
source.viaMat(flow1)(Keep.left).named("nestedSource")
//#mat-combine-1
@ -206,11 +211,11 @@ class CompositionDocSpec extends AkkaSpec {
//#mat-combine-3
//#mat-combine-4
case class MyClass(private val p: Promise[Unit], conn: OutgoingConnection) {
def close() = p.success(())
case class MyClass(private val p: Promise[Option[Int]], conn: OutgoingConnection) {
def close() = p.trySuccess(None)
}
def f(p: Promise[Unit],
def f(p: Promise[Option[Int]],
rest: (Future[OutgoingConnection], Future[String])): Future[MyClass] = {
val connFuture = rest._1

View file

@ -4,6 +4,7 @@
package docs.stream
import akka.actor.Cancellable
import akka.stream.{ ClosedShape, FlowShape }
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
@ -147,17 +148,17 @@ class FlowDocSpec extends AkkaSpec {
"various ways of transforming materialized values" in {
import scala.concurrent.duration._
val throttler = Flow(Source(1.second, 1.second, "test")) { implicit builder =>
val throttler = Flow.fromGraph(FlowGraph.create(Source(1.second, 1.second, "test")) { implicit builder =>
tickSource =>
import FlowGraph.Implicits._
val zip = builder.add(ZipWith[String, Int, Int](Keep.right))
tickSource ~> zip.in0
(zip.in1, zip.out)
}
FlowShape(zip.in1, zip.out)
})
//#flow-mat-combine
// An empty source that can be shut down explicitly from the outside
val source: Source[Int, Promise[Unit]] = Source.lazyEmpty[Int]
// An source that can be signalled explicitly from the outside
val source: Source[Int, Promise[Option[Int]]] = Source.maybe[Int]
// A flow that internally throttles elements to 1/second, and returns a Cancellable
// which can be used to shut down the stream
@ -167,7 +168,7 @@ class FlowDocSpec extends AkkaSpec {
val sink: Sink[Int, Future[Int]] = Sink.head[Int]
// By default, the materialized value of the leftmost stage is preserved
val r1: RunnableGraph[Promise[Unit]] = source.via(flow).to(sink)
val r1: RunnableGraph[Promise[Option[Int]]] = source.via(flow).to(sink)
// Simple selection of materialized values by using Keep.right
val r2: RunnableGraph[Cancellable] = source.viaMat(flow)(Keep.right).to(sink)
@ -176,17 +177,17 @@ class FlowDocSpec extends AkkaSpec {
// Using runWith will always give the materialized values of the stages added
// by runWith() itself
val r4: Future[Int] = source.via(flow).runWith(sink)
val r5: Promise[Unit] = flow.to(sink).runWith(source)
val r6: (Promise[Unit], Future[Int]) = flow.runWith(source, sink)
val r5: Promise[Option[Int]] = flow.to(sink).runWith(source)
val r6: (Promise[Option[Int]], Future[Int]) = flow.runWith(source, sink)
// Using more complext combinations
val r7: RunnableGraph[(Promise[Unit], Cancellable)] =
val r7: RunnableGraph[(Promise[Option[Int]], Cancellable)] =
source.viaMat(flow)(Keep.both).to(sink)
val r8: RunnableGraph[(Promise[Unit], Future[Int])] =
val r8: RunnableGraph[(Promise[Option[Int]], Future[Int])] =
source.via(flow).toMat(sink)(Keep.both)
val r9: RunnableGraph[((Promise[Unit], Cancellable), Future[Int])] =
val r9: RunnableGraph[((Promise[Option[Int]], Cancellable), Future[Int])] =
source.viaMat(flow)(Keep.both).toMat(sink)(Keep.both)
val r10: RunnableGraph[(Cancellable, Future[Int])] =
@ -194,7 +195,7 @@ class FlowDocSpec extends AkkaSpec {
// It is also possible to map over the materialized values. In r9 we had a
// doubly nested pair, but we want to flatten it out
val r11: RunnableGraph[(Promise[Unit], Cancellable, Future[Int])] =
val r11: RunnableGraph[(Promise[Option[Int]], Cancellable, Future[Int])] =
r9.mapMaterializedValue {
case ((promise, cancellable), future) =>
(promise, cancellable, future)
@ -204,17 +205,18 @@ class FlowDocSpec extends AkkaSpec {
val (promise, cancellable, future) = r11.run()
// Type inference works as expected
promise.success(())
promise.success(None)
cancellable.cancel()
future.map(_ + 3)
// The result of r11 can be also achieved by using the Graph API
val r12: RunnableGraph[(Promise[Unit], Cancellable, Future[Int])] =
FlowGraph.closed(source, flow, sink)((_, _, _)) { implicit builder =>
val r12: RunnableGraph[(Promise[Option[Int]], Cancellable, Future[Int])] =
RunnableGraph.fromGraph(FlowGraph.create(source, flow, sink)((_, _, _)) { implicit builder =>
(src, f, dst) =>
import FlowGraph.Implicits._
src ~> f ~> dst
}
ClosedShape
})
//#flow-mat-combine
}

View file

@ -21,7 +21,7 @@ class FlowGraphDocSpec extends AkkaSpec {
"build simple graph" in {
//format: OFF
//#simple-flow-graph
val g = FlowGraph.closed() { implicit builder: FlowGraph.Builder[Unit] =>
val g = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder: FlowGraph.Builder[Unit] =>
import FlowGraph.Implicits._
val in = Source(1 to 10)
val out = Sink.ignore
@ -32,8 +32,9 @@ class FlowGraphDocSpec extends AkkaSpec {
val f1, f2, f3, f4 = Flow[Int].map(_ + 10)
in ~> f1 ~> bcast ~> f2 ~> merge ~> f3 ~> out
bcast ~> f4 ~> merge
}
bcast ~> f4 ~> merge
ClosedShape
})
//#simple-flow-graph
//format: ON
@ -42,33 +43,10 @@ class FlowGraphDocSpec extends AkkaSpec {
//#simple-graph-run
}
"build simple graph without implicits" in {
//#simple-flow-graph-no-implicits
val g = FlowGraph.closed() { builder: FlowGraph.Builder[Unit] =>
val in = Source(1 to 10)
val out = Sink.ignore
val broadcast = builder.add(Broadcast[Int](2))
val merge = builder.add(Merge[Int](2))
val f1 = Flow[Int].map(_ + 10)
val f3 = Flow[Int].map(_.toString)
val f2 = Flow[Int].map(_ + 20)
builder.addEdge(builder.add(in), broadcast.in)
builder.addEdge(broadcast.out(0), f1, merge.in(0))
builder.addEdge(broadcast.out(1), f2, merge.in(1))
builder.addEdge(merge.out, f3, builder.add(out))
}
//#simple-flow-graph-no-implicits
g.run()
}
"flow connection errors" in {
intercept[IllegalArgumentException] {
//#simple-graph
FlowGraph.closed() { implicit builder =>
RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
import FlowGraph.Implicits._
val source1 = Source(1 to 10)
val source2 = Source(1 to 10)
@ -78,9 +56,10 @@ class FlowGraphDocSpec extends AkkaSpec {
source1 ~> zip.in0
source2 ~> zip.in1
// unconnected zip.out (!) => "must have at least 1 outgoing edge"
}
ClosedShape
})
//#simple-graph
}.getMessage should include("unconnected ports: ZipWith2.out")
}.getMessage should include("ZipWith2.out")
}
"reusing a flow in a graph" in {
@ -95,7 +74,7 @@ class FlowGraphDocSpec extends AkkaSpec {
// format: OFF
val g =
//#flow-graph-reusing-a-flow
FlowGraph.closed(topHeadSink, bottomHeadSink)((_, _)) { implicit builder =>
RunnableGraph.fromGraph(FlowGraph.create(topHeadSink, bottomHeadSink)((_, _)) { implicit builder =>
(topHS, bottomHS) =>
import FlowGraph.Implicits._
val broadcast = builder.add(Broadcast[Int](2))
@ -103,7 +82,8 @@ class FlowGraphDocSpec extends AkkaSpec {
broadcast.out(0) ~> sharedDoubler ~> topHS.inlet
broadcast.out(1) ~> sharedDoubler ~> bottomHS.inlet
}
ClosedShape
})
//#flow-graph-reusing-a-flow
// format: ON
val (topFuture, bottomFuture) = g.run()
@ -153,7 +133,7 @@ class FlowGraphDocSpec extends AkkaSpec {
worker: Flow[In, Out, Any],
workerCount: Int): Graph[PriorityWorkerPoolShape[In, Out], Unit] = {
FlowGraph.partial() { implicit b
FlowGraph.create() { implicit b
import FlowGraph.Implicits._
val priorityMerge = b.add(MergePreferred[In](1))
@ -188,7 +168,7 @@ class FlowGraphDocSpec extends AkkaSpec {
val worker1 = Flow[String].map("step 1 " + _)
val worker2 = Flow[String].map("step 2 " + _)
FlowGraph.closed() { implicit b =>
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._
val priorityPool1 = b.add(PriorityWorkerPool(worker1, 4))
@ -201,7 +181,8 @@ class FlowGraphDocSpec extends AkkaSpec {
Source(1 to 100).map("one-step, priority " + _) ~> priorityPool2.priorityJobsIn
priorityPool2.resultsOut ~> Sink.foreach(println)
}.run()
ClosedShape
}).run()
//#flow-graph-components-use
//#flow-graph-components-shape2
@ -223,11 +204,11 @@ class FlowGraphDocSpec extends AkkaSpec {
"access to materialized value" in {
//#flow-graph-matvalue
import FlowGraph.Implicits._
val foldFlow: Flow[Int, Int, Future[Int]] = Flow(Sink.fold[Int, Int](0)(_ + _)) {
val foldFlow: Flow[Int, Int, Future[Int]] = Flow.fromGraph(FlowGraph.create(Sink.fold[Int, Int](0)(_ + _)) {
implicit builder
fold
(fold.inlet, builder.materializedValue.mapAsync(4)(identity).outlet)
}
FlowShape(fold.inlet, builder.materializedValue.mapAsync(4)(identity).outlet)
})
//#flow-graph-matvalue
Await.result(Source(1 to 10).via(foldFlow).runWith(Sink.head), 3.seconds) should ===(55)
@ -235,7 +216,7 @@ class FlowGraphDocSpec extends AkkaSpec {
//#flow-graph-matvalue-cycle
import FlowGraph.Implicits._
// This cannot produce any value:
val cyclicFold: Source[Int, Future[Int]] = Source(Sink.fold[Int, Int](0)(_ + _)) {
val cyclicFold: Source[Int, Future[Int]] = Source.fromGraph(FlowGraph.create(Sink.fold[Int, Int](0)(_ + _)) {
implicit builder =>
fold =>
// - Fold cannot complete until its upstream mapAsync completes
@ -244,8 +225,8 @@ class FlowGraphDocSpec extends AkkaSpec {
// As a result this Source will never emit anything, and its materialited
// Future will never complete
builder.materializedValue.mapAsync(4)(identity) ~> fold
builder.materializedValue.mapAsync(4)(identity).outlet
}
SourceShape(builder.materializedValue.mapAsync(4)(identity).outlet)
})
//#flow-graph-matvalue-cycle
}

View file

@ -1,5 +1,6 @@
package docs.stream
import akka.stream.FlowShape
import akka.stream.scaladsl.{ FlowGraph, Merge, Balance, Source, Flow }
import akka.stream.testkit.AkkaSpec
@ -37,7 +38,7 @@ class FlowParallelismDocSpec extends AkkaSpec {
val fryingPan: Flow[ScoopOfBatter, Pancake, Unit] =
Flow[ScoopOfBatter].map { batter => Pancake() }
val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] = Flow() { implicit builder =>
val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] = Flow.fromGraph(FlowGraph.create() { implicit builder =>
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
val mergePancakes = builder.add(Merge[Pancake](2))
@ -49,54 +50,57 @@ class FlowParallelismDocSpec extends AkkaSpec {
// uses of "fryingPan" mean actually different stages in the graph.
dispatchBatter.out(1) ~> fryingPan ~> mergePancakes.in(1)
(dispatchBatter.in, mergePancakes.out)
}
FlowShape(dispatchBatter.in, mergePancakes.out)
})
//#parallelism
}
"Demonstrate parallelized pipelines" in {
//#parallel-pipeline
val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] = Flow() { implicit builder =>
val pancakeChef: Flow[ScoopOfBatter, Pancake, Unit] =
Flow.fromGraph(FlowGraph.create() { implicit builder =>
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
val mergePancakes = builder.add(Merge[Pancake](2))
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
val mergePancakes = builder.add(Merge[Pancake](2))
// Using two pipelines, having two frying pans each, in total using
// four frying pans
dispatchBatter.out(0) ~> fryingPan1 ~> fryingPan2 ~> mergePancakes.in(0)
dispatchBatter.out(1) ~> fryingPan1 ~> fryingPan2 ~> mergePancakes.in(1)
// Using two pipelines, having two frying pans each, in total using
// four frying pans
dispatchBatter.out(0) ~> fryingPan1 ~> fryingPan2 ~> mergePancakes.in(0)
dispatchBatter.out(1) ~> fryingPan1 ~> fryingPan2 ~> mergePancakes.in(1)
(dispatchBatter.in, mergePancakes.out)
}
FlowShape(dispatchBatter.in, mergePancakes.out)
})
//#parallel-pipeline
}
"Demonstrate pipelined parallel processing" in {
//#pipelined-parallel
val pancakeChefs1: Flow[ScoopOfBatter, HalfCookedPancake, Unit] = Flow() { implicit builder =>
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
val mergeHalfPancakes = builder.add(Merge[HalfCookedPancake](2))
val pancakeChefs1: Flow[ScoopOfBatter, HalfCookedPancake, Unit] =
Flow.fromGraph(FlowGraph.create() { implicit builder =>
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
val mergeHalfPancakes = builder.add(Merge[HalfCookedPancake](2))
// Two chefs work with one frying pan for each, half-frying the pancakes then putting
// them into a common pool
dispatchBatter.out(0) ~> fryingPan1 ~> mergeHalfPancakes.in(0)
dispatchBatter.out(1) ~> fryingPan1 ~> mergeHalfPancakes.in(1)
// Two chefs work with one frying pan for each, half-frying the pancakes then putting
// them into a common pool
dispatchBatter.out(0) ~> fryingPan1 ~> mergeHalfPancakes.in(0)
dispatchBatter.out(1) ~> fryingPan1 ~> mergeHalfPancakes.in(1)
(dispatchBatter.in, mergeHalfPancakes.out)
}
FlowShape(dispatchBatter.in, mergeHalfPancakes.out)
})
val pancakeChefs2: Flow[HalfCookedPancake, Pancake, Unit] = Flow() { implicit builder =>
val dispatchHalfPancakes = builder.add(Balance[HalfCookedPancake](2))
val mergePancakes = builder.add(Merge[Pancake](2))
val pancakeChefs2: Flow[HalfCookedPancake, Pancake, Unit] =
Flow.fromGraph(FlowGraph.create() { implicit builder =>
val dispatchHalfPancakes = builder.add(Balance[HalfCookedPancake](2))
val mergePancakes = builder.add(Merge[Pancake](2))
// Two chefs work with one frying pan for each, finishing the pancakes then putting
// them into a common pool
dispatchHalfPancakes.out(0) ~> fryingPan2 ~> mergePancakes.in(0)
dispatchHalfPancakes.out(1) ~> fryingPan2 ~> mergePancakes.in(1)
// Two chefs work with one frying pan for each, finishing the pancakes then putting
// them into a common pool
dispatchHalfPancakes.out(0) ~> fryingPan2 ~> mergePancakes.in(0)
dispatchHalfPancakes.out(1) ~> fryingPan2 ~> mergePancakes.in(1)
(dispatchHalfPancakes.in, mergePancakes.out)
}
FlowShape(dispatchHalfPancakes.in, mergePancakes.out)
})
val kitchen: Flow[ScoopOfBatter, Pancake, Unit] = pancakeChefs1.via(pancakeChefs2)
//#pipelined-parallel

View file

@ -1,6 +1,6 @@
package docs.stream
import akka.stream.{ OverflowStrategy, ActorMaterializer }
import akka.stream.{ ClosedShape, OverflowStrategy, ActorMaterializer }
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
@ -16,7 +16,7 @@ class GraphCyclesSpec extends AkkaSpec {
// format: OFF
//#deadlocked
// WARNING! The graph below deadlocks!
FlowGraph.closed() { implicit b =>
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._
val merge = b.add(Merge[Int](2))
@ -24,7 +24,8 @@ class GraphCyclesSpec extends AkkaSpec {
source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore
merge <~ bcast
}
ClosedShape
})
//#deadlocked
// format: ON
}
@ -33,7 +34,7 @@ class GraphCyclesSpec extends AkkaSpec {
// format: OFF
//#unfair
// WARNING! The graph below stops consuming from "source" after a few steps
FlowGraph.closed() { implicit b =>
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._
val merge = b.add(MergePreferred[Int](1))
@ -41,7 +42,8 @@ class GraphCyclesSpec extends AkkaSpec {
source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore
merge.preferred <~ bcast
}
ClosedShape
})
//#unfair
// format: ON
}
@ -49,7 +51,7 @@ class GraphCyclesSpec extends AkkaSpec {
"include a dropping cycle" in {
// format: OFF
//#dropping
FlowGraph.closed() { implicit b =>
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._
val merge = b.add(Merge[Int](2))
@ -57,7 +59,8 @@ class GraphCyclesSpec extends AkkaSpec {
source ~> merge ~> Flow[Int].map { s => println(s); s } ~> bcast ~> Sink.ignore
merge <~ Flow[Int].buffer(10, OverflowStrategy.dropHead) <~ bcast
}
ClosedShape
})
//#dropping
// format: ON
}
@ -66,7 +69,7 @@ class GraphCyclesSpec extends AkkaSpec {
// format: OFF
//#zipping-dead
// WARNING! The graph below never processes any elements
FlowGraph.closed() { implicit b =>
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._
val zip = b.add(ZipWith[Int, Int, Int]((left, right) => right))
@ -75,7 +78,8 @@ class GraphCyclesSpec extends AkkaSpec {
source ~> zip.in0
zip.out.map { s => println(s); s } ~> bcast ~> Sink.ignore
zip.in1 <~ bcast
}
ClosedShape
})
//#zipping-dead
// format: ON
}
@ -83,7 +87,7 @@ class GraphCyclesSpec extends AkkaSpec {
"include a live zipping cycle" in {
// format: OFF
//#zipping-live
FlowGraph.closed() { implicit b =>
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._
val zip = b.add(ZipWith((left: Int, right: Int) => left))
@ -95,7 +99,8 @@ class GraphCyclesSpec extends AkkaSpec {
zip.out.map { s => println(s); s } ~> bcast ~> Sink.ignore
zip.in1 <~ concat <~ start
concat <~ bcast
}
ClosedShape
})
//#zipping-live
// format: ON
}

View file

@ -1,9 +1,8 @@
package docs.stream
import akka.stream.{ OverflowStrategy, ActorMaterializerSettings, ActorMaterializer }
import akka.stream._
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.stream.Attributes
class StreamBuffersRateSpec extends AkkaSpec {
implicit val mat = ActorMaterializer()
@ -40,7 +39,7 @@ class StreamBuffersRateSpec extends AkkaSpec {
import scala.concurrent.duration._
case class Tick()
FlowGraph.closed() { implicit b =>
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._
val zipper = b.add(ZipWith[Tick, Int, Int]((tick, count) => count))
@ -51,7 +50,8 @@ class StreamBuffersRateSpec extends AkkaSpec {
.conflate(seed = (_) => 1)((count, _) => count + 1) ~> zipper.in1
zipper.out ~> Sink.foreach(println)
}
ClosedShape
})
//#buffering-abstraction-leak
}

View file

@ -19,7 +19,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
"build with open ports" in {
//#simple-partial-flow-graph
val pickMaxOfThree = FlowGraph.partial() { implicit b =>
val pickMaxOfThree = FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._
val zip1 = b.add(ZipWith[Int, Int, Int](math.max _))
@ -31,7 +31,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
val resultSink = Sink.head[Int]
val g = FlowGraph.closed(resultSink) { implicit b =>
val g = RunnableGraph.fromGraph(FlowGraph.create(resultSink) { implicit b =>
sink =>
import FlowGraph.Implicits._
@ -42,7 +42,8 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
Source.single(2) ~> pm3.in(1)
Source.single(3) ~> pm3.in(2)
pm3.out ~> sink.inlet
}
ClosedShape
})
val max: Future[Int] = g.run()
Await.result(max, 300.millis) should equal(3)
@ -51,7 +52,7 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
"build source from partial flow graph" in {
//#source-from-partial-flow-graph
val pairs = Source() { implicit b =>
val pairs = Source.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._
// prepare graph elements
@ -63,8 +64,8 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
ints.filter(_ % 2 == 0) ~> zip.in1
// expose port
zip.out
}
SourceShape(zip.out)
})
val firstPair: Future[(Int, Int)] = pairs.runWith(Sink.head)
//#source-from-partial-flow-graph
@ -73,20 +74,21 @@ class StreamPartialFlowGraphDocSpec extends AkkaSpec {
"build flow from partial flow graph" in {
//#flow-from-partial-flow-graph
val pairUpWithToString = Flow() { implicit b =>
import FlowGraph.Implicits._
val pairUpWithToString =
Flow.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._
// prepare graph elements
val broadcast = b.add(Broadcast[Int](2))
val zip = b.add(Zip[Int, String]())
// prepare graph elements
val broadcast = b.add(Broadcast[Int](2))
val zip = b.add(Zip[Int, String]())
// connect the graph
broadcast.out(0).map(identity) ~> zip.in0
broadcast.out(1).map(_.toString) ~> zip.in1
// connect the graph
broadcast.out(0).map(identity) ~> zip.in0
broadcast.out(1).map(_.toString) ~> zip.in1
// expose ports
(broadcast.in, zip.out)
}
// expose ports
FlowShape(broadcast.in, zip.out)
})
//#flow-from-partial-flow-graph

View file

@ -6,8 +6,7 @@ package docs.stream
//#imports
import akka.actor.ActorSystem
import akka.stream.ActorMaterializer
import akka.stream.OverflowStrategy
import akka.stream.{ ClosedShape, ActorMaterializer, OverflowStrategy }
import akka.stream.scaladsl._
import scala.concurrent.Await
@ -119,14 +118,15 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
// format: OFF
//#flow-graph-broadcast
val g = FlowGraph.closed() { implicit b =>
val g = RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._
val bcast = b.add(Broadcast[Tweet](2))
tweets ~> bcast.in
bcast.out(0) ~> Flow[Tweet].map(_.author) ~> writeAuthors
bcast.out(1) ~> Flow[Tweet].mapConcat(_.hashtags.toList) ~> writeHashtags
}
ClosedShape
})
g.run()
//#flow-graph-broadcast
// format: ON

View file

@ -1,6 +1,6 @@
package docs.stream.cookbook
import akka.stream.OverflowStrategy
import akka.stream.{ ClosedShape, OverflowStrategy }
import akka.stream.scaladsl._
import akka.stream.testkit._
@ -24,7 +24,7 @@ class RecipeDroppyBroadcast extends RecipeSpec {
val mySink3 = Sink(sub3)
//#droppy-bcast
val graph = FlowGraph.closed(mySink1, mySink2, mySink3)((_, _, _)) { implicit b =>
val graph = RunnableGraph.fromGraph(FlowGraph.create(mySink1, mySink2, mySink3)((_, _, _)) { implicit b =>
(sink1, sink2, sink3) =>
import FlowGraph.Implicits._
@ -34,7 +34,8 @@ class RecipeDroppyBroadcast extends RecipeSpec {
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink1
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink2
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink3
}
ClosedShape
})
//#droppy-bcast
graph.run()

View file

@ -2,6 +2,7 @@ package docs.stream.cookbook
import akka.actor.{ Props, ActorRef, Actor }
import akka.actor.Actor.Receive
import akka.stream.ClosedShape
import akka.stream.scaladsl._
import akka.stream.testkit._
@ -98,12 +99,13 @@ class RecipeGlobalRateLimit extends RecipeSpec {
val probe = TestSubscriber.manualProbe[String]()
FlowGraph.closed() { implicit b =>
RunnableGraph.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._
val merge = b.add(Merge[String](2))
source1 ~> merge ~> Sink(probe)
source2 ~> merge
}.run()
ClosedShape
}).run()
probe.expectSubscription().request(1000)

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook
import akka.stream.ClosedShape
import akka.stream.scaladsl._
import akka.stream.testkit._
import akka.util.ByteString
@ -25,14 +26,15 @@ class RecipeKeepAlive extends RecipeSpec {
val tickToKeepAlivePacket: Flow[Tick, ByteString, Unit] = Flow[Tick]
.conflate(seed = (tick) => keepaliveMessage)((msg, newTick) => msg)
val graph = FlowGraph.closed() { implicit builder =>
val graph = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
import FlowGraph.Implicits._
val unfairMerge = builder.add(MergePreferred[ByteString](1))
// If data is available then no keepalive is injected
dataStream ~> unfairMerge.preferred
ticks ~> tickToKeepAlivePacket ~> unfairMerge ~> sink
}
ClosedShape
})
//#inject-keepalive
graph.run()

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook
import akka.stream.ClosedShape
import akka.stream.scaladsl._
import akka.stream.testkit._
import scala.concurrent.duration._
@ -17,13 +18,14 @@ class RecipeManualTrigger extends RecipeSpec {
val sink = Sink(sub)
//#manually-triggered-stream
val graph = FlowGraph.closed() { implicit builder =>
val graph = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
import FlowGraph.Implicits._
val zip = builder.add(Zip[Message, Trigger]())
elements ~> zip.in0
triggerSource ~> zip.in1
zip.out ~> Flow[(Message, Trigger)].map { case (msg, trigger) => msg } ~> sink
}
ClosedShape
})
//#manually-triggered-stream
graph.run()
@ -55,14 +57,15 @@ class RecipeManualTrigger extends RecipeSpec {
val sink = Sink(sub)
//#manually-triggered-stream-zipwith
val graph = FlowGraph.closed() { implicit builder =>
val graph = RunnableGraph.fromGraph(FlowGraph.create() { implicit builder =>
import FlowGraph.Implicits._
val zip = builder.add(ZipWith((msg: Message, trigger: Trigger) => msg))
elements ~> zip.in0
triggerSource ~> zip.in1
zip.out ~> sink
}
ClosedShape
})
//#manually-triggered-stream-zipwith
graph.run()

View file

@ -1,5 +1,6 @@
package docs.stream.cookbook
import akka.stream.FlowShape
import akka.stream.scaladsl._
import akka.testkit.TestProbe
@ -20,7 +21,7 @@ class RecipeWorkerPool extends RecipeSpec {
def balancer[In, Out](worker: Flow[In, Out, Any], workerCount: Int): Flow[In, Out, Unit] = {
import FlowGraph.Implicits._
Flow() { implicit b =>
Flow.fromGraph(FlowGraph.create() { implicit b =>
val balancer = b.add(Balance[In](workerCount, waitForAllDownstreams = true))
val merge = b.add(Merge[Out](workerCount))
@ -30,8 +31,8 @@ class RecipeWorkerPool extends RecipeSpec {
balancer ~> worker ~> merge
}
(balancer.in, merge.out)
}
FlowShape(balancer.in, merge.out)
})
}
val processedJobs: Source[Result, Unit] = myJobs.via(balancer(worker, 3))

View file

@ -73,7 +73,7 @@ class StreamTcpDocSpec extends AkkaSpec {
connections runForeach { connection =>
val serverLogic = Flow() { implicit b =>
val serverLogic = Flow.fromGraph(FlowGraph.create() { implicit b =>
import FlowGraph.Implicits._
// server logic, parses incoming commands
@ -109,8 +109,8 @@ class StreamTcpDocSpec extends AkkaSpec {
// then we continue using the echo-logic Flow
echo.outlet ~> concat.in(1)
(echo.inlet, concat.out)
}
FlowShape(echo.inlet, concat.out)
})
connection.handleWith(serverLogic)
}

View file

@ -126,8 +126,8 @@ As a first example, let's look at a more complex layout:
The diagram shows a :class:`RunnableGraph` (remember, if there are no unwired ports, the graph is closed, and therefore
can be materialized) that encapsulates a non-trivial stream processing network. It contains fan-in, fan-out stages,
directed and non-directed cycles. The ``closed()`` method of the :class:`FlowGraph` object allows the creation of a
general closed graph. For example the network on the diagram can be realized like this:
directed and non-directed cycles. The ``runnable()`` method of the :class:`FlowGraph` object allows the creation of a
general, closed, and runnable graph. For example the network on the diagram can be realized like this:
.. includecode:: code/docs/stream/CompositionDocSpec.scala#complex-graph
@ -141,7 +141,7 @@ explicitly, and it is not necessary to import our linear stages via ``add()``, s
Similar to the case in the first section, so far we have not considered modularity. We created a complex graph, but
the layout is flat, not modularized. We will modify our example, and create a reusable component with the graph DSL.
The way to do it is to use the ``partial()`` factory method on :class:`FlowGraph`. If we remove the sources and sinks
The way to do it is to use the ``create()`` factory method on :class:`FlowGraph`. If we remove the sources and sinks
from the previous example, what remains is a partial graph:
|
@ -284,7 +284,7 @@ Attributes
----------
We have seen that we can use ``named()`` to introduce a nesting level in the fluid DSL (and also explicit nesting by using
``partial()`` from :class:`FlowGraph`). Apart from having the effect of adding a nesting level, ``named()`` is actually
``create()`` from :class:`FlowGraph`). Apart from having the effect of adding a nesting level, ``named()`` is actually
a shorthand for calling ``withAttributes(Attributes.name("someName"))``. Attributes provide a way to fine-tune certain
aspects of the materialized running entity. For example buffer sizes can be controlled via attributes (see
:ref:`stream-buffers-scala`). When it comes to hierarchic composition, attributes are inherited by nested modules,

View file

@ -60,9 +60,6 @@ will be inferred.
Notice the ``import FlowGraph.Implicits._`` which brings into scope the ``~>`` operator (read as "edge", "via" or "to")
and its inverted counterpart ``<~`` (for noting down flows in the opposite direction where appropriate).
It is also possible to construct graphs without the ``~>`` operator in case you prefer to use the graph builder explicitly:
.. includecode:: code/docs/stream/FlowGraphDocSpec.scala#simple-flow-graph-no-implicits
By looking at the snippets above, it should be apparent that the :class:`FlowGraph.Builder` object is *mutable*.
It is used (implicitly) by the ``~>`` operator, also making it a mutable operation as well.