!str #19005 make groupBy et al return a SubFlow
A SubFlow (or SubSource) is not a Graph, it is an unfinished builder that accepts transformations. This allows us to capture the substreams’ transformations before materializing the flow, which will be very helpful in fully fusing all operators. Another change is that groupBy now requires a maxSubstreams parameter in order to bound its resource usage. In exchange the matching merge can be unbounded. This trades silent deadlock for explicit stream failure. This commit also changes all uses of Predef.identity to use `conforms` and removes the HTTP impl.util.identityFunc.
This commit is contained in:
parent
654fa41443
commit
1500d1f36d
56 changed files with 3484 additions and 720 deletions
|
|
@ -30,17 +30,22 @@ class RecipeMultiGroupBy extends RecipeSpec {
|
|||
topicsForMessage.map(msg -> _)
|
||||
}
|
||||
|
||||
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))
|
||||
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.map {
|
||||
case (topic, topicMessages) => topicMessages.grouped(10).map(topic.name + _.mkString("[", ", ", "]")).runWith(Sink.head)
|
||||
}.mapAsync(4)(identity).grouped(10).runWith(Sink.head)
|
||||
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]",
|
||||
|
|
|
|||
|
|
@ -1,10 +1,10 @@
|
|||
package docs.stream.cookbook
|
||||
|
||||
import akka.stream.OverflowStrategy
|
||||
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 {
|
||||
|
||||
|
|
@ -17,22 +17,15 @@ class RecipeReduceByKey extends RecipeSpec {
|
|||
def words = Source(List("hello", "world", "and", "hello", "universe", "akka") ++ List.fill(1000)("rocks!"))
|
||||
|
||||
//#word-count
|
||||
// split the words into separate streams first
|
||||
val wordStreams: Source[(String, Source[String, Unit]), Unit] = words.groupBy(identity)
|
||||
|
||||
// add counting logic to the streams
|
||||
val countedWords: Source[Future[(String, Int)], Unit] = wordStreams.map {
|
||||
case (word, wordStream) =>
|
||||
wordStream.runFold((word, 0)) {
|
||||
case ((w, count), _) => (w, count + 1)
|
||||
}
|
||||
}
|
||||
|
||||
// get a stream of word counts
|
||||
val counts: Source[(String, Int), Unit] =
|
||||
countedWords
|
||||
.buffer(MaximumDistinctWords, OverflowStrategy.fail)
|
||||
.mapAsync(4)(identity)
|
||||
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(
|
||||
|
|
@ -54,15 +47,17 @@ class RecipeReduceByKey extends RecipeSpec {
|
|||
groupKey: (In) => K,
|
||||
foldZero: (K) => Out)(fold: (Out, In) => Out): Flow[In, (K, Out), Unit] = {
|
||||
|
||||
val groupStreams = Flow[In].groupBy(groupKey)
|
||||
val reducedValues = groupStreams.map {
|
||||
case (key, groupStream) =>
|
||||
groupStream.runFold((key, foldZero(key))) {
|
||||
case ((key, aggregated), elem) => (key, fold(aggregated, elem))
|
||||
}
|
||||
}
|
||||
|
||||
reducedValues.buffer(maximumGroupSize, OverflowStrategy.fail).mapAsync(4)(identity)
|
||||
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(
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue