Add GroupedWeighted FlowOp and Make Grouped use GroupedWeighted #29066

This commit is contained in:
Michael Marshall 2021-01-27 10:03:30 -07:00 committed by GitHub
parent dffd7099fd
commit 4d9b25579d
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
18 changed files with 351 additions and 30 deletions

View file

@ -15,6 +15,12 @@ Accumulate incoming events until the specified number of elements have been accu
Accumulate incoming events until the specified number of elements have been accumulated and then pass the collection of Accumulate incoming events until the specified number of elements have been accumulated and then pass the collection of
elements downstream. elements downstream.
See also:
* @ref[groupedWeighted](groupedWeighted.md) for a variant that groups based on element weight
* @ref[groupedWithin](groupedWithin.md) for a variant that groups based on number of elements and a time window
* @ref[groupedWeightedWithin](groupedWeightedWithin.md) for a variant that groups based on element weight and a time window
## Examples ## Examples
The below example demonstrates how `grouped` groups the accumulated elements into @scala[`Seq`] @java[`List`] The below example demonstrates how `grouped` groups the accumulated elements into @scala[`Seq`] @java[`List`]

View file

@ -0,0 +1,46 @@
# groupedWeighted
Accumulate incoming events until the combined weight of elements is greater than or equal to the minimum weight and then pass the collection of elements downstream.
@ref[Simple operators](../index.md#simple-operators)
## Signature
@apidoc[Source.groupedWeighted](Source) { scala="#groupedWeighted(minWeight:Long)(costFn:Out=>Long):FlowOps.this.Repr[scala.collection.immutable.Seq[Out]]" java="#groupedWeighted(long,akka.japi.function.Function)" }
@apidoc[Flow.groupedWeighted](Flow) { scala="#groupedWeighted(minWeight:Long)(costFn:Out=>Long):FlowOps.this.Repr[scala.collection.immutable.Seq[Out]]" java="#groupedWeighted(long,akka.japi.function.Function)" }
## Description
Chunk up this stream into groups of elements that have a cumulative weight greater than or equal to the `minWeight`, with the last group possibly smaller than requested `minWeight` due to end-of-stream.
See also:
* @ref[grouped](grouped.md) for a variant that groups based on number of elements
* @ref[groupedWithin](groupedWithin.md) for a variant that groups based on number of elements and a time window
* @ref[groupedWeightedWithin](groupedWeightedWithin.md) for a variant that groups based on element weight and a time window
## Examples
The below example demonstrates how `groupedWeighted` groups the accumulated elements into @scala[`Seq`] @java[`List`]
and maps with other operation.
Scala
: @@snip [groupedWeighted.scala](/akka-docs/src/test/scala/docs/stream/operators/sourceorflow/GroupedWeighted.scala) { #groupedWeighted }
Java
: @@snip [SourceOrFlow.java](/akka-docs/src/test/java/jdocs/stream/operators/SourceOrFlow.java) { #groupedWeighted }
## Reactive Streams semantics
@@@div { .callout }
**emits** when the cumulative weight of elements is greater than or equal to the minimum weight or upstream completed
**backpressures** when a group has been assembled and downstream backpressures
**completes** when upstream completes
@@@

View file

@ -16,6 +16,12 @@ Chunk up this stream into groups of elements received within a time window, or l
whatever happens first. Empty groups will not be emitted if no elements are received from upstream. whatever happens first. Empty groups will not be emitted if no elements are received from upstream.
The last group before end-of-stream will contain the buffered elements since the previously emitted group. The last group before end-of-stream will contain the buffered elements since the previously emitted group.
See also:
* @ref[grouped](grouped.md) for a variant that groups based on number of elements
* @ref[groupedWeighted](groupedWeighted.md) for a variant that groups based on element weight
* @ref[groupedWithin](groupedWithin.md) for a variant that groups based on number of elements and a time window
## Reactive Streams semantics ## Reactive Streams semantics
@@@div { .callout } @@@div { .callout }

View file

@ -16,6 +16,12 @@ Chunk up this stream into groups of elements received within a time window, or l
whatever happens first. Empty groups will not be emitted if no elements are received from upstream. whatever happens first. Empty groups will not be emitted if no elements are received from upstream.
The last group before end-of-stream will contain the buffered elements since the previously emitted group. The last group before end-of-stream will contain the buffered elements since the previously emitted group.
See also:
* @ref[grouped](grouped.md) for a variant that groups based on number of elements
* @ref[groupedWeighted](groupedWeighted.md) for a variant that groups based on element weight
* @ref[groupedWeightedWithin](groupedWeightedWithin.md) for a variant that groups based on element weight and a time window
## Reactive Streams semantics ## Reactive Streams semantics
@@@div { .callout } @@@div { .callout }

View file

@ -154,6 +154,7 @@ depending on being backpressured by downstream or not.
|Source/Flow|<a name="frommaterializer"></a>@ref[fromMaterializer](Source-or-Flow/fromMaterializer.md)|Defer the creation of a `Source/Flow` until materialization and access `Materializer` and `Attributes`| |Source/Flow|<a name="frommaterializer"></a>@ref[fromMaterializer](Source-or-Flow/fromMaterializer.md)|Defer the creation of a `Source/Flow` until materialization and access `Materializer` and `Attributes`|
|Flow|<a name="futureflow"></a>@ref[futureFlow](Flow/futureFlow.md)|Streams the elements through the given future flow once it successfully completes.| |Flow|<a name="futureflow"></a>@ref[futureFlow](Flow/futureFlow.md)|Streams the elements through the given future flow once it successfully completes.|
|Source/Flow|<a name="grouped"></a>@ref[grouped](Source-or-Flow/grouped.md)|Accumulate incoming events until the specified number of elements have been accumulated and then pass the collection of elements downstream.| |Source/Flow|<a name="grouped"></a>@ref[grouped](Source-or-Flow/grouped.md)|Accumulate incoming events until the specified number of elements have been accumulated and then pass the collection of elements downstream.|
|Source/Flow|<a name="groupedweighted"></a>@ref[groupedWeighted](Source-or-Flow/groupedWeighted.md)|Accumulate incoming events until the combined weight of elements is greater than or equal to the minimum weight and then pass the collection of elements downstream.|
|Source/Flow|<a name="intersperse"></a>@ref[intersperse](Source-or-Flow/intersperse.md)|Intersperse stream with provided element similar to `List.mkString`.| |Source/Flow|<a name="intersperse"></a>@ref[intersperse](Source-or-Flow/intersperse.md)|Intersperse stream with provided element similar to `List.mkString`.|
|Flow|<a name="lazycompletionstageflow"></a>@ref[lazyCompletionStageFlow](Flow/lazyCompletionStageFlow.md)|Defers creation and materialization of a `Flow` until there is a first element.| |Flow|<a name="lazycompletionstageflow"></a>@ref[lazyCompletionStageFlow](Flow/lazyCompletionStageFlow.md)|Defers creation and materialization of a `Flow` until there is a first element.|
|Flow|<a name="lazyflow"></a>@ref[lazyFlow](Flow/lazyFlow.md)|Defers creation and materialization of a `Flow` until there is a first element.| |Flow|<a name="lazyflow"></a>@ref[lazyFlow](Flow/lazyFlow.md)|Defers creation and materialization of a `Flow` until there is a first element.|
@ -442,6 +443,7 @@ For more background see the @ref[Error Handling in Streams](../stream-error.md)
* [futureSource](Source/futureSource.md) * [futureSource](Source/futureSource.md)
* [groupBy](Source-or-Flow/groupBy.md) * [groupBy](Source-or-Flow/groupBy.md)
* [grouped](Source-or-Flow/grouped.md) * [grouped](Source-or-Flow/grouped.md)
* [groupedWeighted](Source-or-Flow/groupedWeighted.md)
* [groupedWeightedWithin](Source-or-Flow/groupedWeightedWithin.md) * [groupedWeightedWithin](Source-or-Flow/groupedWeightedWithin.md)
* [groupedWithin](Source-or-Flow/groupedWithin.md) * [groupedWithin](Source-or-Flow/groupedWithin.md)
* [gunzip](Compression/gunzip.md) * [gunzip](Compression/gunzip.md)

View file

@ -46,6 +46,7 @@ import akka.stream.Attributes;
// #log // #log
import java.time.Duration; import java.time.Duration;
import java.util.Collection;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage; import java.util.concurrent.CompletionStage;
import java.util.function.IntSupplier; import java.util.function.IntSupplier;
@ -340,6 +341,22 @@ class SourceOrFlow {
// #grouped // #grouped
} }
void groupedWeightedExample() {
// #groupedWeighted
Source.from(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4), Arrays.asList(5, 6)))
.groupedWeighted(4, x -> (long) x.size())
.runForeach(System.out::println, system);
// [[1, 2], [3, 4]]
// [[5, 6]]
Source.from(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3, 4), Arrays.asList(5, 6)))
.groupedWeighted(3, x -> (long) x.size())
.runForeach(System.out::println, system);
// [[1, 2], [3, 4]]
// [[5, 6]]
// #groupedWeighted
}
static static
// #fold // #foldAsync // #fold // #foldAsync
class Histogram { class Histogram {

View file

@ -0,0 +1,27 @@
/*
* Copyright (C) 2019-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package docs.stream.operators.sourceorflow
import akka.stream.scaladsl.Source
import scala.collection.immutable
object GroupedWeighted {
def groupedWeightedExample(): Unit = {
import akka.actor.ActorSystem
implicit val system: ActorSystem = ActorSystem()
//#groupedWeighted
val collections = immutable.Iterable(Seq(1, 2), Seq(3, 4), Seq(5, 6))
Source[Seq[Int]](collections).groupedWeighted(4)(_.length).runForeach(println)
// Vector(Seq(1, 2), Seq(3, 4))
// Vector(Seq(5, 6))
Source[Seq[Int]](collections).groupedWeighted(3)(_.length).runForeach(println)
// Vector(Seq(1, 2), Seq(3, 4))
// Vector(Seq(5, 6))
//#groupedWeighted
}
}

View file

@ -223,7 +223,7 @@ class InterpreterSpec extends StreamSpec with GraphInterpreterSpecKit {
lastEvents() should be(Set(OnComplete, OnNext(0))) lastEvents() should be(Set(OnComplete, OnNext(0)))
} }
"implement grouped" in new OneBoundedSetup[Int](Grouped(3)) { "implement grouped" in new OneBoundedSetup[Int](GroupedWeighted(3, ConstantFun.oneLong)) {
lastEvents() should be(Set.empty) lastEvents() should be(Set.empty)
downstream.requestOne() downstream.requestOne()

View file

@ -5,9 +5,9 @@
package akka.stream.impl.fusing package akka.stream.impl.fusing
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.Supervision import akka.stream.Supervision
import akka.stream.testkit.StreamSpec import akka.stream.testkit.StreamSpec
import akka.util.ConstantFun
class InterpreterSupervisionSpec extends StreamSpec with GraphInterpreterSpecKit { class InterpreterSupervisionSpec extends StreamSpec with GraphInterpreterSpecKit {
@ -102,7 +102,7 @@ class InterpreterSupervisionSpec extends StreamSpec with GraphInterpreterSpecKit
Supervision.resumingDecider, Supervision.resumingDecider,
Map((x: Int) => x + 1), Map((x: Int) => x + 1),
Map((x: Int) => if (x <= 0) throw TE else x + 10), Map((x: Int) => if (x <= 0) throw TE else x + 10),
Grouped(3)) { GroupedWeighted(3, ConstantFun.oneLong)) {
downstream.requestOne() downstream.requestOne()
lastEvents() should be(Set(RequestOne)) lastEvents() should be(Set(RequestOne))
@ -123,7 +123,7 @@ class InterpreterSupervisionSpec extends StreamSpec with GraphInterpreterSpecKit
Supervision.resumingDecider, Supervision.resumingDecider,
Map((x: Int) => x + 1), Map((x: Int) => x + 1),
Map((x: Int) => if (x <= 0) throw TE else x + 10), Map((x: Int) => if (x <= 0) throw TE else x + 10),
Grouped(1000)) { GroupedWeighted(1000, ConstantFun.oneLong)) {
downstream.requestOne() downstream.requestOne()
lastEvents() should be(Set(RequestOne)) lastEvents() should be(Set(RequestOne))

View file

@ -0,0 +1,105 @@
/*
* Copyright (C) 2020-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.stream.scaladsl
import akka.stream.testkit.{ ScriptedTest, StreamSpec, TestPublisher, TestSubscriber }
import akka.testkit.TimingTest
import akka.util.unused
import scala.collection.immutable
import scala.concurrent.Await
import scala.concurrent.duration._
class FlowGroupedWeightedSpec extends StreamSpec("""
akka.stream.materializer.initial-input-buffer-size = 2
""") with ScriptedTest {
"A GroupedWeighted" must {
"produce no group (empty sink sequence) when source is empty" in {
val input = immutable.Seq.empty
def costFn(@unused e: Int): Long = 999999L // set to an arbitrarily big value
val future = Source(input).groupedWeighted(1)(costFn).runWith(Sink.seq)
val result = Await.result(future, remainingOrDefault)
result should be(Seq.empty)
}
"always exhaust a source into a single group if cost is 0" in {
val input = (1 to 15)
def costFn(@unused e: Int): Long = 0L
val minWeight = 1 // chose the least possible value for minWeight
val future = Source(input).groupedWeighted(minWeight)(costFn).runWith(Sink.seq)
val result = Await.result(future, remainingOrDefault)
result should be(Seq(input))
}
"exhaust source into one group if minWeight equals the accumulated cost of the source" in {
val input = (1 to 16)
def costFn(@unused e: Int): Long = 1L
val minWeight = input.length
val future = Source(input).groupedWeighted(minWeight)(costFn).runWith(Sink.seq)
val result = Await.result(future, remainingOrDefault)
result should be(Seq(input))
}
"exhaust a source into one group if minWeight is greater than the accumulated cost of source" in {
val input = List("this", "is", "some", "string")
def costFn(e: String): Long = e.length
val minWeight = Long.MaxValue
val future = Source(input).groupedWeighted(minWeight)(costFn).runWith(Sink.seq)
val result = Await.result(future, remainingOrDefault)
result should be(Seq(input))
}
"emit a group each time the grouped weight is greater than minWeight" in {
val input = List(1, 2, 1, 2)
def costFn(e: Int): Long = e
val minWeight = 2 // must produce two groups of List(1, 2)
val future = Source(input).groupedWeighted(minWeight)(costFn).runWith(Sink.seq)
val result: Seq[Seq[Int]] = Await.result(future, remainingOrDefault)
result should be(Seq(Seq(1, 2), Seq(1, 2)))
}
"not emit group when grouped weight is less than minWeight and upstream has not completed" taggedAs TimingTest in {
val p = TestPublisher.probe[Int]()
val c = TestSubscriber.probe[immutable.Seq[Int]]()
// Note that the cost function set to zero here means the stream will accumulate elements until completed
Source.fromPublisher(p).groupedWeighted(10)(_ => 0L).to(Sink.fromSubscriber(c)).run()
p.sendNext(1)
c.expectSubscription().request(1) // create downstream demand so Grouped pulls on upstream
c.expectNoMessage(50.millis) // message should not be emitted yet
p.sendComplete() // Force Grouped to emit the small group
c.expectNext(50.millis, immutable.Seq(1))
c.expectComplete()
}
"fail during stream initialization when minWeight is negative" in {
val ex = the[IllegalArgumentException] thrownBy Source(1 to 5)
.groupedWeighted(-1)(_ => 1L)
.to(Sink.collection)
.run()
ex.getMessage should be("requirement failed: minWeight must be greater than 0")
}
"fail during stream initialization when minWeight is 0" in {
val ex = the[IllegalArgumentException] thrownBy Source(1 to 5)
.groupedWeighted(0)(_ => 1L)
.to(Sink.collection)
.run()
ex.getMessage should be("requirement failed: minWeight must be greater than 0")
}
"fail the stage when costFn has a negative result" in {
val p = TestPublisher.probe[Int]()
val c = TestSubscriber.probe[immutable.Seq[Int]]()
Source.fromPublisher(p).groupedWeighted(1)(_ => -1L).to(Sink.fromSubscriber(c)).run()
c.expectSubscription().request(1) // create downstream demand so Grouped pulls on upstream
c.expectNoMessage(50.millis) // shouldn't fail until the message is sent
p.sendNext(1) // Send a message that will result in negative cost
val error = c.expectError()
error shouldBe an[IllegalArgumentException]
error.getMessage should be("Negative weight [-1] for element [1] is not allowed")
}
}
}

View file

@ -0,0 +1,3 @@
# Disable compatibility check for @InternalApi private[akka] class
ProblemFilters.exclude[MissingClassProblem]("akka.stream.impl.fusing.Grouped$")
ProblemFilters.exclude[MissingClassProblem]("akka.stream.impl.fusing.Grouped")

View file

@ -33,6 +33,7 @@ import akka.stream.Attributes._
val ask = name("ask") val ask = name("ask")
val grouped = name("grouped") val grouped = name("grouped")
val groupedWithin = name("groupedWithin") val groupedWithin = name("groupedWithin")
val groupedWeighted = name("groupedWeighted")
val groupedWeightedWithin = name("groupedWeightedWithin") val groupedWeightedWithin = name("groupedWeightedWithin")
val limit = name("limit") val limit = name("limit")
val limitWeighted = name("limitWeighted") val limitWeighted = name("limitWeighted")

View file

@ -762,34 +762,37 @@ private[stream] object Collect {
/** /**
* INTERNAL API * INTERNAL API
*/ */
@InternalApi private[akka] final case class Grouped[T](n: Int) extends GraphStage[FlowShape[T, immutable.Seq[T]]] { @InternalApi private[akka] final case class GroupedWeighted[T](minWeight: Long, costFn: T => Long)
require(n > 0, "n must be greater than 0") extends GraphStage[FlowShape[T, immutable.Seq[T]]] {
require(minWeight > 0, "minWeight must be greater than 0")
val in = Inlet[T]("Grouped.in") val in = Inlet[T]("GroupedWeighted.in")
val out = Outlet[immutable.Seq[T]]("Grouped.out") val out = Outlet[immutable.Seq[T]]("GroupedWeighted.out")
override val shape: FlowShape[T, immutable.Seq[T]] = FlowShape(in, out) override val shape: FlowShape[T, immutable.Seq[T]] = FlowShape(in, out)
override protected val initialAttributes: Attributes = DefaultAttributes.grouped override def initialAttributes: Attributes = DefaultAttributes.groupedWeighted
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
new GraphStageLogic(shape) with InHandler with OutHandler { new GraphStageLogic(shape) with InHandler with OutHandler {
private val buf = { private val buf = Vector.newBuilder[T]
val b = Vector.newBuilder[T] var left: Long = minWeight
b.sizeHint(n)
b
}
var left = n
override def onPush(): Unit = { override def onPush(): Unit = {
buf += grab(in) val elem = grab(in)
left -= 1 val cost = costFn(elem)
if (left == 0) { if (cost < 0L)
val elements = buf.result() failStage(new IllegalArgumentException(s"Negative weight [$cost] for element [$elem] is not allowed"))
buf.clear() else {
left = n buf += elem
push(out, elements) left -= cost
} else { if (left <= 0) {
pull(in) val elements = buf.result()
buf.clear()
left = minWeight
push(out, elements)
} else {
pull(in)
}
} }
} }
@ -798,12 +801,11 @@ private[stream] object Collect {
} }
override def onUpstreamFinish(): Unit = { override def onUpstreamFinish(): Unit = {
// This means the buf is filled with some elements but not enough (left < n) to group together. // Since the upstream has finished we have to push any buffered elements downstream.
// Since the upstream has finished we have to push them to downstream though. val elements = buf.result()
if (left < n) { if (elements.nonEmpty) {
val elements = buf.result()
buf.clear() buf.clear()
left = n left = minWeight
push(out, elements) push(out, elements)
} }
completeStage() completeStage()

View file

@ -934,6 +934,26 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
def grouped(n: Int): javadsl.Flow[In, java.util.List[Out], Mat] = def grouped(n: Int): javadsl.Flow[In, java.util.List[Out], Mat] =
new Flow(delegate.grouped(n).map(_.asJava)) // TODO optimize to one step new Flow(delegate.grouped(n).map(_.asJava)) // TODO optimize to one step
/**
* Chunk up this stream into groups of elements that have a cumulative weight greater than or equal to
* the `minWeight`, with the last group possibly smaller than requested `minWeight` due to end-of-stream.
*
* `minWeight` must be positive, otherwise IllegalArgumentException is thrown.
* `costFn` must return a non-negative result for all inputs, otherwise the stage will fail
* with an IllegalArgumentException.
*
* '''Emits when''' the cumulative weight of elements is greater than or equal to the `minWeight` or upstream completed
*
* '''Backpressures when''' a buffered group weighs more than `minWeight` and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def groupedWeighted(minWeight: Long)(
costFn: java.util.function.Function[Out, java.lang.Long]): javadsl.Flow[In, java.util.List[Out], Mat] =
new Flow(delegate.groupedWeighted(minWeight)(costFn.apply).map(_.asJava)) // TODO optimize to one step
/** /**
* Ensure stream boundedness by limiting the number of elements from upstream. * Ensure stream boundedness by limiting the number of elements from upstream.
* If the number of incoming elements exceeds max, it will signal * If the number of incoming elements exceeds max, it will signal

View file

@ -2384,6 +2384,26 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
def grouped(n: Int): javadsl.Source[java.util.List[Out @uncheckedVariance], Mat] = def grouped(n: Int): javadsl.Source[java.util.List[Out @uncheckedVariance], Mat] =
new Source(delegate.grouped(n).map(_.asJava)) new Source(delegate.grouped(n).map(_.asJava))
/**
* Chunk up this stream into groups of elements that have a cumulative weight greater than or equal to
* the `minWeight`, with the last group possibly smaller than requested `minWeight` due to end-of-stream.
*
* `minWeight` must be positive, otherwise IllegalArgumentException is thrown.
* `costFn` must return a non-negative result for all inputs, otherwise the stage will fail
* with an IllegalArgumentException.
*
* '''Emits when''' the cumulative weight of elements is greater than or equal to the `minWeight` or upstream completed
*
* '''Backpressures when''' a buffered group weighs more than `minWeight` and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def groupedWeighted(minWeight: Long)(costFn: java.util.function.Function[Out, java.lang.Long])
: javadsl.Source[java.util.List[Out @uncheckedVariance], Mat] =
new Source(delegate.groupedWeighted(minWeight)(costFn.apply).map(_.asJava))
/** /**
* Ensure stream boundedness by limiting the number of elements from upstream. * Ensure stream boundedness by limiting the number of elements from upstream.
* If the number of incoming elements exceeds max, it will signal * If the number of incoming elements exceeds max, it will signal

View file

@ -380,6 +380,26 @@ class SubFlow[In, Out, Mat](
def grouped(n: Int): SubFlow[In, java.util.List[Out @uncheckedVariance], Mat] = def grouped(n: Int): SubFlow[In, java.util.List[Out @uncheckedVariance], Mat] =
new SubFlow(delegate.grouped(n).map(_.asJava)) // TODO optimize to one step new SubFlow(delegate.grouped(n).map(_.asJava)) // TODO optimize to one step
/**
* Chunk up this stream into groups of elements that have a cumulative weight greater than or equal to
* the `minWeight`, with the last group possibly smaller than requested `minWeight` due to end-of-stream.
*
* `minWeight` must be positive, otherwise IllegalArgumentException is thrown.
* `costFn` must return a non-negative result for all inputs, otherwise the stage will fail
* with an IllegalArgumentException.
*
* '''Emits when''' the cumulative weight of elements is greater than or equal to the `minWeight` or upstream completed
*
* '''Backpressures when''' a buffered group weighs more than `minWeight` and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def groupedWeighted(minWeight: Long)(
costFn: function.Function[Out, java.lang.Long]): SubFlow[In, java.util.List[Out @uncheckedVariance], Mat] =
new SubFlow(delegate.groupedWeighted(minWeight)(costFn.apply).map(_.asJava)) // TODO optimize to one step
/** /**
* Ensure stream boundedness by limiting the number of elements from upstream. * Ensure stream boundedness by limiting the number of elements from upstream.
* If the number of incoming elements exceeds max, it will signal * If the number of incoming elements exceeds max, it will signal

View file

@ -371,6 +371,26 @@ class SubSource[Out, Mat](
def grouped(n: Int): SubSource[java.util.List[Out @uncheckedVariance], Mat] = def grouped(n: Int): SubSource[java.util.List[Out @uncheckedVariance], Mat] =
new SubSource(delegate.grouped(n).map(_.asJava)) // TODO optimize to one step new SubSource(delegate.grouped(n).map(_.asJava)) // TODO optimize to one step
/**
* Chunk up this stream into groups of elements that have a cumulative weight greater than or equal to
* the `minWeight`, with the last group possibly smaller than requested `minWeight` due to end-of-stream.
*
* `minWeight` must be positive, otherwise IllegalArgumentException is thrown.
* `costFn` must return a non-negative result for all inputs, otherwise the stage will fail
* with an IllegalArgumentException.
*
* '''Emits when''' the cumulative weight of elements is greater than or equal to the `minWeight` or upstream completed
*
* '''Backpressures when''' a buffered group weighs more than `minWeight` and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def groupedWeighted(minWeight: Long)(
costFn: function.Function[Out, java.lang.Long]): SubSource[java.util.List[Out @uncheckedVariance], Mat] =
new SubSource(delegate.groupedWeighted(minWeight)(costFn.apply).map(_.asJava)) // TODO optimize to one step
/** /**
* Apply a sliding window over the stream and return the windows as groups of elements, with the last group * Apply a sliding window over the stream and return the windows as groups of elements, with the last group
* possibly smaller than requested due to end-of-stream. * possibly smaller than requested due to end-of-stream.

View file

@ -1316,7 +1316,27 @@ trait FlowOps[+Out, +Mat] {
* *
* '''Cancels when''' downstream cancels * '''Cancels when''' downstream cancels
*/ */
def grouped(n: Int): Repr[immutable.Seq[Out]] = via(Grouped(n)) def grouped(n: Int): Repr[immutable.Seq[Out]] =
via(GroupedWeighted[Out](n, ConstantFun.oneLong).withAttributes(DefaultAttributes.grouped))
/**
* Chunk up this stream into groups of elements that have a cumulative weight greater than or equal to
* the `minWeight`, with the last group possibly smaller than requested `minWeight` due to end-of-stream.
*
* `minWeight` must be positive, otherwise IllegalArgumentException is thrown.
* `costFn` must return a non-negative result for all inputs, otherwise the stage will fail
* with an IllegalArgumentException.
*
* '''Emits when''' the cumulative weight of elements is greater than or equal to the `minWeight` or upstream completed
*
* '''Backpressures when''' a buffered group weighs more than `minWeight` and downstream backpressures
*
* '''Completes when''' upstream completes
*
* '''Cancels when''' downstream cancels
*/
def groupedWeighted(minWeight: Long)(costFn: Out => Long): Repr[immutable.Seq[Out]] =
via(GroupedWeighted[Out](minWeight, costFn))
/** /**
* Ensure stream boundedness by limiting the number of elements from upstream. * Ensure stream boundedness by limiting the number of elements from upstream.