+doc: Updating basics, quickstart and some of graph docs
This commit is contained in:
parent
ac9c61a3a5
commit
41f6a0bf26
6 changed files with 285 additions and 42 deletions
|
|
@ -7,7 +7,7 @@ import akka.actor.Cancellable
|
|||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
|
||||
import concurrent.Future
|
||||
import scala.concurrent.{ Promise, Future }
|
||||
|
||||
class FlowDocSpec extends AkkaSpec {
|
||||
|
||||
|
|
@ -54,7 +54,7 @@ class FlowDocSpec extends AkkaSpec {
|
|||
//#materialization-runWith
|
||||
}
|
||||
|
||||
"materializedMap is unique" in {
|
||||
"materialization is unique" in {
|
||||
//#stream-reuse
|
||||
// connect the Source to the Sink, obtaining a RunnableFlow
|
||||
val sink = Sink.fold[Int, Int](0)(_ + _)
|
||||
|
|
@ -108,10 +108,10 @@ class FlowDocSpec extends AkkaSpec {
|
|||
Source.empty
|
||||
|
||||
// Sink that folds over the stream and returns a Future
|
||||
// of the final result in the MaterializedMap
|
||||
// of the final result as its materialized value
|
||||
Sink.fold[Int, Int](0)(_ + _)
|
||||
|
||||
// Sink that returns a Future in the MaterializedMap,
|
||||
// Sink that returns a Future as its materialized value,
|
||||
// containing the first element of the stream
|
||||
Sink.head
|
||||
|
||||
|
|
@ -138,4 +138,79 @@ class FlowDocSpec extends AkkaSpec {
|
|||
|
||||
//#flow-connecting
|
||||
}
|
||||
|
||||
"various ways of transforming materialized values" in {
|
||||
import scala.concurrent.duration._
|
||||
|
||||
val throttler = Flow(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)
|
||||
}
|
||||
|
||||
//#flow-mat-combine
|
||||
// An empty source that can be shut down explicitly from the outside
|
||||
val source: Source[Int, Promise[Unit]] = Source.lazyEmpty[Int]()
|
||||
|
||||
// A flow that internally throttles elements to 1/second, and returns a Cancellable
|
||||
// which can be used to shut down the stream
|
||||
val flow: Flow[Int, Int, Cancellable] = throttler
|
||||
|
||||
// A sink that returns the first element of a stream in the returned Future
|
||||
val sink: Sink[Int, Future[Int]] = Sink.head[Int]()
|
||||
|
||||
// By default, the materialized value of the leftmost stage is preserved
|
||||
val r1: RunnableFlow[Promise[Unit]] = source.via(flow).to(sink)
|
||||
|
||||
// Simple selection of materialized values by using Keep.right
|
||||
val r2: RunnableFlow[Cancellable] = source.viaMat(flow)(Keep.right).to(sink)
|
||||
val r3: RunnableFlow[Future[Int]] = source.via(flow).toMat(sink)(Keep.right)
|
||||
|
||||
// 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)
|
||||
|
||||
// Using more complext combinations
|
||||
val r7: RunnableFlow[(Promise[Unit], Cancellable)] =
|
||||
source.viaMat(flow)(Keep.both).to(sink)
|
||||
|
||||
val r8: RunnableFlow[(Promise[Unit], Future[Int])] =
|
||||
source.via(flow).toMat(sink)(Keep.both)
|
||||
|
||||
val r9: RunnableFlow[((Promise[Unit], Cancellable), Future[Int])] =
|
||||
source.viaMat(flow)(Keep.both).toMat(sink)(Keep.both)
|
||||
|
||||
val r10: RunnableFlow[(Cancellable, Future[Int])] =
|
||||
source.viaMat(flow)(Keep.right).toMat(sink)(Keep.both)
|
||||
|
||||
// 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: RunnableFlow[(Promise[Unit], Cancellable, Future[Int])] =
|
||||
r9.mapMaterialized {
|
||||
case ((promise, cancellable), future) =>
|
||||
(promise, cancellable, future)
|
||||
}
|
||||
|
||||
// Now we can use pattern matching to get the resulting materialized values
|
||||
val (promise, cancellable, future) = r11.run()
|
||||
|
||||
// Type inference works as expected
|
||||
promise.success(0)
|
||||
cancellable.cancel()
|
||||
future.map(_ + 3)
|
||||
|
||||
// The result of r11 can be also achieved by using the Graph API
|
||||
val r12: RunnableFlow[(Promise[Unit], Cancellable, Future[Int])] =
|
||||
FlowGraph.closed(source, flow, sink)((_, _, _)) { implicit builder =>
|
||||
(src, f, dst) =>
|
||||
import FlowGraph.Implicits._
|
||||
src ~> f ~> dst
|
||||
}
|
||||
|
||||
//#flow-mat-combine
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -3,16 +3,11 @@
|
|||
*/
|
||||
package docs.stream
|
||||
|
||||
import akka.stream.ActorFlowMaterializer
|
||||
import akka.stream.scaladsl.Broadcast
|
||||
import akka.stream.scaladsl.Flow
|
||||
import akka.stream.scaladsl.FlowGraph
|
||||
import akka.stream.scaladsl.Merge
|
||||
import akka.stream.scaladsl.Sink
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.scaladsl.Zip
|
||||
import akka.stream._
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
|
|
@ -117,4 +112,112 @@ class FlowGraphDocSpec extends AkkaSpec {
|
|||
Await.result(bottomFuture, 300.millis) shouldEqual 2
|
||||
}
|
||||
|
||||
"building a reusable component" in {
|
||||
|
||||
//#flow-graph-components-shape
|
||||
// A shape represents the input and output ports of a reusable
|
||||
// processing module
|
||||
case class PriorityWorkerPoolShape[In, Out](
|
||||
jobsIn: Inlet[In],
|
||||
priorityJobsIn: Inlet[In],
|
||||
resultsOut: Outlet[Out]) extends Shape {
|
||||
|
||||
// It is important to provide the list of all input and output
|
||||
// ports with a stable order. Duplicates are not allowed.
|
||||
override val inlets: immutable.Seq[Inlet[_]] =
|
||||
jobsIn :: priorityJobsIn :: Nil
|
||||
override val outlets: immutable.Seq[Outlet[_]] =
|
||||
resultsOut :: Nil
|
||||
|
||||
// A Shape must be able to create a copy of itself. Basically
|
||||
// it means a new instance with copies of the ports
|
||||
override def deepCopy() = PriorityWorkerPoolShape(
|
||||
new Inlet[In](jobsIn.toString),
|
||||
new Inlet[In](priorityJobsIn.toString),
|
||||
new Outlet[Out](resultsOut.toString))
|
||||
|
||||
// A Shape must also be able to create itself from existing ports
|
||||
override def copyFromPorts(
|
||||
inlets: immutable.Seq[Inlet[_]],
|
||||
outlets: immutable.Seq[Outlet[_]]) = {
|
||||
assert(inlets.size == this.inlets.size)
|
||||
assert(outlets.size == this.outlets.size)
|
||||
// This is why order matters when overriding inlets and outlets
|
||||
PriorityWorkerPoolShape(inlets(0), inlets(1), outlets(0))
|
||||
}
|
||||
}
|
||||
//#flow-graph-components-shape
|
||||
|
||||
//#flow-graph-components-create
|
||||
object PriorityWorkerPool {
|
||||
def apply[In, Out](
|
||||
worker: Flow[In, Out, _],
|
||||
workerCount: Int): Graph[PriorityWorkerPoolShape[In, Out], Unit] = {
|
||||
|
||||
FlowGraph.partial() { implicit b ⇒
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val priorityMerge = b.add(MergePreferred[In](1))
|
||||
val balance = b.add(Balance[In](workerCount))
|
||||
val resultsMerge = b.add(Merge[Out](workerCount))
|
||||
|
||||
// After merging priority and ordinary jobs, we feed them to the balancer
|
||||
priorityMerge ~> balance
|
||||
|
||||
// Wire up each of the outputs of the balancer to a worker flow
|
||||
// then merge them back
|
||||
for (i <- 0 until workerCount)
|
||||
balance.out(i) ~> worker ~> resultsMerge.in(i)
|
||||
|
||||
// We now expose the input ports of the priorityMerge and the output
|
||||
// of the resultsMerge as our PriorityWorkerPool ports
|
||||
// -- all neatly wrapped in our domain specific Shape
|
||||
PriorityWorkerPoolShape(
|
||||
jobsIn = priorityMerge.in(0),
|
||||
priorityJobsIn = priorityMerge.preferred,
|
||||
resultsOut = resultsMerge.out)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
//#flow-graph-components-create
|
||||
|
||||
def println(s: Any): Unit = ()
|
||||
|
||||
//#flow-graph-components-use
|
||||
val worker1 = Flow[String].map("step 1 " + _)
|
||||
val worker2 = Flow[String].map("step 2 " + _)
|
||||
|
||||
FlowGraph.closed() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
val priorityPool1 = b.add(PriorityWorkerPool(worker1, 4))
|
||||
val priorityPool2 = b.add(PriorityWorkerPool(worker2, 2))
|
||||
|
||||
Source(1 to 100).map("job: " + _) ~> priorityPool1.jobsIn
|
||||
Source(1 to 100).map("priority job: " + _) ~> priorityPool1.priorityJobsIn
|
||||
|
||||
priorityPool1.resultsOut ~> priorityPool2.jobsIn
|
||||
Source(1 to 100).map("one-step, priority " + _) ~> priorityPool2.priorityJobsIn
|
||||
|
||||
priorityPool2.resultsOut ~> Sink.foreach(println)
|
||||
}.run()
|
||||
//#flow-graph-components-use
|
||||
|
||||
//#flow-graph-components-shape2
|
||||
import FanInShape.Name
|
||||
import FanInShape.Init
|
||||
|
||||
case class PriorityWorkerPoolShape2[In, Out](
|
||||
_init: Init[Out] = Name("PriorityWorkerPool")) extends FanInShape2[In, In, Out](_init) {
|
||||
|
||||
def jobsIn: Inlet[In] = in0
|
||||
def priorityJobsIn: Inlet[In] = in1
|
||||
def resultsOut: Outlet[Out] = out
|
||||
}
|
||||
//#flow-graph-components-shape2
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -62,7 +62,7 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
|
|||
trait Example1 {
|
||||
//#materializer-setup
|
||||
implicit val system = ActorSystem("reactive-tweets")
|
||||
implicit val mat = ActorFlowMaterializer()
|
||||
implicit val materializer = ActorFlowMaterializer()
|
||||
//#materializer-setup
|
||||
}
|
||||
|
||||
|
|
@ -155,7 +155,7 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
|
|||
|
||||
"count elements on finite stream" in {
|
||||
//#tweets-fold-count
|
||||
val sumSink = Sink.fold[Int, Int](0)(_ + _)
|
||||
val sumSink: Sink[Int, Future[Int]] = Sink.fold[Int, Int](0)(_ + _)
|
||||
|
||||
val counter: RunnableFlow[Future[Int]] = tweets.map(t => 1).toMat(sumSink)(Keep.right)
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue