+doc #19429 initial merge of docs-dev and docs

This commit is contained in:
Konrad Malawski 2016-01-13 16:25:24 +01:00
parent be0c8af4c0
commit 5a18d43435
501 changed files with 9876 additions and 3681 deletions

View file

@ -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)
}
}
}

View file

@ -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
}
}
}

View file

@ -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))
}
}
}

View file

@ -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()
}
}
}

View file

@ -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"))
}
}
}

View file

@ -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()
}
}
}

View file

@ -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()
}
}
}

View file

@ -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
}
}
}

View file

@ -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)
}
}
}
}

View file

@ -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()
}
}
}

View file

@ -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()
}
}
}

View file

@ -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]"))
}
}
}

View file

@ -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!",
""))
}
}
}

View file

@ -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)))
}
}
}

View file

@ -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()
}
}
}

View file

@ -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
}

View file

@ -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"))
}
}
}

View file

@ -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"))
}
}
}