parent
63ccdeec16
commit
b1df13d4d4
221 changed files with 1528 additions and 1580 deletions
|
|
@ -29,9 +29,9 @@ object ActorPublisherDocSpec {
|
|||
var buf = Vector.empty[Job]
|
||||
|
||||
def receive = {
|
||||
case job: Job if buf.size == MaxBufferSize =>
|
||||
case job: Job if buf.size == MaxBufferSize ⇒
|
||||
sender() ! JobDenied
|
||||
case job: Job =>
|
||||
case job: Job ⇒
|
||||
sender() ! JobAccepted
|
||||
if (buf.isEmpty && totalDemand > 0)
|
||||
onNext(job)
|
||||
|
|
@ -39,9 +39,9 @@ object ActorPublisherDocSpec {
|
|||
buf :+= job
|
||||
deliverBuf()
|
||||
}
|
||||
case Request(_) =>
|
||||
case Request(_) ⇒
|
||||
deliverBuf()
|
||||
case Cancel =>
|
||||
case Cancel ⇒
|
||||
context.stop(self)
|
||||
}
|
||||
|
||||
|
|
@ -79,7 +79,7 @@ class ActorPublisherDocSpec extends AkkaSpec {
|
|||
val jobManagerSource = Source.actorPublisher[JobManager.Job](JobManager.props)
|
||||
val ref = Flow[JobManager.Job]
|
||||
.map(_.payload.toUpperCase)
|
||||
.map { elem => println(elem); elem }
|
||||
.map { elem ⇒ println(elem); elem }
|
||||
.to(Sink.ignore)
|
||||
.runWith(jobManagerSource)
|
||||
|
||||
|
|
|
|||
|
|
@ -48,17 +48,17 @@ object ActorSubscriberDocSpec {
|
|||
}
|
||||
|
||||
def receive = {
|
||||
case OnNext(Msg(id, replyTo)) =>
|
||||
case OnNext(Msg(id, replyTo)) ⇒
|
||||
queue += (id -> replyTo)
|
||||
assert(queue.size <= MaxQueueSize, s"queued too many: ${queue.size}")
|
||||
router.route(Work(id), self)
|
||||
case Reply(id) =>
|
||||
case Reply(id) ⇒
|
||||
queue(id) ! Done(id)
|
||||
queue -= id
|
||||
if (canceled && queue.isEmpty) {
|
||||
context.stop(self)
|
||||
}
|
||||
case OnComplete =>
|
||||
case OnComplete ⇒
|
||||
if (queue.isEmpty) {
|
||||
context.stop(self)
|
||||
}
|
||||
|
|
@ -68,7 +68,7 @@ object ActorSubscriberDocSpec {
|
|||
class Worker extends Actor {
|
||||
import WorkerPool._
|
||||
def receive = {
|
||||
case Work(id) =>
|
||||
case Work(id) ⇒
|
||||
// ...
|
||||
sender() ! Reply(id)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -25,8 +25,8 @@ object BidiFlowDocSpec {
|
|||
//#implementation-details-elided
|
||||
implicit val order = ByteOrder.LITTLE_ENDIAN
|
||||
msg match {
|
||||
case Ping(id) => ByteString.newBuilder.putByte(1).putInt(id).result()
|
||||
case Pong(id) => ByteString.newBuilder.putByte(2).putInt(id).result()
|
||||
case Ping(id) ⇒ ByteString.newBuilder.putByte(1).putInt(id).result()
|
||||
case Pong(id) ⇒ ByteString.newBuilder.putByte(2).putInt(id).result()
|
||||
}
|
||||
//#implementation-details-elided
|
||||
}
|
||||
|
|
@ -36,15 +36,15 @@ object BidiFlowDocSpec {
|
|||
implicit val order = ByteOrder.LITTLE_ENDIAN
|
||||
val it = bytes.iterator
|
||||
it.getByte match {
|
||||
case 1 => Ping(it.getInt)
|
||||
case 2 => Pong(it.getInt)
|
||||
case other => throw new RuntimeException(s"parse error: expected 1|2 got $other")
|
||||
case 1 ⇒ Ping(it.getInt)
|
||||
case 2 ⇒ Pong(it.getInt)
|
||||
case other ⇒ throw new RuntimeException(s"parse error: expected 1|2 got $other")
|
||||
}
|
||||
//#implementation-details-elided
|
||||
}
|
||||
//#codec-impl
|
||||
|
||||
val codecVerbose = BidiFlow.fromGraph(GraphDSL.create() { b =>
|
||||
val codecVerbose = BidiFlow.fromGraph(GraphDSL.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
|
||||
|
|
@ -58,7 +58,7 @@ object BidiFlowDocSpec {
|
|||
//#codec
|
||||
|
||||
//#framing
|
||||
val framing = BidiFlow.fromGraph(GraphDSL.create() { b =>
|
||||
val framing = BidiFlow.fromGraph(GraphDSL.create() { b ⇒
|
||||
implicit val order = ByteOrder.LITTLE_ENDIAN
|
||||
|
||||
def addLengthHeader(bytes: ByteString) = {
|
||||
|
|
@ -135,12 +135,12 @@ object BidiFlowDocSpec {
|
|||
})
|
||||
//#framing
|
||||
|
||||
val chopUp = BidiFlow.fromGraph(GraphDSL.create() { b =>
|
||||
val chopUp = BidiFlow.fromGraph(GraphDSL.create() { b ⇒
|
||||
val f = Flow[ByteString].mapConcat(_.map(ByteString(_)))
|
||||
BidiShape.fromFlows(b.add(f), b.add(f))
|
||||
})
|
||||
|
||||
val accumulate = BidiFlow.fromGraph(GraphDSL.create() { b =>
|
||||
val accumulate = BidiFlow.fromGraph(GraphDSL.create() { b ⇒
|
||||
val f = Flow[ByteString].grouped(1000).map(_.fold(ByteString.empty)(_ ++ _))
|
||||
BidiShape.fromFlows(b.add(f), b.add(f))
|
||||
})
|
||||
|
|
@ -169,7 +169,7 @@ class BidiFlowDocSpec extends AkkaSpec {
|
|||
val stack = codec.atop(framing)
|
||||
|
||||
// test it by plugging it into its own inverse and closing the right end
|
||||
val pingpong = Flow[Message].collect { case Ping(id) => Pong(id) }
|
||||
val pingpong = Flow[Message].collect { case Ping(id) ⇒ Pong(id) }
|
||||
val flow = stack.atop(stack.reversed).join(pingpong)
|
||||
val result = Source((0 to 9).map(Ping)).via(flow).limit(20).runWith(Sink.seq)
|
||||
Await.result(result, 1.second) should ===((0 to 9).map(Pong))
|
||||
|
|
@ -178,14 +178,14 @@ class BidiFlowDocSpec extends AkkaSpec {
|
|||
|
||||
"work when chopped up" in {
|
||||
val stack = codec.atop(framing)
|
||||
val flow = stack.atop(chopUp).atop(stack.reversed).join(Flow[Message].map { case Ping(id) => Pong(id) })
|
||||
val flow = stack.atop(chopUp).atop(stack.reversed).join(Flow[Message].map { case Ping(id) ⇒ Pong(id) })
|
||||
val f = Source((0 to 9).map(Ping)).via(flow).limit(20).runWith(Sink.seq)
|
||||
Await.result(f, 1.second) should ===((0 to 9).map(Pong))
|
||||
}
|
||||
|
||||
"work when accumulated" in {
|
||||
val stack = codec.atop(framing)
|
||||
val flow = stack.atop(accumulate).atop(stack.reversed).join(Flow[Message].map { case Ping(id) => Pong(id) })
|
||||
val flow = stack.atop(accumulate).atop(stack.reversed).join(Flow[Message].map { case Ping(id) ⇒ Pong(id) })
|
||||
val f = Source((0 to 9).map(Ping)).via(flow).limit(20).runWith(Sink.seq)
|
||||
Await.result(f, 1.second) should ===((0 to 9).map(Pong))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -168,7 +168,7 @@ class CompositionDocSpec extends AkkaSpec {
|
|||
"closed graph" in {
|
||||
//#embed-closed
|
||||
val closed1 = Source.single(0).to(Sink.foreach(println))
|
||||
val closed2 = RunnableGraph.fromGraph(GraphDSL.create() { implicit builder =>
|
||||
val closed2 = RunnableGraph.fromGraph(GraphDSL.create() { implicit builder ⇒
|
||||
val embeddedClosed: ClosedShape = builder.add(closed1)
|
||||
// …
|
||||
embeddedClosed
|
||||
|
|
@ -191,7 +191,7 @@ class CompositionDocSpec extends AkkaSpec {
|
|||
|
||||
//#mat-combine-2
|
||||
// Materializes to NotUsed (orange)
|
||||
val flow2: Flow[Int, ByteString, NotUsed] = Flow[Int].map { i => ByteString(i.toString) }
|
||||
val flow2: Flow[Int, ByteString, NotUsed] = Flow[Int].map { i ⇒ ByteString(i.toString) }
|
||||
|
||||
// Materializes to Future[OutgoingConnection] (yellow)
|
||||
val flow3: Flow[ByteString, ByteString, Future[OutgoingConnection]] =
|
||||
|
|
|
|||
|
|
@ -24,10 +24,10 @@ class FlowDocSpec extends AkkaSpec {
|
|||
"source is immutable" in {
|
||||
//#source-immutable
|
||||
val source = Source(1 to 10)
|
||||
source.map(_ => 0) // has no effect on source, since it's immutable
|
||||
source.map(_ ⇒ 0) // has no effect on source, since it's immutable
|
||||
source.runWith(Sink.fold(0)(_ + _)) // 55
|
||||
|
||||
val zeroes = source.map(_ => 0) // returns new Source[Int], with `map()` appended
|
||||
val zeroes = source.map(_ ⇒ 0) // returns new Source[Int], with `map()` appended
|
||||
zeroes.runWith(Sink.fold(0)(_ + _)) // 0
|
||||
//#source-immutable
|
||||
}
|
||||
|
|
@ -78,12 +78,12 @@ class FlowDocSpec extends AkkaSpec {
|
|||
import scala.concurrent.duration._
|
||||
case object Tick
|
||||
|
||||
val timer = Source.tick(initialDelay = 1.second, interval = 1.seconds, tick = () => Tick)
|
||||
val timer = Source.tick(initialDelay = 1.second, interval = 1.seconds, tick = () ⇒ Tick)
|
||||
|
||||
val timerCancel: Cancellable = Sink.ignore.runWith(timer)
|
||||
timerCancel.cancel()
|
||||
|
||||
val timerMap = timer.map(tick => "tick")
|
||||
val timerMap = timer.map(tick ⇒ "tick")
|
||||
// materialize the flow and retrieve the timers Cancellable
|
||||
val timerCancellable = Sink.ignore.runWith(timerMap)
|
||||
timerCancellable.cancel()
|
||||
|
|
@ -149,7 +149,7 @@ class FlowDocSpec extends AkkaSpec {
|
|||
"various ways of transforming materialized values" in {
|
||||
import scala.concurrent.duration._
|
||||
|
||||
val throttler = Flow.fromGraph(GraphDSL.create(Source.tick(1.second, 1.second, "test")) { implicit builder => tickSource =>
|
||||
val throttler = Flow.fromGraph(GraphDSL.create(Source.tick(1.second, 1.second, "test")) { implicit builder ⇒ tickSource ⇒
|
||||
import GraphDSL.Implicits._
|
||||
val zip = builder.add(ZipWith[String, Int, Int](Keep.right))
|
||||
tickSource ~> zip.in0
|
||||
|
|
@ -197,7 +197,7 @@ class FlowDocSpec extends AkkaSpec {
|
|||
// doubly nested pair, but we want to flatten it out
|
||||
val r11: RunnableGraph[(Promise[Option[Int]], Cancellable, Future[Int])] =
|
||||
r9.mapMaterializedValue {
|
||||
case ((promise, cancellable), future) =>
|
||||
case ((promise, cancellable), future) ⇒
|
||||
(promise, cancellable, future)
|
||||
}
|
||||
|
||||
|
|
@ -211,7 +211,7 @@ class FlowDocSpec extends AkkaSpec {
|
|||
|
||||
// The result of r11 can be also achieved by using the Graph API
|
||||
val r12: RunnableGraph[(Promise[Option[Int]], Cancellable, Future[Int])] =
|
||||
RunnableGraph.fromGraph(GraphDSL.create(source, flow, sink)((_, _, _)) { implicit builder => (src, f, dst) =>
|
||||
RunnableGraph.fromGraph(GraphDSL.create(source, flow, sink)((_, _, _)) { implicit builder ⇒ (src, f, dst) ⇒
|
||||
import GraphDSL.Implicits._
|
||||
src ~> f ~> dst
|
||||
ClosedShape
|
||||
|
|
|
|||
|
|
@ -32,8 +32,8 @@ class FlowErrorDocSpec extends AkkaSpec {
|
|||
"demonstrate resume stream" in {
|
||||
//#resume
|
||||
val decider: Supervision.Decider = {
|
||||
case _: ArithmeticException => Supervision.Resume
|
||||
case _ => Supervision.Stop
|
||||
case _: ArithmeticException ⇒ Supervision.Resume
|
||||
case _ ⇒ Supervision.Stop
|
||||
}
|
||||
implicit val materializer = ActorMaterializer(
|
||||
ActorMaterializerSettings(system).withSupervisionStrategy(decider))
|
||||
|
|
@ -50,11 +50,11 @@ class FlowErrorDocSpec extends AkkaSpec {
|
|||
//#resume-section
|
||||
implicit val materializer = ActorMaterializer()
|
||||
val decider: Supervision.Decider = {
|
||||
case _: ArithmeticException => Supervision.Resume
|
||||
case _ => Supervision.Stop
|
||||
case _: ArithmeticException ⇒ Supervision.Resume
|
||||
case _ ⇒ Supervision.Stop
|
||||
}
|
||||
val flow = Flow[Int]
|
||||
.filter(100 / _ < 50).map(elem => 100 / (5 - elem))
|
||||
.filter(100 / _ < 50).map(elem ⇒ 100 / (5 - elem))
|
||||
.withAttributes(ActorAttributes.supervisionStrategy(decider))
|
||||
val source = Source(0 to 5).via(flow)
|
||||
|
||||
|
|
@ -70,11 +70,11 @@ class FlowErrorDocSpec extends AkkaSpec {
|
|||
//#restart-section
|
||||
implicit val materializer = ActorMaterializer()
|
||||
val decider: Supervision.Decider = {
|
||||
case _: IllegalArgumentException => Supervision.Restart
|
||||
case _ => Supervision.Stop
|
||||
case _: IllegalArgumentException ⇒ Supervision.Restart
|
||||
case _ ⇒ Supervision.Stop
|
||||
}
|
||||
val flow = Flow[Int]
|
||||
.scan(0) { (acc, elem) =>
|
||||
.scan(0) { (acc, elem) ⇒
|
||||
if (elem < 0) throw new IllegalArgumentException("negative not allowed")
|
||||
else acc + elem
|
||||
}
|
||||
|
|
@ -92,11 +92,11 @@ class FlowErrorDocSpec extends AkkaSpec {
|
|||
"demonstrate recover" in {
|
||||
implicit val materializer = ActorMaterializer()
|
||||
//#recover
|
||||
Source(0 to 6).map(n =>
|
||||
Source(0 to 6).map(n ⇒
|
||||
if (n < 5) n.toString
|
||||
else throw new RuntimeException("Boom!")
|
||||
).recover {
|
||||
case _: RuntimeException => "stream truncated"
|
||||
case _: RuntimeException ⇒ "stream truncated"
|
||||
}.runForeach(println)
|
||||
//#recover
|
||||
|
||||
|
|
@ -118,11 +118,11 @@ stream truncated
|
|||
//#recoverWithRetries
|
||||
val planB = Source(List("five", "six", "seven", "eight"))
|
||||
|
||||
Source(0 to 10).map(n =>
|
||||
Source(0 to 10).map(n ⇒
|
||||
if (n < 5) n.toString
|
||||
else throw new RuntimeException("Boom!")
|
||||
).recoverWithRetries(attempts = 1, {
|
||||
case _: RuntimeException => planB
|
||||
case _: RuntimeException ⇒ planB
|
||||
}).runForeach(println)
|
||||
//#recoverWithRetries
|
||||
|
||||
|
|
|
|||
|
|
@ -37,9 +37,9 @@ class FlowParallelismDocSpec extends AkkaSpec {
|
|||
"Demonstrate parallel processing" in {
|
||||
//#parallelism
|
||||
val fryingPan: Flow[ScoopOfBatter, Pancake, NotUsed] =
|
||||
Flow[ScoopOfBatter].map { batter => Pancake() }
|
||||
Flow[ScoopOfBatter].map { batter ⇒ Pancake() }
|
||||
|
||||
val pancakeChef: Flow[ScoopOfBatter, Pancake, NotUsed] = Flow.fromGraph(GraphDSL.create() { implicit builder =>
|
||||
val pancakeChef: Flow[ScoopOfBatter, Pancake, NotUsed] = Flow.fromGraph(GraphDSL.create() { implicit builder ⇒
|
||||
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
|
||||
val mergePancakes = builder.add(Merge[Pancake](2))
|
||||
|
||||
|
|
@ -60,7 +60,7 @@ class FlowParallelismDocSpec extends AkkaSpec {
|
|||
"Demonstrate parallelized pipelines" in {
|
||||
//#parallel-pipeline
|
||||
val pancakeChef: Flow[ScoopOfBatter, Pancake, NotUsed] =
|
||||
Flow.fromGraph(GraphDSL.create() { implicit builder =>
|
||||
Flow.fromGraph(GraphDSL.create() { implicit builder ⇒
|
||||
|
||||
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
|
||||
val mergePancakes = builder.add(Merge[Pancake](2))
|
||||
|
|
@ -78,7 +78,7 @@ class FlowParallelismDocSpec extends AkkaSpec {
|
|||
"Demonstrate pipelined parallel processing" in {
|
||||
//#pipelined-parallel
|
||||
val pancakeChefs1: Flow[ScoopOfBatter, HalfCookedPancake, NotUsed] =
|
||||
Flow.fromGraph(GraphDSL.create() { implicit builder =>
|
||||
Flow.fromGraph(GraphDSL.create() { implicit builder ⇒
|
||||
val dispatchBatter = builder.add(Balance[ScoopOfBatter](2))
|
||||
val mergeHalfPancakes = builder.add(Merge[HalfCookedPancake](2))
|
||||
|
||||
|
|
@ -91,7 +91,7 @@ class FlowParallelismDocSpec extends AkkaSpec {
|
|||
})
|
||||
|
||||
val pancakeChefs2: Flow[HalfCookedPancake, Pancake, NotUsed] =
|
||||
Flow.fromGraph(GraphDSL.create() { implicit builder =>
|
||||
Flow.fromGraph(GraphDSL.create() { implicit builder ⇒
|
||||
val dispatchHalfPancakes = builder.add(Balance[HalfCookedPancake](2))
|
||||
val mergePancakes = builder.add(Merge[Pancake](2))
|
||||
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ class GraphCyclesSpec extends AkkaSpec {
|
|||
implicit val materializer = ActorMaterializer()
|
||||
|
||||
"Cycle demonstration" must {
|
||||
val source = Source.fromIterator(() => Iterator.from(0))
|
||||
val source = Source.fromIterator(() ⇒ Iterator.from(0))
|
||||
|
||||
"include a deadlocked cycle" in {
|
||||
|
||||
|
|
|
|||
|
|
@ -46,7 +46,7 @@ class GraphDSLDocSpec extends AkkaSpec {
|
|||
"flow connection errors" in {
|
||||
intercept[IllegalStateException] {
|
||||
//#simple-graph
|
||||
RunnableGraph.fromGraph(GraphDSL.create() { implicit builder =>
|
||||
RunnableGraph.fromGraph(GraphDSL.create() { implicit builder ⇒
|
||||
import GraphDSL.Implicits._
|
||||
val source1 = Source(1 to 10)
|
||||
val source2 = Source(1 to 10)
|
||||
|
|
@ -124,7 +124,7 @@ class GraphDSLDocSpec extends AkkaSpec {
|
|||
worker: Flow[In, Out, Any],
|
||||
workerCount: Int): Graph[PriorityWorkerPoolShape[In, Out], NotUsed] = {
|
||||
|
||||
GraphDSL.create() { implicit b =>
|
||||
GraphDSL.create() { implicit b ⇒
|
||||
import GraphDSL.Implicits._
|
||||
|
||||
val priorityMerge = b.add(MergePreferred[In](1))
|
||||
|
|
@ -136,7 +136,7 @@ class GraphDSLDocSpec extends AkkaSpec {
|
|||
|
||||
// Wire up each of the outputs of the balancer to a worker flow
|
||||
// then merge them back
|
||||
for (i <- 0 until workerCount)
|
||||
for (i ← 0 until workerCount)
|
||||
balance.out(i) ~> worker ~> resultsMerge.in(i)
|
||||
|
||||
// We now expose the input ports of the priorityMerge and the output
|
||||
|
|
@ -159,7 +159,7 @@ class GraphDSLDocSpec extends AkkaSpec {
|
|||
val worker1 = Flow[String].map("step 1 " + _)
|
||||
val worker2 = Flow[String].map("step 2 " + _)
|
||||
|
||||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b =>
|
||||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
import GraphDSL.Implicits._
|
||||
|
||||
val priorityPool1 = b.add(PriorityWorkerPool(worker1, 4))
|
||||
|
|
@ -194,7 +194,7 @@ class GraphDSLDocSpec extends AkkaSpec {
|
|||
"access to materialized value" in {
|
||||
//#graph-dsl-matvalue
|
||||
import GraphDSL.Implicits._
|
||||
val foldFlow: Flow[Int, Int, Future[Int]] = Flow.fromGraph(GraphDSL.create(Sink.fold[Int, Int](0)(_ + _)) { implicit builder => fold =>
|
||||
val foldFlow: Flow[Int, Int, Future[Int]] = Flow.fromGraph(GraphDSL.create(Sink.fold[Int, Int](0)(_ + _)) { implicit builder ⇒ fold ⇒
|
||||
FlowShape(fold.in, builder.materializedValue.mapAsync(4)(identity).outlet)
|
||||
})
|
||||
//#graph-dsl-matvalue
|
||||
|
|
@ -204,7 +204,7 @@ class GraphDSLDocSpec extends AkkaSpec {
|
|||
//#graph-dsl-matvalue-cycle
|
||||
import GraphDSL.Implicits._
|
||||
// This cannot produce any value:
|
||||
val cyclicFold: Source[Int, Future[Int]] = Source.fromGraph(GraphDSL.create(Sink.fold[Int, Int](0)(_ + _)) { implicit builder => fold =>
|
||||
val cyclicFold: Source[Int, Future[Int]] = Source.fromGraph(GraphDSL.create(Sink.fold[Int, Int](0)(_ + _)) { implicit builder ⇒ fold ⇒
|
||||
// - Fold cannot complete until its upstream mapAsync completes
|
||||
// - mapAsync cannot complete until the materialized Future produced by
|
||||
// fold completes
|
||||
|
|
|
|||
|
|
@ -115,7 +115,7 @@ class GraphStageDocSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
//#one-to-one
|
||||
class Map[A, B](f: A => B) extends GraphStage[FlowShape[A, B]] {
|
||||
class Map[A, B](f: A ⇒ B) extends GraphStage[FlowShape[A, B]] {
|
||||
|
||||
val in = Inlet[A]("Map.in")
|
||||
val out = Outlet[B]("Map.out")
|
||||
|
|
@ -145,13 +145,13 @@ class GraphStageDocSpec extends AkkaSpec {
|
|||
val result =
|
||||
Source(Vector("one", "two", "three"))
|
||||
.via(stringLength)
|
||||
.runFold(Seq.empty[Int])((elem, acc) => elem :+ acc)
|
||||
.runFold(Seq.empty[Int])((elem, acc) ⇒ elem :+ acc)
|
||||
|
||||
Await.result(result, 3.seconds) should ===(Seq(3, 3, 5))
|
||||
}
|
||||
|
||||
//#many-to-one
|
||||
class Filter[A](p: A => Boolean) extends GraphStage[FlowShape[A, A]] {
|
||||
class Filter[A](p: A ⇒ Boolean) extends GraphStage[FlowShape[A, A]] {
|
||||
|
||||
val in = Inlet[A]("Filter.in")
|
||||
val out = Outlet[A]("Filter.out")
|
||||
|
|
@ -184,7 +184,7 @@ class GraphStageDocSpec extends AkkaSpec {
|
|||
val result =
|
||||
Source(Vector(1, 2, 3, 4, 5, 6))
|
||||
.via(evenFilter)
|
||||
.runFold(Seq.empty[Int])((elem, acc) => elem :+ acc)
|
||||
.runFold(Seq.empty[Int])((elem, acc) ⇒ elem :+ acc)
|
||||
|
||||
Await.result(result, 3.seconds) should ===(Seq(2, 4, 6))
|
||||
}
|
||||
|
|
@ -237,7 +237,7 @@ class GraphStageDocSpec extends AkkaSpec {
|
|||
val result =
|
||||
Source(Vector(1, 2, 3))
|
||||
.via(duplicator)
|
||||
.runFold(Seq.empty[Int])((elem, acc) => elem :+ acc)
|
||||
.runFold(Seq.empty[Int])((elem, acc) ⇒ elem :+ acc)
|
||||
|
||||
Await.result(result, 3.seconds) should ===(Seq(1, 1, 2, 2, 3, 3))
|
||||
}
|
||||
|
|
@ -277,14 +277,14 @@ class GraphStageDocSpec extends AkkaSpec {
|
|||
val result =
|
||||
Source(Vector(1, 2, 3))
|
||||
.via(duplicator)
|
||||
.runFold(Seq.empty[Int])((elem, acc) => elem :+ acc)
|
||||
.runFold(Seq.empty[Int])((elem, acc) ⇒ elem :+ acc)
|
||||
|
||||
Await.result(result, 3.seconds) should ===(Seq(1, 1, 2, 2, 3, 3))
|
||||
|
||||
}
|
||||
|
||||
"Demonstrate chaining of graph stages" in {
|
||||
val sink = Sink.fold[List[Int], Int](List.empty[Int])((acc, n) => acc :+ n)
|
||||
val sink = Sink.fold[List[Int], Int](List.empty[Int])((acc, n) ⇒ acc :+ n)
|
||||
|
||||
//#graph-stage-chain
|
||||
val resultFuture = Source(1 to 5)
|
||||
|
|
@ -314,7 +314,7 @@ class GraphStageDocSpec extends AkkaSpec {
|
|||
new GraphStageLogic(shape) {
|
||||
|
||||
override def preStart(): Unit = {
|
||||
val callback = getAsyncCallback[Unit] { (_) =>
|
||||
val callback = getAsyncCallback[Unit] { (_) ⇒
|
||||
completeStage()
|
||||
}
|
||||
switch.foreach(callback.invoke)
|
||||
|
|
@ -401,7 +401,7 @@ class GraphStageDocSpec extends AkkaSpec {
|
|||
Source(Vector(1, 2, 3))
|
||||
.via(new TimedGate[Int](2.second))
|
||||
.takeWithin(250.millis)
|
||||
.runFold(Seq.empty[Int])((elem, acc) => elem :+ acc)
|
||||
.runFold(Seq.empty[Int])((elem, acc) ⇒ elem :+ acc)
|
||||
|
||||
Await.result(result, 3.seconds) should ===(Seq(1))
|
||||
}
|
||||
|
|
@ -526,7 +526,7 @@ class GraphStageDocSpec extends AkkaSpec {
|
|||
// tests:
|
||||
val result1 = Source(Vector(1, 2, 3))
|
||||
.via(new TwoBuffer)
|
||||
.runFold(Vector.empty[Int])((acc, n) => acc :+ n)
|
||||
.runFold(Vector.empty[Int])((acc, n) ⇒ acc :+ n)
|
||||
|
||||
Await.result(result1, 3.seconds) should ===(Vector(1, 2, 3))
|
||||
|
||||
|
|
|
|||
|
|
@ -60,8 +60,8 @@ class HubsDocSpec extends AkkaSpec with CompileOnlySpec {
|
|||
val fromProducer: Source[String, NotUsed] = runnableGraph.run()
|
||||
|
||||
// Print out messages from the producer in two independent consumers
|
||||
fromProducer.runForeach(msg => println("consumer1: " + msg))
|
||||
fromProducer.runForeach(msg => println("consumer2: " + msg))
|
||||
fromProducer.runForeach(msg ⇒ println("consumer1: " + msg))
|
||||
fromProducer.runForeach(msg ⇒ println("consumer2: " + msg))
|
||||
//#broadcast-hub
|
||||
}
|
||||
|
||||
|
|
@ -109,7 +109,7 @@ class HubsDocSpec extends AkkaSpec with CompileOnlySpec {
|
|||
//#partition-hub
|
||||
// A simple producer that publishes a new "message-" every second
|
||||
val producer = Source.tick(1.second, 1.second, "message")
|
||||
.zipWith(Source(1 to 100))((a, b) => s"$a-$b")
|
||||
.zipWith(Source(1 to 100))((a, b) ⇒ s"$a-$b")
|
||||
|
||||
// Attach a PartitionHub Sink to the producer. This will materialize to a
|
||||
// corresponding Source.
|
||||
|
|
@ -117,7 +117,7 @@ class HubsDocSpec extends AkkaSpec with CompileOnlySpec {
|
|||
// value to the left is used)
|
||||
val runnableGraph: RunnableGraph[Source[String, NotUsed]] =
|
||||
producer.toMat(PartitionHub.sink(
|
||||
(size, elem) => math.abs(elem.hashCode) % size,
|
||||
(size, elem) ⇒ math.abs(elem.hashCode) % size,
|
||||
startAfterNrOfConsumers = 2, bufferSize = 256))(Keep.right)
|
||||
|
||||
// By running/materializing the producer, we get back a Source, which
|
||||
|
|
@ -125,8 +125,8 @@ class HubsDocSpec extends AkkaSpec with CompileOnlySpec {
|
|||
val fromProducer: Source[String, NotUsed] = runnableGraph.run()
|
||||
|
||||
// Print out messages from the producer in two independent consumers
|
||||
fromProducer.runForeach(msg => println("consumer1: " + msg))
|
||||
fromProducer.runForeach(msg => println("consumer2: " + msg))
|
||||
fromProducer.runForeach(msg ⇒ println("consumer1: " + msg))
|
||||
fromProducer.runForeach(msg ⇒ println("consumer2: " + msg))
|
||||
//#partition-hub
|
||||
}
|
||||
|
||||
|
|
@ -134,14 +134,14 @@ class HubsDocSpec extends AkkaSpec with CompileOnlySpec {
|
|||
//#partition-hub-stateful
|
||||
// A simple producer that publishes a new "message-" every second
|
||||
val producer = Source.tick(1.second, 1.second, "message")
|
||||
.zipWith(Source(1 to 100))((a, b) => s"$a-$b")
|
||||
.zipWith(Source(1 to 100))((a, b) ⇒ s"$a-$b")
|
||||
|
||||
// New instance of the partitioner function and its state is created
|
||||
// for each materialization of the PartitionHub.
|
||||
def roundRobin(): (PartitionHub.ConsumerInfo, String) ⇒ Long = {
|
||||
var i = -1L
|
||||
|
||||
(info, elem) => {
|
||||
(info, elem) ⇒ {
|
||||
i += 1
|
||||
info.consumerIdByIdx((i % info.size).toInt)
|
||||
}
|
||||
|
|
@ -153,7 +153,7 @@ class HubsDocSpec extends AkkaSpec with CompileOnlySpec {
|
|||
// value to the left is used)
|
||||
val runnableGraph: RunnableGraph[Source[String, NotUsed]] =
|
||||
producer.toMat(PartitionHub.statefulSink(
|
||||
() => roundRobin(),
|
||||
() ⇒ roundRobin(),
|
||||
startAfterNrOfConsumers = 2, bufferSize = 256))(Keep.right)
|
||||
|
||||
// By running/materializing the producer, we get back a Source, which
|
||||
|
|
@ -161,8 +161,8 @@ class HubsDocSpec extends AkkaSpec with CompileOnlySpec {
|
|||
val fromProducer: Source[String, NotUsed] = runnableGraph.run()
|
||||
|
||||
// Print out messages from the producer in two independent consumers
|
||||
fromProducer.runForeach(msg => println("consumer1: " + msg))
|
||||
fromProducer.runForeach(msg => println("consumer2: " + msg))
|
||||
fromProducer.runForeach(msg ⇒ println("consumer1: " + msg))
|
||||
fromProducer.runForeach(msg ⇒ println("consumer2: " + msg))
|
||||
//#partition-hub-stateful
|
||||
}
|
||||
|
||||
|
|
@ -174,14 +174,14 @@ class HubsDocSpec extends AkkaSpec with CompileOnlySpec {
|
|||
// Note that this is a moving target since the elements are consumed concurrently.
|
||||
val runnableGraph: RunnableGraph[Source[Int, NotUsed]] =
|
||||
producer.toMat(PartitionHub.statefulSink(
|
||||
() => (info, elem) ⇒ info.consumerIds.minBy(id ⇒ info.queueSize(id)),
|
||||
() ⇒ (info, elem) ⇒ info.consumerIds.minBy(id ⇒ info.queueSize(id)),
|
||||
startAfterNrOfConsumers = 2, bufferSize = 16))(Keep.right)
|
||||
|
||||
val fromProducer: Source[Int, NotUsed] = runnableGraph.run()
|
||||
|
||||
fromProducer.runForeach(msg => println("consumer1: " + msg))
|
||||
fromProducer.runForeach(msg ⇒ println("consumer1: " + msg))
|
||||
fromProducer.throttle(10, 100.millis, 10, ThrottleMode.Shaping)
|
||||
.runForeach(msg => println("consumer2: " + msg))
|
||||
.runForeach(msg ⇒ println("consumer2: " + msg))
|
||||
//#partition-hub-fastest
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -92,7 +92,7 @@ object IntegrationDocSpec {
|
|||
|
||||
class DatabaseService(probe: ActorRef) extends Actor {
|
||||
override def receive = {
|
||||
case Save(tweet: Tweet) =>
|
||||
case Save(tweet: Tweet) ⇒
|
||||
probe ! tweet.author.handle
|
||||
sender() ! SaveDone
|
||||
}
|
||||
|
|
@ -123,7 +123,7 @@ object IntegrationDocSpec {
|
|||
//#ask-actor
|
||||
class Translator extends Actor {
|
||||
def receive = {
|
||||
case word: String =>
|
||||
case word: String ⇒
|
||||
// ... process message
|
||||
val reply = word.toUpperCase
|
||||
sender() ! reply // reply to the ask
|
||||
|
|
@ -148,7 +148,7 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
|
|||
Source(List("hello", "hi"))
|
||||
|
||||
words
|
||||
.mapAsync(parallelism = 5)(elem => (ref ? elem).mapTo[String])
|
||||
.mapAsync(parallelism = 5)(elem ⇒ (ref ? elem).mapTo[String])
|
||||
// continue processing of the replies from the actor
|
||||
.map(_.toLowerCase)
|
||||
.runWith(Sink.ignore)
|
||||
|
|
@ -170,14 +170,14 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
|
|||
//#email-addresses-mapAsync
|
||||
val emailAddresses: Source[String, NotUsed] =
|
||||
authors
|
||||
.mapAsync(4)(author => addressSystem.lookupEmail(author.handle))
|
||||
.collect { case Some(emailAddress) => emailAddress }
|
||||
.mapAsync(4)(author ⇒ addressSystem.lookupEmail(author.handle))
|
||||
.collect { case Some(emailAddress) ⇒ emailAddress }
|
||||
//#email-addresses-mapAsync
|
||||
|
||||
//#send-emails
|
||||
val sendEmails: RunnableGraph[NotUsed] =
|
||||
emailAddresses
|
||||
.mapAsync(4)(address => {
|
||||
.mapAsync(4)(address ⇒ {
|
||||
emailServer.send(
|
||||
Email(to = address, title = "Akka", body = "I like your tweet"))
|
||||
})
|
||||
|
|
@ -206,7 +206,7 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
|
|||
|
||||
val emailAddresses: Source[String, NotUsed] =
|
||||
authors.via(
|
||||
Flow[Author].mapAsync(4)(author => addressSystem.lookupEmail(author.handle))
|
||||
Flow[Author].mapAsync(4)(author ⇒ addressSystem.lookupEmail(author.handle))
|
||||
.withAttributes(supervisionStrategy(resumingDecider)))
|
||||
//#email-addresses-mapAsync-supervision
|
||||
}
|
||||
|
|
@ -222,12 +222,12 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
|
|||
|
||||
val emailAddresses: Source[String, NotUsed] =
|
||||
authors
|
||||
.mapAsyncUnordered(4)(author => addressSystem.lookupEmail(author.handle))
|
||||
.collect { case Some(emailAddress) => emailAddress }
|
||||
.mapAsyncUnordered(4)(author ⇒ addressSystem.lookupEmail(author.handle))
|
||||
.collect { case Some(emailAddress) ⇒ emailAddress }
|
||||
|
||||
val sendEmails: RunnableGraph[NotUsed] =
|
||||
emailAddresses
|
||||
.mapAsyncUnordered(4)(address => {
|
||||
.mapAsyncUnordered(4)(address ⇒ {
|
||||
emailServer.send(
|
||||
Email(to = address, title = "Akka", body = "I like your tweet"))
|
||||
})
|
||||
|
|
@ -254,15 +254,15 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
|
|||
val authors = tweets.filter(_.hashtags.contains(akkaTag)).map(_.author)
|
||||
|
||||
val phoneNumbers =
|
||||
authors.mapAsync(4)(author => addressSystem.lookupPhoneNumber(author.handle))
|
||||
.collect { case Some(phoneNo) => phoneNo }
|
||||
authors.mapAsync(4)(author ⇒ addressSystem.lookupPhoneNumber(author.handle))
|
||||
.collect { case Some(phoneNo) ⇒ phoneNo }
|
||||
|
||||
//#blocking-mapAsync
|
||||
val blockingExecutionContext = system.dispatchers.lookup("blocking-dispatcher")
|
||||
|
||||
val sendTextMessages: RunnableGraph[NotUsed] =
|
||||
phoneNumbers
|
||||
.mapAsync(4)(phoneNo => {
|
||||
.mapAsync(4)(phoneNo ⇒ {
|
||||
Future {
|
||||
smsServer.send(
|
||||
TextMessage(to = phoneNo, body = "I like your tweet"))
|
||||
|
|
@ -291,12 +291,12 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
|
|||
val authors = tweets.filter(_.hashtags.contains(akkaTag)).map(_.author)
|
||||
|
||||
val phoneNumbers =
|
||||
authors.mapAsync(4)(author => addressSystem.lookupPhoneNumber(author.handle))
|
||||
.collect { case Some(phoneNo) => phoneNo }
|
||||
authors.mapAsync(4)(author ⇒ addressSystem.lookupPhoneNumber(author.handle))
|
||||
.collect { case Some(phoneNo) ⇒ phoneNo }
|
||||
|
||||
//#blocking-map
|
||||
val send = Flow[String]
|
||||
.map { phoneNo =>
|
||||
.map { phoneNo ⇒
|
||||
smsServer.send(TextMessage(to = phoneNo, body = "I like your tweet"))
|
||||
}
|
||||
.withAttributes(ActorAttributes.dispatcher("blocking-dispatcher"))
|
||||
|
|
@ -327,7 +327,7 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
|
|||
implicit val timeout = Timeout(3.seconds)
|
||||
val saveTweets: RunnableGraph[NotUsed] =
|
||||
akkaTweets
|
||||
.mapAsync(4)(tweet => database ? Save(tweet))
|
||||
.mapAsync(4)(tweet ⇒ database ? Save(tweet))
|
||||
.to(Sink.ignore)
|
||||
//#save-tweets
|
||||
|
||||
|
|
@ -357,9 +357,9 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
|
|||
ActorMaterializerSettings(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 })
|
||||
.map(elem ⇒ { println(s"before: $elem"); elem })
|
||||
.mapAsync(4)(service.convert)
|
||||
.runForeach(elem => println(s"after: $elem"))
|
||||
.runForeach(elem ⇒ println(s"after: $elem"))
|
||||
//#sometimes-slow-mapAsync
|
||||
|
||||
probe.expectMsg("after: A")
|
||||
|
|
@ -389,9 +389,9 @@ class IntegrationDocSpec extends AkkaSpec(IntegrationDocSpec.config) {
|
|||
ActorMaterializerSettings(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 })
|
||||
.map(elem ⇒ { println(s"before: $elem"); elem })
|
||||
.mapAsyncUnordered(4)(service.convert)
|
||||
.runForeach(elem => println(s"after: $elem"))
|
||||
.runForeach(elem ⇒ println(s"after: $elem"))
|
||||
//#sometimes-slow-mapAsyncUnordered
|
||||
|
||||
probe.receiveN(10).toSet should be(Set(
|
||||
|
|
|
|||
|
|
@ -15,7 +15,7 @@ class MigrationsScala extends AkkaSpec {
|
|||
Flow[Int].expand(Iterator.continually(_))
|
||||
//#expand-continually
|
||||
//#expand-state
|
||||
Flow[Int].expand(i => {
|
||||
Flow[Int].expand(i ⇒ {
|
||||
var state = 0
|
||||
Iterator.continually({
|
||||
state += 1
|
||||
|
|
|
|||
|
|
@ -42,15 +42,15 @@ class QuickStartDocSpec extends WordSpec with BeforeAndAfterAll with ScalaFuture
|
|||
//#create-source
|
||||
|
||||
//#run-source
|
||||
source.runForeach(i => println(i))(materializer)
|
||||
source.runForeach(i ⇒ println(i))(materializer)
|
||||
//#run-source
|
||||
|
||||
//#transform-source
|
||||
val factorials = source.scan(BigInt(1))((acc, next) => acc * next)
|
||||
val factorials = source.scan(BigInt(1))((acc, next) ⇒ acc * next)
|
||||
|
||||
val result: Future[IOResult] =
|
||||
factorials
|
||||
.map(num => ByteString(s"$num\n"))
|
||||
.map(num ⇒ ByteString(s"$num\n"))
|
||||
.runWith(FileIO.toPath(Paths.get("factorials.txt")))
|
||||
//#transform-source
|
||||
|
||||
|
|
@ -60,7 +60,7 @@ class QuickStartDocSpec extends WordSpec with BeforeAndAfterAll with ScalaFuture
|
|||
|
||||
//#add-streams
|
||||
factorials
|
||||
.zipWith(Source(0 to 100))((num, idx) => s"$idx! = $num")
|
||||
.zipWith(Source(0 to 100))((num, idx) ⇒ s"$idx! = $num")
|
||||
.throttle(1, 1.second, 1, ThrottleMode.shaping)
|
||||
//#add-streams
|
||||
.take(3)
|
||||
|
|
@ -69,10 +69,10 @@ class QuickStartDocSpec extends WordSpec with BeforeAndAfterAll with ScalaFuture
|
|||
//#add-streams
|
||||
|
||||
//#run-source-and-terminate
|
||||
val done: Future[Done] = source.runForeach(i => println(i))(materializer)
|
||||
val done: Future[Done] = source.runForeach(i ⇒ println(i))(materializer)
|
||||
|
||||
implicit val ec = system.dispatcher
|
||||
done.onComplete(_ => system.terminate())
|
||||
done.onComplete(_ ⇒ system.terminate())
|
||||
//#run-source-and-terminate
|
||||
|
||||
done.futureValue
|
||||
|
|
@ -81,7 +81,7 @@ class QuickStartDocSpec extends WordSpec with BeforeAndAfterAll with ScalaFuture
|
|||
//#transform-sink
|
||||
def lineSink(filename: String): Sink[String, Future[IOResult]] =
|
||||
Flow[String]
|
||||
.map(s => ByteString(s + "\n"))
|
||||
.map(s ⇒ ByteString(s + "\n"))
|
||||
.toMat(FileIO.toPath(Paths.get(filename)))(Keep.right)
|
||||
//#transform-sink
|
||||
|
||||
|
|
|
|||
|
|
@ -25,15 +25,15 @@ class RateTransformationDocSpec extends AkkaSpec {
|
|||
//#conflate-summarize
|
||||
val statsFlow = Flow[Double]
|
||||
.conflateWithSeed(Seq(_))(_ :+ _)
|
||||
.map { s =>
|
||||
.map { s ⇒
|
||||
val μ = s.sum / s.size
|
||||
val se = s.map(x => pow(x - μ, 2))
|
||||
val se = s.map(x ⇒ pow(x - μ, 2))
|
||||
val σ = sqrt(se.sum / se.size)
|
||||
(σ, μ, s.size)
|
||||
}
|
||||
//#conflate-summarize
|
||||
|
||||
val fut = Source.fromIterator(() => Iterator.continually(Random.nextGaussian))
|
||||
val fut = Source.fromIterator(() ⇒ Iterator.continually(Random.nextGaussian))
|
||||
.via(statsFlow)
|
||||
.grouped(10)
|
||||
.runWith(Sink.head)
|
||||
|
|
@ -46,8 +46,8 @@ class RateTransformationDocSpec extends AkkaSpec {
|
|||
val p = 0.01
|
||||
val sampleFlow = Flow[Double]
|
||||
.conflateWithSeed(Seq(_)) {
|
||||
case (acc, elem) if Random.nextDouble < p => acc :+ elem
|
||||
case (acc, _) => acc
|
||||
case (acc, elem) if Random.nextDouble < p ⇒ acc :+ elem
|
||||
case (acc, _) ⇒ acc
|
||||
}
|
||||
.mapConcat(identity)
|
||||
//#conflate-sample
|
||||
|
|
@ -81,11 +81,11 @@ class RateTransformationDocSpec extends AkkaSpec {
|
|||
"expand should track drift" in {
|
||||
//#expand-drift
|
||||
val driftFlow = Flow[Double]
|
||||
.expand(i => Iterator.from(0).map(i -> _))
|
||||
.expand(i ⇒ Iterator.from(0).map(i -> _))
|
||||
//#expand-drift
|
||||
val latch = TestLatch(2)
|
||||
val realDriftFlow = Flow[Double]
|
||||
.expand(d => { latch.countDown(); Iterator.from(0).map(d -> _) })
|
||||
.expand(d ⇒ { latch.countDown(); Iterator.from(0).map(d -> _) })
|
||||
|
||||
val (pub, sub) = TestSource.probe[Double]
|
||||
.via(realDriftFlow)
|
||||
|
|
|
|||
|
|
@ -141,7 +141,7 @@ class ReactiveStreamsDocSpec extends AkkaSpec {
|
|||
// An example Processor factory
|
||||
def createProcessor: Processor[Int, Int] = Flow[Int].toProcessor.run()
|
||||
|
||||
val flow: Flow[Int, Int, NotUsed] = Flow.fromProcessor(() => createProcessor)
|
||||
val flow: Flow[Int, Int, NotUsed] = Flow.fromProcessor(() ⇒ createProcessor)
|
||||
//#use-processor
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -38,7 +38,7 @@ class RestartDocSpec extends AkkaSpec with CompileOnlySpec {
|
|||
minBackoff = 3.seconds,
|
||||
maxBackoff = 30.seconds,
|
||||
randomFactor = 0.2 // adds 20% "noise" to vary the intervals slightly
|
||||
) { () =>
|
||||
) { () ⇒
|
||||
// Create a source from a future of a source
|
||||
Source.fromFutureSource {
|
||||
// Make a single request with akka-http
|
||||
|
|
@ -54,7 +54,7 @@ class RestartDocSpec extends AkkaSpec with CompileOnlySpec {
|
|||
//#with-kill-switch
|
||||
val killSwitch = restartSource
|
||||
.viaMat(KillSwitches.single)(Keep.right)
|
||||
.toMat(Sink.foreach(event => println(s"Got event: $event")))(Keep.left)
|
||||
.toMat(Sink.foreach(event ⇒ println(s"Got event: $event")))(Keep.left)
|
||||
.run()
|
||||
|
||||
doSomethingElse()
|
||||
|
|
@ -64,4 +64,4 @@ class RestartDocSpec extends AkkaSpec with CompileOnlySpec {
|
|||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -12,9 +12,9 @@ class StreamBuffersRateSpec extends AkkaSpec {
|
|||
def println(s: Any) = ()
|
||||
//#pipelining
|
||||
Source(1 to 3)
|
||||
.map { i => println(s"A: $i"); i }.async
|
||||
.map { i => println(s"B: $i"); i }.async
|
||||
.map { i => println(s"C: $i"); i }.async
|
||||
.map { i ⇒ println(s"A: $i"); i }.async
|
||||
.map { i ⇒ println(s"B: $i"); i }.async
|
||||
.map { i ⇒ println(s"C: $i"); i }.async
|
||||
.runWith(Sink.ignore)
|
||||
//#pipelining
|
||||
}
|
||||
|
|
@ -40,16 +40,16 @@ class StreamBuffersRateSpec extends AkkaSpec {
|
|||
import scala.concurrent.duration._
|
||||
case class Tick()
|
||||
|
||||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b =>
|
||||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
import GraphDSL.Implicits._
|
||||
|
||||
// this is the asynchronous stage in this graph
|
||||
val zipper = b.add(ZipWith[Tick, Int, Int]((tick, count) => count).async)
|
||||
val zipper = b.add(ZipWith[Tick, Int, Int]((tick, count) ⇒ count).async)
|
||||
|
||||
Source.tick(initialDelay = 3.second, interval = 3.second, Tick()) ~> zipper.in0
|
||||
|
||||
Source.tick(initialDelay = 1.second, interval = 1.second, "message!")
|
||||
.conflateWithSeed(seed = (_) => 1)((count, _) => count + 1) ~> zipper.in1
|
||||
.conflateWithSeed(seed = (_) ⇒ 1)((count, _) ⇒ count + 1) ~> zipper.in1
|
||||
|
||||
zipper.out ~> Sink.foreach(println)
|
||||
ClosedShape
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ class StreamPartialGraphDSLDocSpec extends AkkaSpec {
|
|||
|
||||
"build with open ports" in {
|
||||
//#simple-partial-graph-dsl
|
||||
val pickMaxOfThree = GraphDSL.create() { implicit b =>
|
||||
val pickMaxOfThree = GraphDSL.create() { implicit b ⇒
|
||||
import GraphDSL.Implicits._
|
||||
|
||||
val zip1 = b.add(ZipWith[Int, Int, Int](math.max _))
|
||||
|
|
@ -31,7 +31,7 @@ class StreamPartialGraphDSLDocSpec extends AkkaSpec {
|
|||
|
||||
val resultSink = Sink.head[Int]
|
||||
|
||||
val g = RunnableGraph.fromGraph(GraphDSL.create(resultSink) { implicit b => sink =>
|
||||
val g = RunnableGraph.fromGraph(GraphDSL.create(resultSink) { implicit b ⇒ sink ⇒
|
||||
import GraphDSL.Implicits._
|
||||
|
||||
// importing the partial graph will return its shape (inlets & outlets)
|
||||
|
|
@ -51,12 +51,12 @@ class StreamPartialGraphDSLDocSpec extends AkkaSpec {
|
|||
|
||||
"build source from partial graph" in {
|
||||
//#source-from-partial-graph-dsl
|
||||
val pairs = Source.fromGraph(GraphDSL.create() { implicit b =>
|
||||
val pairs = Source.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
import GraphDSL.Implicits._
|
||||
|
||||
// prepare graph elements
|
||||
val zip = b.add(Zip[Int, Int]())
|
||||
def ints = Source.fromIterator(() => Iterator.from(1))
|
||||
def ints = Source.fromIterator(() ⇒ Iterator.from(1))
|
||||
|
||||
// connect the graph
|
||||
ints.filter(_ % 2 != 0) ~> zip.in0
|
||||
|
|
@ -74,7 +74,7 @@ class StreamPartialGraphDSLDocSpec extends AkkaSpec {
|
|||
"build flow from partial graph" in {
|
||||
//#flow-from-partial-graph-dsl
|
||||
val pairUpWithToString =
|
||||
Flow.fromGraph(GraphDSL.create() { implicit b =>
|
||||
Flow.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
import GraphDSL.Implicits._
|
||||
|
||||
// prepare graph elements
|
||||
|
|
@ -116,7 +116,7 @@ class StreamPartialGraphDSLDocSpec extends AkkaSpec {
|
|||
val actorRef: ActorRef = testActor
|
||||
//#sink-combine
|
||||
val sendRmotely = Sink.actorRef(actorRef, "Done")
|
||||
val localProcessing = Sink.foreach[Int](_ => /* do something usefull */ ())
|
||||
val localProcessing = Sink.foreach[Int](_ ⇒ /* do something usefull */ ())
|
||||
|
||||
val sink = Sink.combine(sendRmotely, localProcessing)(Broadcast[Int](_))
|
||||
|
||||
|
|
|
|||
|
|
@ -137,7 +137,7 @@ class StreamTestKitDocSpec extends AkkaSpec {
|
|||
"test source and a sink" in {
|
||||
import system.dispatcher
|
||||
//#test-source-and-sink
|
||||
val flowUnderTest = Flow[Int].mapAsyncUnordered(2) { sleep =>
|
||||
val flowUnderTest = Flow[Int].mapAsyncUnordered(2) { sleep ⇒
|
||||
pattern.after(10.millis * sleep, using = system.scheduler)(Future.successful(sleep))
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -30,7 +30,7 @@ object TwitterStreamQuickstartDocSpec {
|
|||
|
||||
final case class Tweet(author: Author, timestamp: Long, body: String) {
|
||||
def hashtags: Set[Hashtag] = body.split(" ").collect {
|
||||
case t if t.startsWith("#") => Hashtag(t.replaceAll("[^#\\w]", ""))
|
||||
case t if t.startsWith("#") ⇒ Hashtag(t.replaceAll("[^#\\w]", ""))
|
||||
}.toSet
|
||||
}
|
||||
|
||||
|
|
@ -98,7 +98,7 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
|
|||
trait Example3 {
|
||||
//#authors-collect
|
||||
val authors: Source[Author, NotUsed] =
|
||||
tweets.collect { case t if t.hashtags.contains(akkaTag) => t.author }
|
||||
tweets.collect { case t if t.hashtags.contains(akkaTag) ⇒ t.author }
|
||||
//#authors-collect
|
||||
}
|
||||
|
||||
|
|
@ -184,8 +184,8 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
|
|||
//#backpressure-by-readline
|
||||
val completion: Future[Done] =
|
||||
Source(1 to 10)
|
||||
.map(i => { println(s"map => $i"); i })
|
||||
.runForeach { i => readLine(s"Element = $i; continue reading? [press enter]\n") }
|
||||
.map(i ⇒ { println(s"map => $i"); i })
|
||||
.runForeach { i ⇒ readLine(s"Element = $i; continue reading? [press enter]\n") }
|
||||
|
||||
Await.ready(completion, 1.minute)
|
||||
//#backpressure-by-readline
|
||||
|
|
@ -194,7 +194,7 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
|
|||
|
||||
"count elements on finite stream" in {
|
||||
//#tweets-fold-count
|
||||
val count: Flow[Tweet, Int, NotUsed] = Flow[Tweet].map(_ => 1)
|
||||
val count: Flow[Tweet, Int, NotUsed] = Flow[Tweet].map(_ ⇒ 1)
|
||||
|
||||
val sumSink: Sink[Int, Future[Int]] = Sink.fold[Int, Int](0)(_ + _)
|
||||
|
||||
|
|
@ -205,12 +205,12 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
|
|||
|
||||
val sum: Future[Int] = counterGraph.run()
|
||||
|
||||
sum.foreach(c => println(s"Total tweets processed: $c"))
|
||||
sum.foreach(c ⇒ println(s"Total tweets processed: $c"))
|
||||
//#tweets-fold-count
|
||||
|
||||
new AnyRef {
|
||||
//#tweets-fold-count-oneline
|
||||
val sum: Future[Int] = tweets.map(t => 1).runWith(sumSink)
|
||||
val sum: Future[Int] = tweets.map(t ⇒ 1).runWith(sumSink)
|
||||
//#tweets-fold-count-oneline
|
||||
}
|
||||
}
|
||||
|
|
@ -223,7 +223,7 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
|
|||
val counterRunnableGraph: RunnableGraph[Future[Int]] =
|
||||
tweetsInMinuteFromNow
|
||||
.filter(_.hashtags contains akkaTag)
|
||||
.map(t => 1)
|
||||
.map(t ⇒ 1)
|
||||
.toMat(sumSink)(Keep.right)
|
||||
|
||||
// materialize the stream once in the morning
|
||||
|
|
@ -235,7 +235,7 @@ class TwitterStreamQuickstartDocSpec extends AkkaSpec {
|
|||
|
||||
val sum: Future[Int] = counterRunnableGraph.run()
|
||||
|
||||
sum.map { c => println(s"Total tweets processed: $c") }
|
||||
sum.map { c ⇒ println(s"Total tweets processed: $c") }
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -23,7 +23,7 @@ class RecipeDroppyBroadcast extends RecipeSpec {
|
|||
val mySink3 = Sink.fromSubscriber(sub3)
|
||||
|
||||
//#droppy-bcast
|
||||
val graph = RunnableGraph.fromGraph(GraphDSL.create(mySink1, mySink2, mySink3)((_, _, _)) { implicit b => (sink1, sink2, sink3) =>
|
||||
val graph = RunnableGraph.fromGraph(GraphDSL.create(mySink1, mySink2, mySink3)((_, _, _)) { implicit b ⇒ (sink1, sink2, sink3) ⇒
|
||||
import GraphDSL.Implicits._
|
||||
|
||||
val bcast = b.add(Broadcast[Int](3))
|
||||
|
|
@ -39,7 +39,7 @@ class RecipeDroppyBroadcast extends RecipeSpec {
|
|||
graph.run()
|
||||
|
||||
sub3.request(100)
|
||||
for (i <- 1 to 100) {
|
||||
for (i ← 1 to 100) {
|
||||
pub.sendNext(i)
|
||||
sub3.expectNext(i)
|
||||
}
|
||||
|
|
@ -49,7 +49,7 @@ class RecipeDroppyBroadcast extends RecipeSpec {
|
|||
sub1.expectSubscription().request(10)
|
||||
sub2.expectSubscription().request(10)
|
||||
|
||||
for (i <- 91 to 100) {
|
||||
for (i ← 91 to 100) {
|
||||
sub1.expectNext(i)
|
||||
sub2.expectNext(i)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -45,19 +45,19 @@ class RecipeGlobalRateLimit extends RecipeSpec {
|
|||
override def receive: Receive = open
|
||||
|
||||
val open: Receive = {
|
||||
case ReplenishTokens =>
|
||||
case ReplenishTokens ⇒
|
||||
permitTokens = math.min(permitTokens + tokenRefreshAmount, maxAvailableTokens)
|
||||
case WantToPass =>
|
||||
case WantToPass ⇒
|
||||
permitTokens -= 1
|
||||
sender() ! MayPass
|
||||
if (permitTokens == 0) context.become(closed)
|
||||
}
|
||||
|
||||
val closed: Receive = {
|
||||
case ReplenishTokens =>
|
||||
case ReplenishTokens ⇒
|
||||
permitTokens = math.min(permitTokens + tokenRefreshAmount, maxAvailableTokens)
|
||||
releaseWaiting()
|
||||
case WantToPass =>
|
||||
case WantToPass ⇒
|
||||
waitQueue = waitQueue.enqueue(sender())
|
||||
}
|
||||
|
||||
|
|
@ -82,11 +82,11 @@ class RecipeGlobalRateLimit extends RecipeSpec {
|
|||
def limitGlobal[T](limiter: ActorRef, maxAllowedWait: FiniteDuration): Flow[T, T, NotUsed] = {
|
||||
import akka.pattern.ask
|
||||
import akka.util.Timeout
|
||||
Flow[T].mapAsync(4)((element: T) => {
|
||||
Flow[T].mapAsync(4)((element: T) ⇒ {
|
||||
import system.dispatcher
|
||||
implicit val triggerTimeout = Timeout(maxAllowedWait)
|
||||
val limiterTriggerFuture = limiter ? Limiter.WantToPass
|
||||
limiterTriggerFuture.map((_) => element)
|
||||
limiterTriggerFuture.map((_) ⇒ element)
|
||||
})
|
||||
|
||||
}
|
||||
|
|
@ -95,12 +95,12 @@ class RecipeGlobalRateLimit extends RecipeSpec {
|
|||
// Use a large period and emulate the timer by hand instead
|
||||
val limiter = system.actorOf(Limiter.props(2, 100.days, 1), "limiter")
|
||||
|
||||
val source1 = Source.fromIterator(() => Iterator.continually("E1")).via(limitGlobal(limiter, 2.seconds.dilated))
|
||||
val source2 = Source.fromIterator(() => Iterator.continually("E2")).via(limitGlobal(limiter, 2.seconds.dilated))
|
||||
val source1 = Source.fromIterator(() ⇒ Iterator.continually("E1")).via(limitGlobal(limiter, 2.seconds.dilated))
|
||||
val source2 = Source.fromIterator(() ⇒ Iterator.continually("E2")).via(limitGlobal(limiter, 2.seconds.dilated))
|
||||
|
||||
val probe = TestSubscriber.manualProbe[String]()
|
||||
|
||||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b =>
|
||||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
import GraphDSL.Implicits._
|
||||
val merge = b.add(Merge[String](2))
|
||||
source1 ~> merge ~> Sink.fromSubscriber(probe)
|
||||
|
|
@ -119,7 +119,7 @@ class RecipeGlobalRateLimit extends RecipeSpec {
|
|||
probe.expectNoMsg(500.millis)
|
||||
|
||||
var resultSet = Set.empty[String]
|
||||
for (_ <- 1 to 100) {
|
||||
for (_ ← 1 to 100) {
|
||||
limiter ! Limiter.ReplenishTokens
|
||||
resultSet += probe.expectNext()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -15,7 +15,7 @@ class RecipeKeepAlive extends RecipeSpec {
|
|||
//#inject-keepalive
|
||||
import scala.concurrent.duration._
|
||||
val injectKeepAlive: Flow[ByteString, ByteString, NotUsed] =
|
||||
Flow[ByteString].keepAlive(1.second, () => keepaliveMessage)
|
||||
Flow[ByteString].keepAlive(1.second, () ⇒ keepaliveMessage)
|
||||
//#inject-keepalive
|
||||
|
||||
// No need to test, this is a built-in stage with proper tests
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ class RecipeLoggingElements extends RecipeSpec {
|
|||
val mySource = Source(List("1", "2", "3"))
|
||||
|
||||
//#println-debug
|
||||
val loggedSource = mySource.map { elem => println(elem); elem }
|
||||
val loggedSource = mySource.map { elem ⇒ println(elem); elem }
|
||||
//#println-debug
|
||||
|
||||
loggedSource.runWith(Sink.ignore)
|
||||
|
|
|
|||
|
|
@ -18,12 +18,12 @@ class RecipeManualTrigger extends RecipeSpec {
|
|||
val sink = Sink.fromSubscriber(sub)
|
||||
|
||||
//#manually-triggered-stream
|
||||
val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit builder =>
|
||||
val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit builder ⇒
|
||||
import GraphDSL.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
|
||||
zip.out ~> Flow[(Message, Trigger)].map { case (msg, trigger) ⇒ msg } ~> sink
|
||||
ClosedShape
|
||||
})
|
||||
//#manually-triggered-stream
|
||||
|
|
@ -57,9 +57,9 @@ class RecipeManualTrigger extends RecipeSpec {
|
|||
val sink = Sink.fromSubscriber(sub)
|
||||
|
||||
//#manually-triggered-stream-zipwith
|
||||
val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit builder =>
|
||||
val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit builder ⇒
|
||||
import GraphDSL.Implicits._
|
||||
val zip = builder.add(ZipWith((msg: Message, trigger: Trigger) => msg))
|
||||
val zip = builder.add(ZipWith((msg: Message, trigger: Trigger) ⇒ msg))
|
||||
|
||||
elements ~> zip.in0
|
||||
triggerSource ~> zip.in1
|
||||
|
|
|
|||
|
|
@ -21,13 +21,13 @@ class RecipeMissedTicks extends RecipeSpec {
|
|||
|
||||
//#missed-ticks
|
||||
val missedTicks: Flow[Tick, Int, NotUsed] =
|
||||
Flow[Tick].conflateWithSeed(seed = (_) => 0)(
|
||||
(missedTicks, tick) => missedTicks + 1)
|
||||
Flow[Tick].conflateWithSeed(seed = (_) ⇒ 0)(
|
||||
(missedTicks, tick) ⇒ missedTicks + 1)
|
||||
//#missed-ticks
|
||||
val latch = TestLatch(3)
|
||||
val realMissedTicks: Flow[Tick, Int, NotUsed] =
|
||||
Flow[Tick].conflateWithSeed(seed = (_) => 0)(
|
||||
(missedTicks, tick) => { latch.countDown(); missedTicks + 1 })
|
||||
Flow[Tick].conflateWithSeed(seed = (_) ⇒ 0)(
|
||||
(missedTicks, tick) ⇒ { latch.countDown(); missedTicks + 1 })
|
||||
|
||||
tickStream.via(realMissedTicks).to(sink).run()
|
||||
|
||||
|
|
|
|||
|
|
@ -16,15 +16,15 @@ class RecipeMultiGroupBy extends RecipeSpec {
|
|||
case class Topic(name: String)
|
||||
|
||||
val elems = Source(List("1: a", "1: b", "all: c", "all: d", "1: e"))
|
||||
val extractTopics = { msg: Message =>
|
||||
val extractTopics = { msg: Message ⇒
|
||||
if (msg.startsWith("1")) List(Topic("1"))
|
||||
else List(Topic("1"), Topic("2"))
|
||||
}
|
||||
|
||||
//#multi-groupby
|
||||
val topicMapper: (Message) => immutable.Seq[Topic] = extractTopics
|
||||
val topicMapper: (Message) ⇒ immutable.Seq[Topic] = extractTopics
|
||||
|
||||
val messageAndTopic: Source[(Message, Topic), NotUsed] = elems.mapConcat { msg: Message =>
|
||||
val messageAndTopic: Source[(Message, Topic), NotUsed] = elems.mapConcat { msg: Message ⇒
|
||||
val topicsForMessage = topicMapper(msg)
|
||||
// Create a (Msg, Topic) pair for each of the topics
|
||||
// the message belongs to
|
||||
|
|
@ -33,7 +33,7 @@ class RecipeMultiGroupBy extends RecipeSpec {
|
|||
|
||||
val multiGroups = messageAndTopic
|
||||
.groupBy(2, _._2).map {
|
||||
case (msg, topic) =>
|
||||
case (msg, topic) ⇒
|
||||
// do what needs to be done
|
||||
//#multi-groupby
|
||||
(msg, topic)
|
||||
|
|
@ -44,7 +44,7 @@ class RecipeMultiGroupBy extends RecipeSpec {
|
|||
val result = multiGroups
|
||||
.grouped(10)
|
||||
.mergeSubstreams
|
||||
.map(g => g.head._2.name + g.map(_._1).mkString("[", ", ", "]"))
|
||||
.map(g ⇒ g.head._2.name + g.map(_._1).mkString("[", ", ", "]"))
|
||||
.limit(10)
|
||||
.runWith(Sink.seq)
|
||||
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ class RecipeReduceByKey extends RecipeSpec {
|
|||
//transform each element to pair with number of words in it
|
||||
.map(_ -> 1)
|
||||
// add counting logic to the streams
|
||||
.reduce((l, r) => (l._1, l._2 + r._2))
|
||||
.reduce((l, r) ⇒ (l._1, l._2 + r._2))
|
||||
// get a stream of word counts
|
||||
.mergeSubstreams
|
||||
//#word-count
|
||||
|
|
@ -45,21 +45,21 @@ class RecipeReduceByKey extends RecipeSpec {
|
|||
//#reduce-by-key-general
|
||||
def reduceByKey[In, K, Out](
|
||||
maximumGroupSize: Int,
|
||||
groupKey: (In) => K,
|
||||
map: (In) => Out)(reduce: (Out, Out) => Out): Flow[In, (K, Out), NotUsed] = {
|
||||
groupKey: (In) ⇒ K,
|
||||
map: (In) ⇒ Out)(reduce: (Out, Out) ⇒ Out): Flow[In, (K, Out), NotUsed] = {
|
||||
|
||||
Flow[In]
|
||||
.groupBy[K](maximumGroupSize, groupKey)
|
||||
.map(e => groupKey(e) -> map(e))
|
||||
.reduce((l, r) => l._1 -> reduce(l._2, r._2))
|
||||
.map(e ⇒ groupKey(e) -> map(e))
|
||||
.reduce((l, r) ⇒ l._1 -> reduce(l._2, r._2))
|
||||
.mergeSubstreams
|
||||
}
|
||||
|
||||
val wordCounts = words.via(
|
||||
reduceByKey(
|
||||
MaximumDistinctWords,
|
||||
groupKey = (word: String) => word,
|
||||
map = (word: String) => 1)((left: Int, right: Int) => left + right))
|
||||
groupKey = (word: String) ⇒ word,
|
||||
map = (word: String) ⇒ 1)((left: Int, right: Int) ⇒ left + right))
|
||||
//#reduce-by-key-general
|
||||
|
||||
Await.result(wordCounts.limit(10).runWith(Sink.seq), 3.seconds).toSet should be(Set(
|
||||
|
|
|
|||
|
|
@ -15,11 +15,11 @@ class RecipeSimpleDrop extends RecipeSpec {
|
|||
|
||||
//#simple-drop
|
||||
val droppyStream: Flow[Message, Message, NotUsed] =
|
||||
Flow[Message].conflate((lastMessage, newMessage) => newMessage)
|
||||
Flow[Message].conflate((lastMessage, newMessage) ⇒ newMessage)
|
||||
//#simple-drop
|
||||
val latch = TestLatch(2)
|
||||
val realDroppyStream =
|
||||
Flow[Message].conflate((lastMessage, newMessage) => { latch.countDown(); newMessage })
|
||||
Flow[Message].conflate((lastMessage, newMessage) ⇒ { latch.countDown(); newMessage })
|
||||
|
||||
val pub = TestPublisher.probe[Message]()
|
||||
val sub = TestSubscriber.manualProbe[Message]()
|
||||
|
|
|
|||
|
|
@ -22,11 +22,11 @@ class RecipeWorkerPool extends RecipeSpec {
|
|||
def balancer[In, Out](worker: Flow[In, Out, Any], workerCount: Int): Flow[In, Out, NotUsed] = {
|
||||
import GraphDSL.Implicits._
|
||||
|
||||
Flow.fromGraph(GraphDSL.create() { implicit b =>
|
||||
Flow.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
val balancer = b.add(Balance[In](workerCount, waitForAllDownstreams = true))
|
||||
val merge = b.add(Merge[Out](workerCount))
|
||||
|
||||
for (_ <- 1 to workerCount) {
|
||||
for (_ ← 1 to workerCount) {
|
||||
// for each worker, add an edge from the balancer to the worker, then wire
|
||||
// it to the merge element
|
||||
balancer ~> worker.async ~> merge
|
||||
|
|
|
|||
|
|
@ -29,9 +29,9 @@ class StreamTcpDocSpec extends AkkaSpec {
|
|||
val binding: Future[ServerBinding] =
|
||||
Tcp().bind("127.0.0.1", 8888).to(Sink.ignore).run()
|
||||
|
||||
binding.map { b =>
|
||||
binding.map { b ⇒
|
||||
b.unbind() onComplete {
|
||||
case _ => // ...
|
||||
case _ ⇒ // ...
|
||||
}
|
||||
}
|
||||
//#echo-server-simple-bind
|
||||
|
|
@ -43,7 +43,7 @@ class StreamTcpDocSpec extends AkkaSpec {
|
|||
|
||||
val connections: Source[IncomingConnection, Future[ServerBinding]] =
|
||||
Tcp().bind(host, port)
|
||||
connections runForeach { connection =>
|
||||
connections runForeach { connection ⇒
|
||||
println(s"New connection from: ${connection.remoteAddress}")
|
||||
|
||||
val echo = Flow[ByteString]
|
||||
|
|
@ -69,7 +69,7 @@ class StreamTcpDocSpec extends AkkaSpec {
|
|||
import akka.stream.scaladsl.Framing
|
||||
//#welcome-banner-chat-server
|
||||
|
||||
connections.runForeach { connection =>
|
||||
connections.runForeach { connection ⇒
|
||||
|
||||
// server logic, parses incoming commands
|
||||
val commandParser = Flow[String].takeWhile(_ != "BYE").map(_ + "!")
|
||||
|
|
@ -85,7 +85,7 @@ class StreamTcpDocSpec extends AkkaSpec {
|
|||
allowTruncation = true))
|
||||
.map(_.utf8String)
|
||||
//#welcome-banner-chat-server
|
||||
.map { command => serverProbe.ref ! command; command }
|
||||
.map { command ⇒ serverProbe.ref ! command; command }
|
||||
//#welcome-banner-chat-server
|
||||
.via(commandParser)
|
||||
// merge in the initial banner after parser
|
||||
|
|
@ -102,8 +102,8 @@ class StreamTcpDocSpec extends AkkaSpec {
|
|||
val input = new AtomicReference("Hello world" :: "What a lovely day" :: Nil)
|
||||
def readLine(prompt: String): String = {
|
||||
input.get() match {
|
||||
case all @ cmd :: tail if input.compareAndSet(all, tail) => cmd
|
||||
case _ => "q"
|
||||
case all @ cmd :: tail if input.compareAndSet(all, tail) ⇒ cmd
|
||||
case _ ⇒ "q"
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -120,7 +120,7 @@ class StreamTcpDocSpec extends AkkaSpec {
|
|||
val replParser =
|
||||
Flow[String].takeWhile(_ != "q")
|
||||
.concat(Source.single("BYE"))
|
||||
.map(elem => ByteString(s"$elem\n"))
|
||||
.map(elem ⇒ ByteString(s"$elem\n"))
|
||||
|
||||
val repl = Flow[ByteString]
|
||||
.via(Framing.delimiter(
|
||||
|
|
@ -128,8 +128,8 @@ class StreamTcpDocSpec extends AkkaSpec {
|
|||
maximumFrameLength = 256,
|
||||
allowTruncation = true))
|
||||
.map(_.utf8String)
|
||||
.map(text => println("Server: " + text))
|
||||
.map(_ => readLine("> "))
|
||||
.map(text ⇒ println("Server: " + text))
|
||||
.map(_ ⇒ readLine("> "))
|
||||
.via(replParser)
|
||||
|
||||
connection.join(repl).run()
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue