!str #16902: Unify stream internal representation
also =str #16912: Fix StreamTcpSpec flakiness
This commit is contained in:
parent
cac9c9f2fb
commit
8d77fa8b29
230 changed files with 7814 additions and 9596 deletions
|
|
@ -41,7 +41,7 @@ class RecipeByteStrings extends RecipeSpec {
|
|||
val chunksStream = rawBytes.transform(() => new Chunker(ChunkLimit))
|
||||
//#bytestring-chunker
|
||||
|
||||
val chunksFuture = chunksStream.grouped(10).runWith(Sink.head)
|
||||
val chunksFuture = chunksStream.grouped(10).runWith(Sink.head())
|
||||
|
||||
val chunks = Await.result(chunksFuture, 3.seconds)
|
||||
|
||||
|
|
@ -70,11 +70,11 @@ class RecipeByteStrings extends RecipeSpec {
|
|||
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)
|
||||
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)
|
||||
Await.result(bytes2.via(limiter).grouped(10).runWith(Sink.head()), 3.seconds)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -83,10 +83,10 @@ class RecipeByteStrings extends RecipeSpec {
|
|||
val data = Source(List(ByteString(1, 2), ByteString(3), ByteString(4, 5, 6), ByteString(7, 8, 9)))
|
||||
|
||||
//#compacting-bytestrings
|
||||
val compacted: Source[ByteString] = data.map(_.compact)
|
||||
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)
|
||||
Await.result(compacted.grouped(10).runWith(Sink.head()), 3.seconds).forall(_.isCompact) should be(true)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -41,10 +41,10 @@ class RecipeDigest extends RecipeSpec {
|
|||
}
|
||||
}
|
||||
|
||||
val digest: Source[ByteString] = data.transform(() => digestCalculator("SHA-256"))
|
||||
val digest: Source[ByteString, Unit] = data.transform(() => digestCalculator("SHA-256"))
|
||||
//#calculating-digest
|
||||
|
||||
Await.result(digest.runWith(Sink.head), 3.seconds) should be(
|
||||
Await.result(digest.runWith(Sink.head()), 3.seconds) should be(
|
||||
ByteString(
|
||||
0x24, 0x8d, 0x6a, 0x61,
|
||||
0xd2, 0x06, 0x38, 0xb8,
|
||||
|
|
|
|||
|
|
@ -16,30 +16,26 @@ class RecipeDroppyBroadcast extends RecipeSpec {
|
|||
|
||||
val sub1 = SubscriberProbe[Int]()
|
||||
val sub2 = SubscriberProbe[Int]()
|
||||
val futureSink = Sink.head[Seq[Int]]
|
||||
val mySink1 = Sink(sub1)
|
||||
val mySink2 = Sink(sub2)
|
||||
val futureSink = Sink.head[Seq[Int]]
|
||||
val mySink3 = Flow[Int].grouped(200).to(futureSink)
|
||||
val mySink3 = Flow[Int].grouped(200).toMat(futureSink)(Keep.right)
|
||||
|
||||
//#droppy-bcast
|
||||
// Makes a sink drop elements if too slow
|
||||
def droppySink[T](sink: Sink[T], bufferSize: Int): Sink[T] = {
|
||||
Flow[T].buffer(bufferSize, OverflowStrategy.dropHead).to(sink)
|
||||
}
|
||||
val graph = FlowGraph.closed(mySink1, mySink2, mySink3)((_, _, _)) { implicit b =>
|
||||
(sink1, sink2, sink3) =>
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
import FlowGraphImplicits._
|
||||
val graph = FlowGraph { implicit builder =>
|
||||
val bcast = Broadcast[Int]
|
||||
val bcast = b.add(Broadcast[Int](3))
|
||||
myElements ~> bcast
|
||||
|
||||
myElements ~> bcast
|
||||
|
||||
bcast ~> droppySink(mySink1, 10)
|
||||
bcast ~> droppySink(mySink2, 10)
|
||||
bcast ~> droppySink(mySink3, 10)
|
||||
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink1
|
||||
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink2
|
||||
bcast.buffer(10, OverflowStrategy.dropHead) ~> sink3
|
||||
}
|
||||
//#droppy-bcast
|
||||
|
||||
Await.result(graph.run().get(futureSink), 3.seconds).sum should be(5050)
|
||||
Await.result(graph.run()._3, 3.seconds).sum should be(5050)
|
||||
|
||||
sub1.expectSubscription().request(10)
|
||||
sub2.expectSubscription().request(10)
|
||||
|
|
|
|||
|
|
@ -15,11 +15,11 @@ class RecipeFlattenSeq extends RecipeSpec {
|
|||
val someDataSource = Source(List(List("1"), List("2"), List("3", "4", "5"), List("6", "7")))
|
||||
|
||||
//#flattening-seqs
|
||||
val myData: Source[List[Message]] = someDataSource
|
||||
val flattened: Source[Message] = myData.mapConcat(identity)
|
||||
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"))
|
||||
Await.result(flattened.grouped(8).runWith(Sink.head()), 3.seconds) should be(List("1", "2", "3", "4", "5", "6", "7"))
|
||||
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -76,7 +76,7 @@ class RecipeGlobalRateLimit extends RecipeSpec {
|
|||
"work" in {
|
||||
|
||||
//#global-limiter-flow
|
||||
def limitGlobal[T](limiter: ActorRef, maxAllowedWait: FiniteDuration): Flow[T, T] = {
|
||||
def limitGlobal[T](limiter: ActorRef, maxAllowedWait: FiniteDuration): Flow[T, T, Unit] = {
|
||||
import akka.pattern.ask
|
||||
import akka.util.Timeout
|
||||
Flow[T].mapAsync { (element: T) =>
|
||||
|
|
@ -97,9 +97,9 @@ class RecipeGlobalRateLimit extends RecipeSpec {
|
|||
|
||||
val probe = SubscriberProbe[String]()
|
||||
|
||||
FlowGraph { implicit b =>
|
||||
import FlowGraphImplicits._
|
||||
val merge = Merge[String]
|
||||
FlowGraph.closed() { implicit b =>
|
||||
import FlowGraph.Implicits._
|
||||
val merge = b.add(Merge[String](2))
|
||||
source1 ~> merge ~> Sink(probe)
|
||||
source2 ~> merge
|
||||
}.run()
|
||||
|
|
|
|||
|
|
@ -23,17 +23,16 @@ class RecipeKeepAlive extends RecipeSpec {
|
|||
val sink = Sink(sub)
|
||||
|
||||
//#inject-keepalive
|
||||
val keepAliveStream: Source[ByteString] = ticks
|
||||
val keepAliveStream: Source[ByteString, Unit] = ticks
|
||||
.conflate(seed = (tick) => keepaliveMessage)((msg, newTick) => msg)
|
||||
|
||||
import FlowGraphImplicits._
|
||||
val graph = FlowGraph { implicit builder =>
|
||||
val unfairMerge = MergePreferred[ByteString]
|
||||
val graph = FlowGraph.closed() { implicit builder =>
|
||||
import FlowGraph.Implicits._
|
||||
val unfairMerge = builder.add(MergePreferred[ByteString](1))
|
||||
|
||||
dataStream ~> unfairMerge.preferred // If data is available then no keepalive is injected
|
||||
keepAliveStream ~> unfairMerge
|
||||
|
||||
unfairMerge ~> sink
|
||||
dataStream ~> unfairMerge.preferred
|
||||
// If data is available then no keepalive is injected
|
||||
keepAliveStream ~> unfairMerge ~> sink
|
||||
}
|
||||
//#inject-keepalive
|
||||
|
||||
|
|
|
|||
|
|
@ -18,11 +18,11 @@ class RecipeManualTrigger extends RecipeSpec {
|
|||
val sink = Sink(sub)
|
||||
|
||||
//#manually-triggered-stream
|
||||
import FlowGraphImplicits._
|
||||
val graph = FlowGraph { implicit builder =>
|
||||
val zip = Zip[Message, Trigger]
|
||||
elements ~> zip.left
|
||||
triggerSource ~> zip.right
|
||||
val graph = FlowGraph.closed() { implicit builder =>
|
||||
import FlowGraph.Implicits._
|
||||
val zip = builder.add(Zip[Message, Trigger]())
|
||||
elements ~> zip.in0
|
||||
triggerSource ~> zip.in1
|
||||
zip.out ~> Flow[(Message, Trigger)].map { case (msg, trigger) => msg } ~> sink
|
||||
}
|
||||
//#manually-triggered-stream
|
||||
|
|
@ -57,13 +57,12 @@ class RecipeManualTrigger extends RecipeSpec {
|
|||
val sink = Sink(sub)
|
||||
|
||||
//#manually-triggered-stream-zipwith
|
||||
import FlowGraphImplicits._
|
||||
val graph = FlowGraph { implicit builder =>
|
||||
val zip = ZipWith[Message, Trigger, Message](
|
||||
(msg: Message, trigger: Trigger) => msg)
|
||||
val graph = FlowGraph.closed() { implicit builder =>
|
||||
import FlowGraph.Implicits._
|
||||
val zip = builder.add(ZipWith((msg: Message, trigger: Trigger) => msg))
|
||||
|
||||
elements ~> zip.left
|
||||
triggerSource ~> zip.right
|
||||
elements ~> zip.in0
|
||||
triggerSource ~> zip.in1
|
||||
zip.out ~> sink
|
||||
}
|
||||
//#manually-triggered-stream-zipwith
|
||||
|
|
|
|||
|
|
@ -20,7 +20,7 @@ class RecipeMissedTicks extends RecipeSpec {
|
|||
|
||||
//#missed-ticks
|
||||
// tickStream is a Source[Tick]
|
||||
val missedTicks: Source[Int] =
|
||||
val missedTicks: Source[Int, Unit] =
|
||||
tickStream.conflate(seed = (_) => 0)(
|
||||
(missedTicks, tick) => missedTicks + 1)
|
||||
//#missed-ticks
|
||||
|
|
|
|||
|
|
@ -15,7 +15,7 @@ class RecipeMultiGroupBy extends RecipeSpec {
|
|||
case class Topic(name: String)
|
||||
|
||||
val elems = Source(List("1: a", "1: b", "all: c", "all: d", "1: e"))
|
||||
val topicMapper: (Message) => immutable.Seq[Topic] = { msg =>
|
||||
val topicMapper = { msg: Message =>
|
||||
if (msg.startsWith("1")) List(Topic("1"))
|
||||
else List(Topic("1"), Topic("2"))
|
||||
}
|
||||
|
|
@ -28,14 +28,14 @@ class RecipeMultiGroupBy extends RecipeSpec {
|
|||
}
|
||||
|
||||
//#multi-groupby
|
||||
val messageAndTopic: Source[(Message, Topic)] = elems.mapConcat { msg: Message =>
|
||||
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: Source[(Topic, Source[String])] = messageAndTopic.groupBy(_._2).map {
|
||||
val multiGroups: Source[(Topic, Source[String, Unit]), Unit] = messageAndTopic.groupBy(_._2).map {
|
||||
case (topic, topicStream) =>
|
||||
// chopping of the topic from the (Message, Topic) pairs
|
||||
(topic, topicStream.map(_._1))
|
||||
|
|
@ -43,8 +43,8 @@ class RecipeMultiGroupBy extends RecipeSpec {
|
|||
//#multi-groupby
|
||||
|
||||
val result = multiGroups.map {
|
||||
case (topic, topicMessages) => topicMessages.grouped(10).map(topic.name + _.mkString("[", ", ", "]")).runWith(Sink.head)
|
||||
}.mapAsync(identity).grouped(10).runWith(Sink.head)
|
||||
case (topic, topicMessages) => topicMessages.grouped(10).map(topic.name + _.mkString("[", ", ", "]")).runWith(Sink.head())
|
||||
}.mapAsync(identity).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]",
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ class RecipeParseLines extends RecipeSpec {
|
|||
|
||||
val linesStream = rawData.transform(() => parseLines("\r\n", 100))
|
||||
|
||||
Await.result(linesStream.grouped(10).runWith(Sink.head), 3.seconds) should be(List(
|
||||
Await.result(linesStream.grouped(10).runWith(Sink.head()), 3.seconds) should be(List(
|
||||
"Hello World\r!",
|
||||
"Hello Akka!",
|
||||
"Hello Streams!",
|
||||
|
|
|
|||
|
|
@ -18,10 +18,10 @@ class RecipeReduceByKey extends RecipeSpec {
|
|||
|
||||
//#word-count
|
||||
// split the words into separate streams first
|
||||
val wordStreams: Source[(String, Source[String])] = words.groupBy(identity)
|
||||
val wordStreams: Source[(String, Source[String, Unit]), Unit] = words.groupBy(identity)
|
||||
|
||||
// add counting logic to the streams
|
||||
val countedWords: Source[Future[(String, Int)]] = wordStreams.map {
|
||||
val countedWords: Source[Future[(String, Int)], Unit] = wordStreams.map {
|
||||
case (word, wordStream) =>
|
||||
wordStream.runFold((word, 0)) {
|
||||
case ((w, count), _) => (w, count + 1)
|
||||
|
|
@ -29,13 +29,13 @@ class RecipeReduceByKey extends RecipeSpec {
|
|||
}
|
||||
|
||||
// get a stream of word counts
|
||||
val counts: Source[(String, Int)] =
|
||||
val counts: Source[(String, Int), Unit] =
|
||||
countedWords
|
||||
.buffer(MaximumDistinctWords, OverflowStrategy.fail)
|
||||
.mapAsync(identity)
|
||||
//#word-count
|
||||
|
||||
Await.result(counts.grouped(10).runWith(Sink.head), 3.seconds).toSet should be(Set(
|
||||
Await.result(counts.grouped(10).runWith(Sink.head()), 3.seconds).toSet should be(Set(
|
||||
("hello", 2),
|
||||
("world", 1),
|
||||
("and", 1),
|
||||
|
|
@ -52,7 +52,7 @@ class RecipeReduceByKey extends RecipeSpec {
|
|||
def reduceByKey[In, K, Out](
|
||||
maximumGroupSize: Int,
|
||||
groupKey: (In) => K,
|
||||
foldZero: (K) => Out)(fold: (Out, In) => Out): Flow[In, (K, Out)] = {
|
||||
foldZero: (K) => Out)(fold: (Out, In) => Out): Flow[In, (K, Out), Unit] = {
|
||||
|
||||
val groupStreams = Flow[In].groupBy(groupKey)
|
||||
val reducedValues = groupStreams.map {
|
||||
|
|
@ -72,7 +72,7 @@ class RecipeReduceByKey extends RecipeSpec {
|
|||
|
||||
//#reduce-by-key-general
|
||||
|
||||
Await.result(wordCounts.grouped(10).runWith(Sink.head), 3.seconds).toSet should be(Set(
|
||||
Await.result(wordCounts.grouped(10).runWith(Sink.head()), 3.seconds).toSet should be(Set(
|
||||
("hello", 2),
|
||||
("world", 1),
|
||||
("and", 1),
|
||||
|
|
|
|||
|
|
@ -13,7 +13,7 @@ class RecipeSimpleDrop extends RecipeSpec {
|
|||
"work" in {
|
||||
|
||||
//#simple-drop
|
||||
val droppyStream: Flow[Message, Message] =
|
||||
val droppyStream: Flow[Message, Message, Unit] =
|
||||
Flow[Message].conflate(seed = identity)((lastMessage, newMessage) => newMessage)
|
||||
//#simple-drop
|
||||
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ class RecipeToStrict extends RecipeSpec {
|
|||
|
||||
//#draining-to-seq
|
||||
val strict: Future[immutable.Seq[Message]] =
|
||||
myData.grouped(MaxAllowedSeqSize).runWith(Sink.head)
|
||||
myData.grouped(MaxAllowedSeqSize).runWith(Sink.head())
|
||||
//#draining-to-seq
|
||||
|
||||
Await.result(strict, 3.seconds) should be(List("1", "2", "3"))
|
||||
|
|
|
|||
|
|
@ -17,18 +17,12 @@ class RecipeWorkerPool extends RecipeSpec {
|
|||
val worker = Flow[String].map(_ + " done")
|
||||
|
||||
//#worker-pool
|
||||
def balancer[In, Out](worker: Flow[In, Out], workerCount: Int): Flow[In, Out] = {
|
||||
import FlowGraphImplicits._
|
||||
def balancer[In, Out](worker: Flow[In, Out, Unit], workerCount: Int): Flow[In, Out, Unit] = {
|
||||
import FlowGraph.Implicits._
|
||||
|
||||
Flow[In, Out]() { implicit graphBuilder =>
|
||||
val jobsIn = UndefinedSource[In]
|
||||
val resultsOut = UndefinedSink[Out]
|
||||
|
||||
val balancer = Balance[In](waitForAllDownstreams = true)
|
||||
val merge = Merge[Out]
|
||||
|
||||
jobsIn ~> balancer // Jobs are fed into the balancer
|
||||
merge ~> resultsOut // the merged results are sent out
|
||||
Flow() { 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
|
||||
|
|
@ -36,14 +30,14 @@ class RecipeWorkerPool extends RecipeSpec {
|
|||
balancer ~> worker ~> merge
|
||||
}
|
||||
|
||||
(jobsIn, resultsOut)
|
||||
(balancer.in, merge.out)
|
||||
}
|
||||
}
|
||||
|
||||
val processedJobs: Source[Result] = myJobs.via(balancer(worker, 3))
|
||||
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(
|
||||
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