+doc #19429 initial merge of docs-dev and docs
This commit is contained in:
parent
be0c8af4c0
commit
5a18d43435
501 changed files with 9876 additions and 3681 deletions
|
|
@ -1,100 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.scaladsl.{ Flow, Sink, Source }
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class RecipeByteStrings extends RecipeSpec {
|
||||
|
||||
"Recipes for bytestring streams" must {
|
||||
|
||||
"have a working chunker" in {
|
||||
val rawBytes = Source(List(ByteString(1, 2), ByteString(3), ByteString(4, 5, 6), ByteString(7, 8, 9)))
|
||||
val ChunkLimit = 2
|
||||
|
||||
//#bytestring-chunker
|
||||
import akka.stream.stage._
|
||||
|
||||
class Chunker(val chunkSize: Int) extends PushPullStage[ByteString, ByteString] {
|
||||
private var buffer = ByteString.empty
|
||||
|
||||
override def onPush(elem: ByteString, ctx: Context[ByteString]): SyncDirective = {
|
||||
buffer ++= elem
|
||||
emitChunkOrPull(ctx)
|
||||
}
|
||||
|
||||
override def onPull(ctx: Context[ByteString]): SyncDirective = emitChunkOrPull(ctx)
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[ByteString]): TerminationDirective =
|
||||
if (buffer.nonEmpty) ctx.absorbTermination()
|
||||
else ctx.finish()
|
||||
|
||||
private def emitChunkOrPull(ctx: Context[ByteString]): SyncDirective = {
|
||||
if (buffer.isEmpty) {
|
||||
if (ctx.isFinishing) ctx.finish()
|
||||
else ctx.pull()
|
||||
} else {
|
||||
val (emit, nextBuffer) = buffer.splitAt(chunkSize)
|
||||
buffer = nextBuffer
|
||||
ctx.push(emit)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
val chunksStream = rawBytes.transform(() => new Chunker(ChunkLimit))
|
||||
//#bytestring-chunker
|
||||
|
||||
val chunksFuture = chunksStream.grouped(10).runWith(Sink.head)
|
||||
|
||||
val chunks = Await.result(chunksFuture, 3.seconds)
|
||||
|
||||
chunks.forall(_.size <= 2) should be(true)
|
||||
chunks.fold(ByteString())(_ ++ _) should be(ByteString(1, 2, 3, 4, 5, 6, 7, 8, 9))
|
||||
}
|
||||
|
||||
"have a working bytes limiter" in {
|
||||
val SizeLimit = 9
|
||||
|
||||
//#bytes-limiter
|
||||
import akka.stream.stage._
|
||||
class ByteLimiter(val maximumBytes: Long) extends PushStage[ByteString, ByteString] {
|
||||
private var count = 0
|
||||
|
||||
override def onPush(chunk: ByteString, ctx: Context[ByteString]): SyncDirective = {
|
||||
count += chunk.size
|
||||
if (count > maximumBytes) ctx.fail(new IllegalStateException("Too much bytes"))
|
||||
else ctx.push(chunk)
|
||||
}
|
||||
}
|
||||
|
||||
val limiter = Flow[ByteString].transform(() => new ByteLimiter(SizeLimit))
|
||||
//#bytes-limiter
|
||||
|
||||
val bytes1 = Source(List(ByteString(1, 2), ByteString(3), ByteString(4, 5, 6), ByteString(7, 8, 9)))
|
||||
val bytes2 = Source(List(ByteString(1, 2), ByteString(3), ByteString(4, 5, 6), ByteString(7, 8, 9, 10)))
|
||||
|
||||
Await.result(bytes1.via(limiter).grouped(10).runWith(Sink.head), 3.seconds)
|
||||
.fold(ByteString())(_ ++ _) should be(ByteString(1, 2, 3, 4, 5, 6, 7, 8, 9))
|
||||
|
||||
an[IllegalStateException] must be thrownBy {
|
||||
Await.result(bytes2.via(limiter).grouped(10).runWith(Sink.head), 3.seconds)
|
||||
}
|
||||
}
|
||||
|
||||
"demonstrate compacting" in {
|
||||
|
||||
val data = Source(List(ByteString(1, 2), ByteString(3), ByteString(4, 5, 6), ByteString(7, 8, 9)))
|
||||
|
||||
//#compacting-bytestrings
|
||||
val compacted: Source[ByteString, Unit] = data.map(_.compact)
|
||||
//#compacting-bytestrings
|
||||
|
||||
Await.result(compacted.grouped(10).runWith(Sink.head), 3.seconds).forall(_.isCompact) should be(true)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,90 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.{ ActorMaterializerSettings, ActorMaterializer }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class RecipeCollectingMetrics extends RecipeSpec {
|
||||
import HoldOps._
|
||||
implicit val m2 = ActorMaterializer(ActorMaterializerSettings(system).withInputBuffer(1, 1))
|
||||
|
||||
"Recipe for periodically collecting metrics" must {
|
||||
|
||||
"work" in {
|
||||
// type Tick = Unit
|
||||
//
|
||||
// val loadPub = TestPublisher.manualProbe[Int]()
|
||||
// val tickPub = TestPublisher.manualProbe[Tick]()
|
||||
// val reportTicks = Source.fromPublisher(tickPub)
|
||||
// val loadUpdates = Source.fromPublisher(loadPub)
|
||||
// val futureSink = Sink.head[immutable.Seq[String]]
|
||||
// val sink = Flow[String].grouped(10).to(futureSink)
|
||||
//
|
||||
// //#periodic-metrics-collection
|
||||
// val currentLoad = loadUpdates.transform(() => new HoldWithWait)
|
||||
//
|
||||
// val graph = GraphDSL { implicit builder =>
|
||||
// import FlowGraphImplicits._
|
||||
// val collector = ZipWith[Int, Tick, String](
|
||||
// (load: Int, tick: Tick) => s"current load is $load")
|
||||
//
|
||||
// currentLoad ~> collector.left
|
||||
// reportTicks ~> collector.right
|
||||
//
|
||||
// collector.out ~> sink
|
||||
// }
|
||||
// //#periodic-metrics-collection
|
||||
//
|
||||
// val reports = graph.run().get(futureSink)
|
||||
// val manualLoad = new StreamTestKit.AutoPublisher(loadPub)
|
||||
// val manualTick = new StreamTestKit.AutoPublisher(tickPub)
|
||||
//
|
||||
// // Prefetch elimination
|
||||
// manualTick.sendNext(())
|
||||
//
|
||||
// manualLoad.sendNext(53)
|
||||
// manualLoad.sendNext(61)
|
||||
// manualTick.sendNext(())
|
||||
//
|
||||
// manualLoad.sendNext(44)
|
||||
// manualLoad.sendNext(54)
|
||||
// manualLoad.sendNext(78)
|
||||
// Thread.sleep(500)
|
||||
//
|
||||
// manualTick.sendNext(())
|
||||
//
|
||||
// manualTick.sendComplete()
|
||||
//
|
||||
// Await.result(reports, 3.seconds) should be(List("current load is 53", "current load is 61", "current load is 78"))
|
||||
|
||||
// Periodically collect values of metrics expressed as stream of updates
|
||||
// ---------------------------------------------------------------------
|
||||
//
|
||||
// **Situation:** Given performance counters expressed as a stream of updates we want to gather a periodic report of these.
|
||||
// We do not want to backpressure the counter updates but always take the last value instead. Whenever we don't have a new counter
|
||||
// value we want to repeat the last value.
|
||||
//
|
||||
// This recipe uses the :class:`HoldWithWait` recipe introduced previously. We use this element to gather updates from
|
||||
// the counter stream and store the final value, and also repeat this final value if no update is received between
|
||||
// metrics collection rounds.
|
||||
//
|
||||
// To finish the recipe, we simply use :class:`ZipWith` to trigger reading the latest value from the ``currentLoad``
|
||||
// stream whenever a new ``Tick`` arrives on the stream of ticks, ``reportTicks``.
|
||||
//
|
||||
// .. includecode:: code/docs/stream/cookbook/RecipeCollectingMetrics.scala#periodic-metrics-collection
|
||||
//
|
||||
// .. warning::
|
||||
// In order for this recipe to work the buffer size for the :class:`ZipWith` must be set to 1. The reason for this is
|
||||
// explained in the "Buffering" section of the documentation.
|
||||
|
||||
// FIXME: This recipe does only work with buffer size of 0, which is only available if graph fusing is implemented
|
||||
pending
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,62 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import java.security.MessageDigest
|
||||
|
||||
import akka.stream.scaladsl.{ Sink, Source }
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class RecipeDigest extends RecipeSpec {
|
||||
|
||||
"Recipe for calculating digest" must {
|
||||
|
||||
"work" in {
|
||||
|
||||
val data = Source(List(
|
||||
ByteString("abcdbcdecdef"),
|
||||
ByteString("defgefghfghighijhijkijkljklmklmnlmnomnopnopq")))
|
||||
|
||||
//#calculating-digest
|
||||
import akka.stream.stage._
|
||||
def digestCalculator(algorithm: String) = new PushPullStage[ByteString, ByteString] {
|
||||
val digest = MessageDigest.getInstance(algorithm)
|
||||
|
||||
override def onPush(chunk: ByteString, ctx: Context[ByteString]): SyncDirective = {
|
||||
digest.update(chunk.toArray)
|
||||
ctx.pull()
|
||||
}
|
||||
|
||||
override def onPull(ctx: Context[ByteString]): SyncDirective = {
|
||||
if (ctx.isFinishing) ctx.pushAndFinish(ByteString(digest.digest()))
|
||||
else ctx.pull()
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[ByteString]): TerminationDirective = {
|
||||
// If the stream is finished, we need to emit the last element in the onPull block.
|
||||
// It is not allowed to directly emit elements from a termination block
|
||||
// (onUpstreamFinish or onUpstreamFailure)
|
||||
ctx.absorbTermination()
|
||||
}
|
||||
}
|
||||
|
||||
val digest: Source[ByteString, Unit] = data.transform(() => digestCalculator("SHA-256"))
|
||||
//#calculating-digest
|
||||
|
||||
Await.result(digest.runWith(Sink.head), 3.seconds) should be(
|
||||
ByteString(
|
||||
0x24, 0x8d, 0x6a, 0x61,
|
||||
0xd2, 0x06, 0x38, 0xb8,
|
||||
0xe5, 0xc0, 0x26, 0x93,
|
||||
0x0c, 0x3e, 0x60, 0x39,
|
||||
0xa3, 0x3c, 0xe4, 0x59,
|
||||
0x64, 0xff, 0x21, 0x67,
|
||||
0xf6, 0xec, 0xed, 0xd4,
|
||||
0x19, 0xdb, 0x06, 0xc1))
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,65 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.{ ClosedShape, OverflowStrategy }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class RecipeDroppyBroadcast extends RecipeSpec {
|
||||
|
||||
"Recipe for a droppy broadcast" must {
|
||||
"work" in {
|
||||
val pub = TestPublisher.probe[Int]()
|
||||
val myElements = Source.fromPublisher(pub)
|
||||
|
||||
val sub1 = TestSubscriber.manualProbe[Int]()
|
||||
val sub2 = TestSubscriber.manualProbe[Int]()
|
||||
val sub3 = TestSubscriber.probe[Int]()
|
||||
val futureSink = Sink.head[Seq[Int]]
|
||||
val mySink1 = Sink.fromSubscriber(sub1)
|
||||
val mySink2 = Sink.fromSubscriber(sub2)
|
||||
val mySink3 = Sink.fromSubscriber(sub3)
|
||||
|
||||
//#droppy-bcast
|
||||
val graph = RunnableGraph.fromGraph(GraphDSL.create(mySink1, mySink2, mySink3)((_, _, _)) { implicit b =>
|
||||
(sink1, sink2, sink3) =>
|
||||
import GraphDSL.Implicits._
|
||||
|
||||
val bcast = b.add(Broadcast[Int](3))
|
||||
myElements ~> bcast
|
||||
|
||||
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink1
|
||||
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink2
|
||||
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink3
|
||||
ClosedShape
|
||||
})
|
||||
//#droppy-bcast
|
||||
|
||||
graph.run()
|
||||
|
||||
sub3.request(100)
|
||||
for (i <- 1 to 100) {
|
||||
pub.sendNext(i)
|
||||
sub3.expectNext(i)
|
||||
}
|
||||
|
||||
pub.sendComplete()
|
||||
|
||||
sub1.expectSubscription().request(10)
|
||||
sub2.expectSubscription().request(10)
|
||||
|
||||
for (i <- 91 to 100) {
|
||||
sub1.expectNext(i)
|
||||
sub2.expectNext(i)
|
||||
}
|
||||
|
||||
sub1.expectComplete()
|
||||
sub2.expectComplete()
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,28 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.scaladsl.{ Sink, Source }
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class RecipeFlattenSeq extends RecipeSpec {
|
||||
|
||||
"Recipe for flatteing a stream of seqs" must {
|
||||
|
||||
"work" in {
|
||||
|
||||
val someDataSource = Source(List(List("1"), List("2"), List("3", "4", "5"), List("6", "7")))
|
||||
|
||||
//#flattening-seqs
|
||||
val myData: Source[List[Message], Unit] = someDataSource
|
||||
val flattened: Source[Message, Unit] = myData.mapConcat(identity)
|
||||
//#flattening-seqs
|
||||
|
||||
Await.result(flattened.grouped(8).runWith(Sink.head), 3.seconds) should be(List("1", "2", "3", "4", "5", "6", "7"))
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,135 +0,0 @@
|
|||
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._
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class RecipeGlobalRateLimit extends RecipeSpec {
|
||||
|
||||
"Global rate limiting recipe" must {
|
||||
|
||||
//#global-limiter-actor
|
||||
object Limiter {
|
||||
case object WantToPass
|
||||
case object MayPass
|
||||
|
||||
case object ReplenishTokens
|
||||
|
||||
def props(maxAvailableTokens: Int, tokenRefreshPeriod: FiniteDuration,
|
||||
tokenRefreshAmount: Int): Props =
|
||||
Props(new Limiter(maxAvailableTokens, tokenRefreshPeriod, tokenRefreshAmount))
|
||||
}
|
||||
|
||||
class Limiter(
|
||||
val maxAvailableTokens: Int,
|
||||
val tokenRefreshPeriod: FiniteDuration,
|
||||
val tokenRefreshAmount: Int) extends Actor {
|
||||
import Limiter._
|
||||
import context.dispatcher
|
||||
import akka.actor.Status
|
||||
|
||||
private var waitQueue = immutable.Queue.empty[ActorRef]
|
||||
private var permitTokens = maxAvailableTokens
|
||||
private val replenishTimer = system.scheduler.schedule(
|
||||
initialDelay = tokenRefreshPeriod,
|
||||
interval = tokenRefreshPeriod,
|
||||
receiver = self,
|
||||
ReplenishTokens)
|
||||
|
||||
override def receive: Receive = open
|
||||
|
||||
val open: Receive = {
|
||||
case ReplenishTokens =>
|
||||
permitTokens = math.min(permitTokens + tokenRefreshAmount, maxAvailableTokens)
|
||||
case WantToPass =>
|
||||
permitTokens -= 1
|
||||
sender() ! MayPass
|
||||
if (permitTokens == 0) context.become(closed)
|
||||
}
|
||||
|
||||
val closed: Receive = {
|
||||
case ReplenishTokens =>
|
||||
permitTokens = math.min(permitTokens + tokenRefreshAmount, maxAvailableTokens)
|
||||
releaseWaiting()
|
||||
case WantToPass =>
|
||||
waitQueue = waitQueue.enqueue(sender())
|
||||
}
|
||||
|
||||
private def releaseWaiting(): Unit = {
|
||||
val (toBeReleased, remainingQueue) = waitQueue.splitAt(permitTokens)
|
||||
waitQueue = remainingQueue
|
||||
permitTokens -= toBeReleased.size
|
||||
toBeReleased foreach (_ ! MayPass)
|
||||
if (permitTokens > 0) context.become(open)
|
||||
}
|
||||
|
||||
override def postStop(): Unit = {
|
||||
replenishTimer.cancel()
|
||||
waitQueue foreach (_ ! Status.Failure(new IllegalStateException("limiter stopped")))
|
||||
}
|
||||
}
|
||||
//#global-limiter-actor
|
||||
|
||||
"work" in {
|
||||
|
||||
//#global-limiter-flow
|
||||
def limitGlobal[T](limiter: ActorRef, maxAllowedWait: FiniteDuration): Flow[T, T, Unit] = {
|
||||
import akka.pattern.ask
|
||||
import akka.util.Timeout
|
||||
Flow[T].mapAsync(4)((element: T) => {
|
||||
import system.dispatcher
|
||||
implicit val triggerTimeout = Timeout(maxAllowedWait)
|
||||
val limiterTriggerFuture = limiter ? Limiter.WantToPass
|
||||
limiterTriggerFuture.map((_) => element)
|
||||
})
|
||||
|
||||
}
|
||||
//#global-limiter-flow
|
||||
|
||||
// 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))
|
||||
val source2 = Source.fromIterator(() => Iterator.continually("E2")).via(limitGlobal(limiter, 2.seconds))
|
||||
|
||||
val probe = TestSubscriber.manualProbe[String]()
|
||||
|
||||
RunnableGraph.fromGraph(GraphDSL.create() { implicit b =>
|
||||
import GraphDSL.Implicits._
|
||||
val merge = b.add(Merge[String](2))
|
||||
source1 ~> merge ~> Sink.fromSubscriber(probe)
|
||||
source2 ~> merge
|
||||
ClosedShape
|
||||
}).run()
|
||||
|
||||
probe.expectSubscription().request(1000)
|
||||
|
||||
probe.expectNext() should startWith("E")
|
||||
probe.expectNext() should startWith("E")
|
||||
probe.expectNoMsg(500.millis)
|
||||
|
||||
limiter ! Limiter.ReplenishTokens
|
||||
probe.expectNext() should startWith("E")
|
||||
probe.expectNoMsg(500.millis)
|
||||
|
||||
var resultSet = Set.empty[String]
|
||||
for (_ <- 1 to 100) {
|
||||
limiter ! Limiter.ReplenishTokens
|
||||
resultSet += probe.expectNext()
|
||||
}
|
||||
|
||||
resultSet.contains("E1") should be(true)
|
||||
resultSet.contains("E2") should be(true)
|
||||
|
||||
probe.expectError()
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,115 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.scaladsl.{ Sink, Source }
|
||||
import akka.stream.testkit._
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
object HoldOps {
|
||||
//#hold-version-1
|
||||
import akka.stream.stage._
|
||||
class HoldWithInitial[T](initial: T) extends DetachedStage[T, T] {
|
||||
private var currentValue: T = initial
|
||||
|
||||
override def onPush(elem: T, ctx: DetachedContext[T]): UpstreamDirective = {
|
||||
currentValue = elem
|
||||
ctx.pull()
|
||||
}
|
||||
|
||||
override def onPull(ctx: DetachedContext[T]): DownstreamDirective = {
|
||||
ctx.push(currentValue)
|
||||
}
|
||||
|
||||
}
|
||||
//#hold-version-1
|
||||
|
||||
//#hold-version-2
|
||||
import akka.stream.stage._
|
||||
class HoldWithWait[T] extends DetachedStage[T, T] {
|
||||
private var currentValue: T = _
|
||||
private var waitingFirstValue = true
|
||||
|
||||
override def onPush(elem: T, ctx: DetachedContext[T]): UpstreamDirective = {
|
||||
currentValue = elem
|
||||
waitingFirstValue = false
|
||||
if (ctx.isHoldingDownstream) ctx.pushAndPull(currentValue)
|
||||
else ctx.pull()
|
||||
}
|
||||
|
||||
override def onPull(ctx: DetachedContext[T]): DownstreamDirective = {
|
||||
if (waitingFirstValue) ctx.holdDownstream()
|
||||
else ctx.push(currentValue)
|
||||
}
|
||||
|
||||
}
|
||||
//#hold-version-2
|
||||
}
|
||||
|
||||
class RecipeHold extends RecipeSpec {
|
||||
import HoldOps._
|
||||
|
||||
"Recipe for creating a holding element" must {
|
||||
|
||||
"work for version 1" in {
|
||||
|
||||
val pub = TestPublisher.probe[Int]()
|
||||
val sub = TestSubscriber.manualProbe[Int]()
|
||||
val source = Source.fromPublisher(pub)
|
||||
val sink = Sink.fromSubscriber(sub)
|
||||
|
||||
source.transform(() => new HoldWithInitial(0)).to(sink).run()
|
||||
|
||||
val subscription = sub.expectSubscription()
|
||||
sub.expectNoMsg(100.millis)
|
||||
|
||||
subscription.request(1)
|
||||
sub.expectNext(0)
|
||||
|
||||
subscription.request(1)
|
||||
sub.expectNext(0)
|
||||
|
||||
pub.sendNext(1)
|
||||
pub.sendNext(2)
|
||||
|
||||
subscription.request(2)
|
||||
sub.expectNext(2)
|
||||
sub.expectNext(2)
|
||||
|
||||
pub.sendComplete()
|
||||
subscription.request(1)
|
||||
sub.expectComplete()
|
||||
}
|
||||
|
||||
"work for version 2" in {
|
||||
|
||||
val pub = TestPublisher.probe[Int]()
|
||||
val sub = TestSubscriber.manualProbe[Int]()
|
||||
val source = Source.fromPublisher(pub)
|
||||
val sink = Sink.fromSubscriber(sub)
|
||||
|
||||
source.transform(() => new HoldWithWait).to(sink).run()
|
||||
|
||||
val subscription = sub.expectSubscription()
|
||||
sub.expectNoMsg(100.millis)
|
||||
|
||||
subscription.request(1)
|
||||
sub.expectNoMsg(100.millis)
|
||||
|
||||
pub.sendNext(1)
|
||||
sub.expectNext(1)
|
||||
|
||||
pub.sendNext(2)
|
||||
pub.sendNext(3)
|
||||
|
||||
subscription.request(2)
|
||||
sub.expectNext(3)
|
||||
sub.expectNext(3)
|
||||
|
||||
pub.sendComplete()
|
||||
subscription.request(1)
|
||||
sub.expectComplete()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,25 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.ClosedShape
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
import akka.util.ByteString
|
||||
|
||||
class RecipeKeepAlive extends RecipeSpec {
|
||||
|
||||
"Recipe for injecting keepalive messages" must {
|
||||
|
||||
"work" in {
|
||||
val keepaliveMessage = ByteString(11)
|
||||
|
||||
//#inject-keepalive
|
||||
import scala.concurrent.duration._
|
||||
val injectKeepAlive: Flow[ByteString, ByteString, Unit] =
|
||||
Flow[ByteString].keepAlive(1.second, () => keepaliveMessage)
|
||||
//#inject-keepalive
|
||||
|
||||
// No need to test, this is a built-in stage with proper tests
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,50 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.event.Logging
|
||||
import akka.stream.Attributes
|
||||
import akka.stream.scaladsl.{ Sink, Source }
|
||||
import akka.testkit.{ EventFilter, TestProbe }
|
||||
|
||||
class RecipeLoggingElements extends RecipeSpec {
|
||||
|
||||
"Simple logging recipe" must {
|
||||
|
||||
"work with println" in {
|
||||
val printProbe = TestProbe()
|
||||
def println(s: String): Unit = printProbe.ref ! s
|
||||
|
||||
val mySource = Source(List("1", "2", "3"))
|
||||
|
||||
//#println-debug
|
||||
val loggedSource = mySource.map { elem => println(elem); elem }
|
||||
//#println-debug
|
||||
|
||||
loggedSource.runWith(Sink.ignore)
|
||||
printProbe.expectMsgAllOf("1", "2", "3")
|
||||
}
|
||||
|
||||
"use log()" in {
|
||||
val mySource = Source(List("1", "2", "3"))
|
||||
def analyse(s: String) = s
|
||||
|
||||
//#log-custom
|
||||
// customise log levels
|
||||
mySource.log("before-map")
|
||||
.withAttributes(Attributes.logLevels(onElement = Logging.WarningLevel))
|
||||
.map(analyse)
|
||||
|
||||
// or provide custom logging adapter
|
||||
implicit val adapter = Logging(system, "customLogger")
|
||||
mySource.log("custom")
|
||||
//#log-custom
|
||||
|
||||
val loggedSource = mySource.log("custom")
|
||||
EventFilter.debug(start = "[custom] Element: ").intercept {
|
||||
loggedSource.runWith(Sink.ignore)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,93 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.ClosedShape
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class RecipeManualTrigger extends RecipeSpec {
|
||||
|
||||
"Recipe for triggering a stream manually" must {
|
||||
|
||||
"work" in {
|
||||
|
||||
val elements = Source(List("1", "2", "3", "4"))
|
||||
val pub = TestPublisher.probe[Trigger]()
|
||||
val sub = TestSubscriber.manualProbe[Message]()
|
||||
val triggerSource = Source.fromPublisher(pub)
|
||||
val sink = Sink.fromSubscriber(sub)
|
||||
|
||||
//#manually-triggered-stream
|
||||
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
|
||||
ClosedShape
|
||||
})
|
||||
//#manually-triggered-stream
|
||||
|
||||
graph.run()
|
||||
|
||||
sub.expectSubscription().request(1000)
|
||||
sub.expectNoMsg(100.millis)
|
||||
|
||||
pub.sendNext(())
|
||||
sub.expectNext("1")
|
||||
sub.expectNoMsg(100.millis)
|
||||
|
||||
pub.sendNext(())
|
||||
pub.sendNext(())
|
||||
sub.expectNext("2")
|
||||
sub.expectNext("3")
|
||||
sub.expectNoMsg(100.millis)
|
||||
|
||||
pub.sendNext(())
|
||||
sub.expectNext("4")
|
||||
sub.expectComplete()
|
||||
}
|
||||
|
||||
"work with ZipWith" in {
|
||||
|
||||
val elements = Source(List("1", "2", "3", "4"))
|
||||
val pub = TestPublisher.probe[Trigger]()
|
||||
val sub = TestSubscriber.manualProbe[Message]()
|
||||
val triggerSource = Source.fromPublisher(pub)
|
||||
val sink = Sink.fromSubscriber(sub)
|
||||
|
||||
//#manually-triggered-stream-zipwith
|
||||
val graph = RunnableGraph.fromGraph(GraphDSL.create() { implicit builder =>
|
||||
import GraphDSL.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()
|
||||
|
||||
sub.expectSubscription().request(1000)
|
||||
sub.expectNoMsg(100.millis)
|
||||
|
||||
pub.sendNext(())
|
||||
sub.expectNext("1")
|
||||
sub.expectNoMsg(100.millis)
|
||||
|
||||
pub.sendNext(())
|
||||
pub.sendNext(())
|
||||
sub.expectNext("2")
|
||||
sub.expectNext("3")
|
||||
sub.expectNoMsg(100.millis)
|
||||
|
||||
pub.sendNext(())
|
||||
sub.expectNext("4")
|
||||
sub.expectComplete()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,57 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit._
|
||||
import scala.concurrent.duration._
|
||||
import akka.testkit.TestLatch
|
||||
import scala.concurrent.Await
|
||||
|
||||
class RecipeMissedTicks extends RecipeSpec {
|
||||
|
||||
"Recipe for collecting missed ticks" must {
|
||||
|
||||
"work" in {
|
||||
type Tick = Unit
|
||||
|
||||
val pub = TestPublisher.probe[Tick]()
|
||||
val sub = TestSubscriber.manualProbe[Int]()
|
||||
val tickStream = Source.fromPublisher(pub)
|
||||
val sink = Sink.fromSubscriber(sub)
|
||||
|
||||
//#missed-ticks
|
||||
val missedTicks: Flow[Tick, Int, Unit] =
|
||||
Flow[Tick].conflate(seed = (_) => 0)(
|
||||
(missedTicks, tick) => missedTicks + 1)
|
||||
//#missed-ticks
|
||||
val latch = TestLatch(3)
|
||||
val realMissedTicks: Flow[Tick, Int, Unit] =
|
||||
Flow[Tick].conflate(seed = (_) => 0)(
|
||||
(missedTicks, tick) => { latch.countDown(); missedTicks + 1 })
|
||||
|
||||
tickStream.via(realMissedTicks).to(sink).run()
|
||||
|
||||
pub.sendNext(())
|
||||
pub.sendNext(())
|
||||
pub.sendNext(())
|
||||
pub.sendNext(())
|
||||
|
||||
val subscription = sub.expectSubscription()
|
||||
Await.ready(latch, 1.second)
|
||||
|
||||
subscription.request(1)
|
||||
sub.expectNext(3)
|
||||
|
||||
subscription.request(1)
|
||||
sub.expectNoMsg(100.millis)
|
||||
|
||||
pub.sendNext(())
|
||||
sub.expectNext(0)
|
||||
|
||||
pub.sendComplete()
|
||||
subscription.request(1)
|
||||
sub.expectComplete()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,58 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.scaladsl.{ Sink, Source }
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class RecipeMultiGroupBy extends RecipeSpec {
|
||||
|
||||
"Recipe for multi-groupBy" must {
|
||||
|
||||
"work" in {
|
||||
|
||||
case class Topic(name: String)
|
||||
|
||||
val elems = Source(List("1: a", "1: b", "all: c", "all: d", "1: e"))
|
||||
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 messageAndTopic: Source[(Message, Topic), Unit] = elems.mapConcat { msg: Message =>
|
||||
val topicsForMessage = topicMapper(msg)
|
||||
// Create a (Msg, Topic) pair for each of the topics
|
||||
// the message belongs to
|
||||
topicsForMessage.map(msg -> _)
|
||||
}
|
||||
|
||||
val multiGroups = messageAndTopic
|
||||
.groupBy(2, _._2).map {
|
||||
case (msg, topic) =>
|
||||
// do what needs to be done
|
||||
//#multi-groupby
|
||||
(msg, topic)
|
||||
//#multi-groupby
|
||||
}
|
||||
//#multi-groupby
|
||||
|
||||
val result = multiGroups
|
||||
.grouped(10)
|
||||
.mergeSubstreams
|
||||
.map(g => g.head._2.name + g.map(_._1).mkString("[", ", ", "]"))
|
||||
.grouped(10)
|
||||
.runWith(Sink.head)
|
||||
|
||||
Await.result(result, 3.seconds).toSet should be(Set(
|
||||
"1[1: a, 1: b, all: c, all: d, 1: e]",
|
||||
"2[all: c, all: d]"))
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,39 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.scaladsl.Sink
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class RecipeParseLines extends RecipeSpec {
|
||||
|
||||
"Recipe for parsing line from bytes" must {
|
||||
|
||||
"work" in {
|
||||
val rawData = Source(List(
|
||||
ByteString("Hello World"),
|
||||
ByteString("\r"),
|
||||
ByteString("!\r"),
|
||||
ByteString("\nHello Akka!\r\nHello Streams!"),
|
||||
ByteString("\r\n\r\n")))
|
||||
|
||||
//#parse-lines
|
||||
import akka.stream.io.Framing
|
||||
val linesStream = rawData.via(Framing.delimiter(
|
||||
ByteString("\r\n"), maximumFrameLength = 100, allowTruncation = true))
|
||||
.map(_.utf8String)
|
||||
//#parse-lines
|
||||
|
||||
Await.result(linesStream.grouped(10).runWith(Sink.head), 3.seconds) should be(List(
|
||||
"Hello World\r!",
|
||||
"Hello Akka!",
|
||||
"Hello Streams!",
|
||||
""))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,81 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.{ Graph, FlowShape, Inlet, Outlet, Attributes, OverflowStrategy }
|
||||
import akka.stream.scaladsl._
|
||||
import scala.concurrent.{ Await, Future }
|
||||
import scala.concurrent.duration._
|
||||
import akka.stream.stage.{ GraphStage, GraphStageLogic }
|
||||
|
||||
class RecipeReduceByKey extends RecipeSpec {
|
||||
|
||||
"Reduce by key recipe" must {
|
||||
|
||||
val MaximumDistinctWords = 1000
|
||||
|
||||
"work with simple word count" in {
|
||||
|
||||
def words = Source(List("hello", "world", "and", "hello", "universe", "akka") ++ List.fill(1000)("rocks!"))
|
||||
|
||||
//#word-count
|
||||
val counts: Source[(String, Int), Unit] = words
|
||||
// split the words into separate streams first
|
||||
.groupBy(MaximumDistinctWords, identity)
|
||||
// add counting logic to the streams
|
||||
.fold(("", 0)) {
|
||||
case ((_, count), word) => (word, count + 1)
|
||||
}
|
||||
// get a stream of word counts
|
||||
.mergeSubstreams
|
||||
//#word-count
|
||||
|
||||
Await.result(counts.grouped(10).runWith(Sink.head), 3.seconds).toSet should be(Set(
|
||||
("hello", 2),
|
||||
("world", 1),
|
||||
("and", 1),
|
||||
("universe", 1),
|
||||
("akka", 1),
|
||||
("rocks!", 1000)))
|
||||
}
|
||||
|
||||
"work generalized" in {
|
||||
|
||||
def words = Source(List("hello", "world", "and", "hello", "universe", "akka") ++ List.fill(1000)("rocks!"))
|
||||
|
||||
//#reduce-by-key-general
|
||||
def reduceByKey[In, K, Out](
|
||||
maximumGroupSize: Int,
|
||||
groupKey: (In) => K,
|
||||
foldZero: (K) => Out)(fold: (Out, In) => Out): Flow[In, (K, Out), Unit] = {
|
||||
|
||||
Flow[In]
|
||||
.groupBy(maximumGroupSize, groupKey)
|
||||
.fold(Option.empty[(K, Out)]) {
|
||||
case (None, elem) =>
|
||||
val key = groupKey(elem)
|
||||
Some((key, fold(foldZero(key), elem)))
|
||||
case (Some((key, out)), elem) =>
|
||||
Some((key, fold(out, elem)))
|
||||
}
|
||||
.map(_.get)
|
||||
.mergeSubstreams
|
||||
}
|
||||
|
||||
val wordCounts = words.via(reduceByKey(
|
||||
MaximumDistinctWords,
|
||||
groupKey = (word: String) => word,
|
||||
foldZero = (key: String) => 0)(fold = (count: Int, elem: String) => count + 1))
|
||||
|
||||
//#reduce-by-key-general
|
||||
|
||||
Await.result(wordCounts.grouped(10).runWith(Sink.head), 3.seconds).toSet should be(Set(
|
||||
("hello", 2),
|
||||
("world", 1),
|
||||
("and", 1),
|
||||
("universe", 1),
|
||||
("akka", 1),
|
||||
("rocks!", 1000)))
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,49 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.scaladsl.{ Flow, Sink, Source }
|
||||
import akka.stream.testkit._
|
||||
import scala.concurrent.duration._
|
||||
import akka.testkit.TestLatch
|
||||
import scala.concurrent.Await
|
||||
|
||||
class RecipeSimpleDrop extends RecipeSpec {
|
||||
|
||||
"Recipe for simply dropping elements for a faster stream" must {
|
||||
|
||||
"work" in {
|
||||
|
||||
//#simple-drop
|
||||
val droppyStream: Flow[Message, Message, Unit] =
|
||||
Flow[Message].conflate(seed = identity)((lastMessage, newMessage) => newMessage)
|
||||
//#simple-drop
|
||||
val latch = TestLatch(2)
|
||||
val realDroppyStream =
|
||||
Flow[Message].conflate(seed = identity)((lastMessage, newMessage) => { latch.countDown(); newMessage })
|
||||
|
||||
val pub = TestPublisher.probe[Message]()
|
||||
val sub = TestSubscriber.manualProbe[Message]()
|
||||
val messageSource = Source.fromPublisher(pub)
|
||||
val sink = Sink.fromSubscriber(sub)
|
||||
|
||||
messageSource.via(realDroppyStream).to(sink).run()
|
||||
|
||||
val subscription = sub.expectSubscription()
|
||||
sub.expectNoMsg(100.millis)
|
||||
|
||||
pub.sendNext("1")
|
||||
pub.sendNext("2")
|
||||
pub.sendNext("3")
|
||||
|
||||
Await.ready(latch, 1.second)
|
||||
|
||||
subscription.request(1)
|
||||
sub.expectNext("3")
|
||||
|
||||
pub.sendComplete()
|
||||
subscription.request(1)
|
||||
sub.expectComplete()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,13 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
|
||||
trait RecipeSpec extends AkkaSpec {
|
||||
|
||||
implicit val m = ActorMaterializer()
|
||||
type Message = String
|
||||
type Trigger = Unit
|
||||
type Job = String
|
||||
|
||||
}
|
||||
|
|
@ -1,27 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.scaladsl.{ Sink, Source }
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.{ Await, Future }
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class RecipeToStrict extends RecipeSpec {
|
||||
|
||||
"Recipe for draining a stream into a strict collection" must {
|
||||
|
||||
"work" in {
|
||||
val myData = Source(List("1", "2", "3"))
|
||||
val MaxAllowedSeqSize = 100
|
||||
|
||||
//#draining-to-seq
|
||||
val strict: Future[immutable.Seq[Message]] =
|
||||
myData.grouped(MaxAllowedSeqSize).runWith(Sink.head)
|
||||
//#draining-to-seq
|
||||
|
||||
Await.result(strict, 3.seconds) should be(List("1", "2", "3"))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,48 +0,0 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.FlowShape
|
||||
import akka.stream.scaladsl._
|
||||
import akka.testkit.TestProbe
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class RecipeWorkerPool extends RecipeSpec {
|
||||
|
||||
"Recipe for a pool of workers" must {
|
||||
|
||||
"work" in {
|
||||
val myJobs = Source(List("1", "2", "3", "4", "5"))
|
||||
type Result = String
|
||||
|
||||
val worker = Flow[String].map(_ + " done")
|
||||
|
||||
//#worker-pool
|
||||
def balancer[In, Out](worker: Flow[In, Out, Any], workerCount: Int): Flow[In, Out, Unit] = {
|
||||
import GraphDSL.Implicits._
|
||||
|
||||
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 each worker, add an edge from the balancer to the worker, then wire
|
||||
// it to the merge element
|
||||
balancer ~> worker ~> merge
|
||||
}
|
||||
|
||||
FlowShape(balancer.in, merge.out)
|
||||
})
|
||||
}
|
||||
|
||||
val processedJobs: Source[Result, Unit] = myJobs.via(balancer(worker, 3))
|
||||
//#worker-pool
|
||||
|
||||
Await.result(processedJobs.grouped(10).runWith(Sink.head), 3.seconds).toSet should be(Set(
|
||||
"1 done", "2 done", "3 done", "4 done", "5 done"))
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue