Replace LazyFlow with FutureFlow (#28729)
And implement other lazy flows on top of it Co-Authored-By: Johannes Rudolph <johannes.rudolph@gmail.com>
This commit is contained in:
parent
9a1d5191b9
commit
de59bb6803
15 changed files with 1435 additions and 739 deletions
|
|
@ -36,5 +36,9 @@ Scala
|
||||||
|
|
||||||
**completes** when upstream completes and all futures have been completed and all elements have been emitted
|
**completes** when upstream completes and all futures have been completed and all elements have been emitted
|
||||||
|
|
||||||
|
**cancels** when downstream cancels (keep reading)
|
||||||
|
The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
|
||||||
|
This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
|
||||||
|
this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
|
||||||
@@@
|
@@@
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -29,5 +29,9 @@ and failed with a `akka.stream.NeverMaterializedException` if the stream fails o
|
||||||
|
|
||||||
**completes** when upstream completes and all futures have been completed and all elements have been emitted
|
**completes** when upstream completes and all futures have been completed and all elements have been emitted
|
||||||
|
|
||||||
|
**cancels** when downstream cancels (keep reading)
|
||||||
|
The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
|
||||||
|
This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
|
||||||
|
this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
|
||||||
@@@
|
@@@
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -35,5 +35,9 @@ See @ref:[lazyFlow](lazyFlow.md) for sample.
|
||||||
|
|
||||||
**completes** when upstream completes and all futures have been completed and all elements have been emitted
|
**completes** when upstream completes and all futures have been completed and all elements have been emitted
|
||||||
|
|
||||||
|
**cancels** when downstream cancels (keep reading)
|
||||||
|
The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
|
||||||
|
This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
|
||||||
|
this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
|
||||||
@@@
|
@@@
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -26,5 +26,9 @@ Defers creation until a first element arrives.
|
||||||
|
|
||||||
**completes** when upstream completes and all futures have been completed and all elements have been emitted
|
**completes** when upstream completes and all futures have been completed and all elements have been emitted
|
||||||
|
|
||||||
|
**cancels** when downstream cancels (keep reading)
|
||||||
|
The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
|
||||||
|
This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
|
||||||
|
this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
|
||||||
@@@
|
@@@
|
||||||
|
|
||||||
|
|
|
||||||
File diff suppressed because it is too large
Load diff
|
|
@ -0,0 +1,529 @@
|
||||||
|
/*
|
||||||
|
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||||
|
*/
|
||||||
|
|
||||||
|
package akka.stream.scaladsl
|
||||||
|
|
||||||
|
import akka.NotUsed
|
||||||
|
import akka.stream.SubscriptionWithCancelException.NonFailureCancellation
|
||||||
|
import akka.stream.{ AbruptStageTerminationException, Attributes, Materializer, NeverMaterializedException }
|
||||||
|
import akka.stream.testkit.StreamSpec
|
||||||
|
import akka.stream.testkit.Utils.TE
|
||||||
|
import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped
|
||||||
|
|
||||||
|
import scala.concurrent.{ Future, Promise }
|
||||||
|
|
||||||
|
class FlowFutureFlowSpec extends StreamSpec {
|
||||||
|
def src10(i: Int = 0) = Source(i until (i + 10))
|
||||||
|
def src10WithFailure(i: Int = 0)(failOn: Int) = src10(i).map {
|
||||||
|
case `failOn` => throw TE(s"fail on $failOn")
|
||||||
|
case x => x
|
||||||
|
}
|
||||||
|
|
||||||
|
//this stage's behaviour in case of an 'early' downstream cancellation is governed by an attribute
|
||||||
|
//so we run all tests cases using both modes of the attributes.
|
||||||
|
//please notice most of the cases don't exhibit any difference in behaviour between the two modes
|
||||||
|
for {
|
||||||
|
att <- List(
|
||||||
|
Attributes.NestedMaterializationCancellationPolicy.EagerCancellation,
|
||||||
|
Attributes.NestedMaterializationCancellationPolicy.PropagateToNested)
|
||||||
|
delayDownstreanCancellation = att.propagateToNestedMaterialization
|
||||||
|
attributes = Attributes(att)
|
||||||
|
} {
|
||||||
|
|
||||||
|
s"a futureFlow with $att" must {
|
||||||
|
"work in the simple case with a completed future" in assertAllStagesStopped {
|
||||||
|
val (fNotUsed, fSeq) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow {
|
||||||
|
Future.successful(Flow[Int])
|
||||||
|
}
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.futureValue should be(NotUsed)
|
||||||
|
fSeq.futureValue should equal(0 until 10)
|
||||||
|
}
|
||||||
|
|
||||||
|
"work in the simple case with a late future" in assertAllStagesStopped {
|
||||||
|
val prFlow = Promise[Flow[Int, Int, NotUsed]]
|
||||||
|
val (fNotUsed, fSeq) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow(prFlow.future)
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.value should be(empty)
|
||||||
|
fSeq.value should be(empty)
|
||||||
|
|
||||||
|
prFlow.success(Flow[Int])
|
||||||
|
|
||||||
|
fNotUsed.futureValue should be(NotUsed)
|
||||||
|
fSeq.futureValue should equal(0 until 10)
|
||||||
|
}
|
||||||
|
|
||||||
|
"fail properly when future is a completed failed future" in assertAllStagesStopped {
|
||||||
|
val (fNotUsed, fSeq) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow {
|
||||||
|
Future.failed[Flow[Int, Int, NotUsed]](TE("damn!"))
|
||||||
|
}
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.failed.futureValue should be(a[NeverMaterializedException])
|
||||||
|
fNotUsed.failed.futureValue.getCause should equal(TE("damn!"))
|
||||||
|
|
||||||
|
fSeq.failed.futureValue should equal(TE("damn!"))
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
"fail properly when future is late completed failed future" in assertAllStagesStopped {
|
||||||
|
val prFlow = Promise[Flow[Int, Int, NotUsed]]
|
||||||
|
val (fNotUsed, fSeq) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow(prFlow.future)
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.value should be(empty)
|
||||||
|
fSeq.value should be(empty)
|
||||||
|
|
||||||
|
prFlow.failure(TE("damn!"))
|
||||||
|
|
||||||
|
fNotUsed.failed.futureValue should be(a[NeverMaterializedException])
|
||||||
|
fNotUsed.failed.futureValue.getCause should equal(TE("damn!"))
|
||||||
|
|
||||||
|
fSeq.failed.futureValue should equal(TE("damn!"))
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
"handle upstream failure when future is pre-completed" in assertAllStagesStopped {
|
||||||
|
val (fNotUsed, fSeq) = src10WithFailure()(5)
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow {
|
||||||
|
Future.successful {
|
||||||
|
Flow[Int].recover {
|
||||||
|
case TE("fail on 5") => 99
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.futureValue should be(NotUsed)
|
||||||
|
fSeq.futureValue should equal(List(0, 1, 2, 3, 4, 99))
|
||||||
|
}
|
||||||
|
|
||||||
|
"handle upstream failure when future is late-completed" in assertAllStagesStopped {
|
||||||
|
val prFlow = Promise[Flow[Int, Int, NotUsed]]
|
||||||
|
val (fNotUsed, fSeq) = src10WithFailure()(5)
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow(prFlow.future)
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.value should be(empty)
|
||||||
|
fSeq.value should be(empty)
|
||||||
|
|
||||||
|
prFlow.success {
|
||||||
|
Flow[Int].recover {
|
||||||
|
case TE("fail on 5") => 99
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
fNotUsed.futureValue should be(NotUsed)
|
||||||
|
fSeq.futureValue should equal(List(0, 1, 2, 3, 4, 99))
|
||||||
|
}
|
||||||
|
|
||||||
|
"propagate upstream failure when future is pre-completed" in assertAllStagesStopped {
|
||||||
|
val (fNotUsed, fSeq) = src10WithFailure()(5)
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow {
|
||||||
|
Future.successful {
|
||||||
|
Flow[Int]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.futureValue should be(NotUsed)
|
||||||
|
fSeq.failed.futureValue should equal(TE("fail on 5"))
|
||||||
|
}
|
||||||
|
|
||||||
|
"propagate upstream failure when future is late-completed" in assertAllStagesStopped {
|
||||||
|
val prFlow = Promise[Flow[Int, Int, NotUsed]]
|
||||||
|
val (fNotUsed, fSeq) = src10WithFailure()(5)
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow(prFlow.future)
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.value should be(empty)
|
||||||
|
fSeq.value should be(empty)
|
||||||
|
|
||||||
|
prFlow.success {
|
||||||
|
Flow[Int]
|
||||||
|
}
|
||||||
|
|
||||||
|
fNotUsed.futureValue should be(NotUsed)
|
||||||
|
fSeq.failed.futureValue should equal(TE("fail on 5"))
|
||||||
|
}
|
||||||
|
|
||||||
|
"handle early upstream error when flow future is pre-completed" in assertAllStagesStopped {
|
||||||
|
val (fNotUsed, fSeq) = Source
|
||||||
|
.failed(TE("not today my friend"))
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow {
|
||||||
|
Future.successful {
|
||||||
|
Flow[Int]
|
||||||
|
.recover {
|
||||||
|
case TE("not today my friend") => 99
|
||||||
|
}
|
||||||
|
.concat(src10())
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.futureValue should be(NotUsed)
|
||||||
|
fSeq.futureValue should equal(99 +: (0 until 10))
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
"handle early upstream error when flow future is late-completed" in assertAllStagesStopped {
|
||||||
|
val prFlow = Promise[Flow[Int, Int, NotUsed]]
|
||||||
|
val (fNotUsed, fSeq) = Source
|
||||||
|
.failed(TE("not today my friend"))
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow(prFlow.future)
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.value should be(empty)
|
||||||
|
fSeq.value should be(empty)
|
||||||
|
|
||||||
|
prFlow.success {
|
||||||
|
Flow[Int]
|
||||||
|
.recover {
|
||||||
|
case TE("not today my friend") => 99
|
||||||
|
}
|
||||||
|
.concat(src10())
|
||||||
|
}
|
||||||
|
|
||||||
|
fNotUsed.futureValue should be(NotUsed)
|
||||||
|
fSeq.futureValue should equal(99 +: (0 until 10))
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
"handle closed downstream when flow future is pre completed" in assertAllStagesStopped {
|
||||||
|
val (fSeq1, fSeq2) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow {
|
||||||
|
Future.successful {
|
||||||
|
Flow[Int].alsoToMat(Sink.seq)(Keep.right)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}(Keep.right)
|
||||||
|
.mapMaterializedValue(_.flatten)
|
||||||
|
.take(0)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fSeq1.futureValue should be(empty)
|
||||||
|
fSeq2.futureValue should be(empty)
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
"handle closed downstream when flow future is late completed" in assertAllStagesStopped {
|
||||||
|
val prFlow = Promise[Flow[Int, Int, Future[collection.immutable.Seq[Int]]]]
|
||||||
|
val (fSeq1, fSeq2) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow(prFlow.future)
|
||||||
|
}(Keep.right)
|
||||||
|
.mapMaterializedValue(_.flatten)
|
||||||
|
.take(0)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
if (delayDownstreanCancellation) {
|
||||||
|
fSeq1.value should be(empty)
|
||||||
|
fSeq2.value should be(empty)
|
||||||
|
|
||||||
|
prFlow.success {
|
||||||
|
Flow[Int].alsoToMat(Sink.seq)(Keep.right)
|
||||||
|
}
|
||||||
|
|
||||||
|
fSeq1.futureValue should be(empty)
|
||||||
|
fSeq2.futureValue should be(empty)
|
||||||
|
} else {
|
||||||
|
fSeq1.failed.futureValue should be(a[NeverMaterializedException])
|
||||||
|
fSeq1.failed.futureValue.getCause should be(a[NonFailureCancellation])
|
||||||
|
fSeq2.futureValue should be(empty)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"handle early downstream failure when flow future is pre-completed" in assertAllStagesStopped {
|
||||||
|
val (fSeq1, fSeq2) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow {
|
||||||
|
Future.successful {
|
||||||
|
Flow[Int].alsoToMat(Sink.seq)(Keep.right)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}(Keep.right)
|
||||||
|
.mapMaterializedValue(_.flatten)
|
||||||
|
.prepend(Source.failed(TE("damn!")))
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fSeq1.failed.futureValue should equal(TE("damn!"))
|
||||||
|
fSeq2.failed.futureValue should equal(TE("damn!"))
|
||||||
|
}
|
||||||
|
|
||||||
|
"handle early downstream failure when flow future is late completed" in assertAllStagesStopped {
|
||||||
|
val prFlow = Promise[Flow[Int, Int, Future[collection.immutable.Seq[Int]]]]
|
||||||
|
val (fSeq1, fSeq2) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow(prFlow.future)
|
||||||
|
}(Keep.right)
|
||||||
|
.mapMaterializedValue(_.flatten)
|
||||||
|
.prepend(Source.failed(TE("damn!")))
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
if (delayDownstreanCancellation) {
|
||||||
|
fSeq2.failed.futureValue should equal(TE("damn!"))
|
||||||
|
fSeq1.value should be(empty)
|
||||||
|
|
||||||
|
prFlow.success {
|
||||||
|
Flow[Int].alsoToMat(Sink.seq)(Keep.right)
|
||||||
|
}
|
||||||
|
|
||||||
|
fSeq1.failed.futureValue should equal(TE("damn!"))
|
||||||
|
} else {
|
||||||
|
fSeq1.failed.futureValue should be(a[NeverMaterializedException])
|
||||||
|
fSeq1.failed.futureValue.getCause should equal(TE("damn!"))
|
||||||
|
fSeq2.failed.futureValue should equal(TE("damn!"))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"handle early upstream completion when flow future is pre-completed" in assertAllStagesStopped {
|
||||||
|
val (fNotUsed, fSeq) = Source
|
||||||
|
.empty[Int]
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow {
|
||||||
|
Future.successful {
|
||||||
|
Flow[Int].orElse(Source.single(99))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.futureValue should be(NotUsed)
|
||||||
|
fSeq.futureValue should equal(99 :: Nil)
|
||||||
|
}
|
||||||
|
|
||||||
|
"handle early upstream completion when flow future is late-completed" in assertAllStagesStopped {
|
||||||
|
val prFlow = Promise[Flow[Int, Int, NotUsed]]
|
||||||
|
val (fNotUsed, fSeq) = Source
|
||||||
|
.empty[Int]
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow(prFlow.future)
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.value should be(empty)
|
||||||
|
fSeq.value should be(empty)
|
||||||
|
|
||||||
|
prFlow.success {
|
||||||
|
Flow[Int].orElse(Source.single(99))
|
||||||
|
}
|
||||||
|
|
||||||
|
fNotUsed.futureValue should be(NotUsed)
|
||||||
|
fSeq.futureValue should equal(99 :: Nil)
|
||||||
|
}
|
||||||
|
|
||||||
|
"fails properly on materialization failure with a completed future" in assertAllStagesStopped {
|
||||||
|
val (fNotUsed, fSeq) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow {
|
||||||
|
Future.successful(Flow[Int].mapMaterializedValue[NotUsed](_ => throw TE("BBOM!")))
|
||||||
|
}
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.failed.futureValue should be(a[NeverMaterializedException])
|
||||||
|
fNotUsed.failed.futureValue.getCause should equal(TE("BBOM!"))
|
||||||
|
fSeq.failed.futureValue should equal(TE("BBOM!"))
|
||||||
|
}
|
||||||
|
|
||||||
|
"fails properly on materialization failure with a late future" in assertAllStagesStopped {
|
||||||
|
val prFlow = Promise[Flow[Int, Int, NotUsed]]
|
||||||
|
val (fNotUsed, fSeq) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow(prFlow.future)
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.value should be(empty)
|
||||||
|
fSeq.value should be(empty)
|
||||||
|
|
||||||
|
prFlow.success(Flow[Int].mapMaterializedValue[NotUsed](_ => throw TE("BBOM!")))
|
||||||
|
|
||||||
|
fNotUsed.failed.futureValue should be(a[NeverMaterializedException])
|
||||||
|
fNotUsed.failed.futureValue.getCause should equal(TE("BBOM!"))
|
||||||
|
fSeq.failed.futureValue should equal(TE("BBOM!"))
|
||||||
|
}
|
||||||
|
|
||||||
|
"propagate flow failures with a completed future" in assertAllStagesStopped {
|
||||||
|
val (fNotUsed, fSeq) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow {
|
||||||
|
Future.successful {
|
||||||
|
Flow[Int].map {
|
||||||
|
case 5 => throw TE("fail on 5")
|
||||||
|
case x => x
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.futureValue should be(NotUsed)
|
||||||
|
fSeq.failed.futureValue should equal(TE("fail on 5"))
|
||||||
|
}
|
||||||
|
|
||||||
|
"propagate flow failures with a late future" in assertAllStagesStopped {
|
||||||
|
val prFlow = Promise[Flow[Int, Int, NotUsed]]
|
||||||
|
val (fNotUsed, fSeq) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow(prFlow.future)
|
||||||
|
}(Keep.right)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fNotUsed.value should be(empty)
|
||||||
|
fSeq.value should be(empty)
|
||||||
|
|
||||||
|
prFlow.success {
|
||||||
|
Flow[Int].map {
|
||||||
|
case 5 => throw TE("fail on 5")
|
||||||
|
case x => x
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
fNotUsed.futureValue should be(NotUsed)
|
||||||
|
fSeq.failed.futureValue should equal(TE("fail on 5"))
|
||||||
|
}
|
||||||
|
|
||||||
|
"allow flow to handle downstream completion with a completed future" in assertAllStagesStopped {
|
||||||
|
val (fSeq1, fSeq2) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow {
|
||||||
|
Future.successful {
|
||||||
|
Flow.fromSinkAndSourceMat(Sink.seq[Int], src10(10))(Keep.left)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}(Keep.right)
|
||||||
|
.take(5)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fSeq1.flatten.futureValue should be(0 until 10)
|
||||||
|
fSeq2.futureValue should equal(10 until 15)
|
||||||
|
}
|
||||||
|
|
||||||
|
"allow flow to handle downstream completion with a late future" in assertAllStagesStopped {
|
||||||
|
val pr = Promise[Flow[Int, Int, Future[Seq[Int]]]]
|
||||||
|
val (fSeq1, fSeq2) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow(pr.future)
|
||||||
|
}(Keep.right)
|
||||||
|
.take(5)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()
|
||||||
|
|
||||||
|
fSeq1.value should be(empty)
|
||||||
|
fSeq2.value should be(empty)
|
||||||
|
|
||||||
|
pr.success {
|
||||||
|
Flow.fromSinkAndSourceMat(Sink.seq[Int], src10(10))(Keep.left)
|
||||||
|
}
|
||||||
|
|
||||||
|
fSeq1.flatten.futureValue should be(0 until 10)
|
||||||
|
fSeq2.futureValue should equal(10 until 15)
|
||||||
|
}
|
||||||
|
|
||||||
|
"abrupt termination before future completion" in assertAllStagesStopped {
|
||||||
|
val mat = Materializer(system)
|
||||||
|
val prFlow = Promise[Flow[Int, Int, Future[collection.immutable.Seq[Int]]]]
|
||||||
|
val (fSeq1, fSeq2) = src10()
|
||||||
|
.viaMat {
|
||||||
|
Flow.futureFlow(prFlow.future)
|
||||||
|
}(Keep.right)
|
||||||
|
.take(5)
|
||||||
|
.toMat(Sink.seq)(Keep.both)
|
||||||
|
.withAttributes(attributes)
|
||||||
|
.run()(mat)
|
||||||
|
|
||||||
|
fSeq1.value should be(empty)
|
||||||
|
fSeq2.value should be(empty)
|
||||||
|
|
||||||
|
mat.shutdown()
|
||||||
|
|
||||||
|
fSeq1.failed.futureValue should be(a[AbruptStageTerminationException])
|
||||||
|
fSeq2.failed.futureValue should be(a[AbruptStageTerminationException])
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"NestedMaterializationCancellationPolicy" must {
|
||||||
|
"default to false" in assertAllStagesStopped {
|
||||||
|
val fl = Flow.fromMaterializer {
|
||||||
|
case (_, attributes) =>
|
||||||
|
val att = attributes.mandatoryAttribute[Attributes.NestedMaterializationCancellationPolicy]
|
||||||
|
att.propagateToNestedMaterialization should be(false)
|
||||||
|
Flow[Any]
|
||||||
|
}
|
||||||
|
Source.empty.via(fl).runWith(Sink.headOption).futureValue should be(empty)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -8,10 +8,8 @@ import scala.collection.immutable
|
||||||
import scala.concurrent.Future
|
import scala.concurrent.Future
|
||||||
import scala.concurrent.Promise
|
import scala.concurrent.Promise
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
import com.github.ghik.silencer.silent
|
import com.github.ghik.silencer.silent
|
||||||
|
import akka.{ Done, NotUsed }
|
||||||
import akka.NotUsed
|
|
||||||
import akka.stream.AbruptStageTerminationException
|
import akka.stream.AbruptStageTerminationException
|
||||||
import akka.stream.Materializer
|
import akka.stream.Materializer
|
||||||
import akka.stream.NeverMaterializedException
|
import akka.stream.NeverMaterializedException
|
||||||
|
|
@ -127,7 +125,8 @@ class LazyFlowSpec extends StreamSpec("""
|
||||||
val deferredMatVal = result._1
|
val deferredMatVal = result._1
|
||||||
val list = result._2
|
val list = result._2
|
||||||
list.failed.futureValue shouldBe a[TE]
|
list.failed.futureValue shouldBe a[TE]
|
||||||
deferredMatVal.failed.futureValue shouldBe a[TE]
|
deferredMatVal.failed.futureValue shouldBe a[NeverMaterializedException]
|
||||||
|
deferredMatVal.failed.futureValue.getCause shouldBe a[TE]
|
||||||
}
|
}
|
||||||
|
|
||||||
"fail the flow when the future is initially failed" in assertAllStagesStopped {
|
"fail the flow when the future is initially failed" in assertAllStagesStopped {
|
||||||
|
|
@ -140,7 +139,8 @@ class LazyFlowSpec extends StreamSpec("""
|
||||||
val deferredMatVal = result._1
|
val deferredMatVal = result._1
|
||||||
val list = result._2
|
val list = result._2
|
||||||
list.failed.futureValue shouldBe a[TE]
|
list.failed.futureValue shouldBe a[TE]
|
||||||
deferredMatVal.failed.futureValue shouldBe a[TE]
|
deferredMatVal.failed.futureValue shouldBe a[NeverMaterializedException]
|
||||||
|
deferredMatVal.failed.futureValue.getCause shouldBe a[TE]
|
||||||
}
|
}
|
||||||
|
|
||||||
"fail the flow when the future is failed after the fact" in assertAllStagesStopped {
|
"fail the flow when the future is failed after the fact" in assertAllStagesStopped {
|
||||||
|
|
@ -156,7 +156,28 @@ class LazyFlowSpec extends StreamSpec("""
|
||||||
|
|
||||||
promise.failure(TE("later-no-flow-for-you"))
|
promise.failure(TE("later-no-flow-for-you"))
|
||||||
list.failed.futureValue shouldBe a[TE]
|
list.failed.futureValue shouldBe a[TE]
|
||||||
deferredMatVal.failed.futureValue shouldBe a[TE]
|
deferredMatVal.failed.futureValue shouldBe a[NeverMaterializedException]
|
||||||
|
deferredMatVal.failed.futureValue.getCause shouldBe a[TE]
|
||||||
|
}
|
||||||
|
|
||||||
|
"work for a single element when the future is completed after the fact" in assertAllStagesStopped {
|
||||||
|
import system.dispatcher
|
||||||
|
val flowPromise = Promise[Flow[Int, String, NotUsed]]()
|
||||||
|
val firstElementArrived = Promise[Done]()
|
||||||
|
|
||||||
|
val result: Future[immutable.Seq[String]] =
|
||||||
|
Source(List(1))
|
||||||
|
.via(Flow.lazyFutureFlow { () =>
|
||||||
|
firstElementArrived.success(Done)
|
||||||
|
flowPromise.future
|
||||||
|
})
|
||||||
|
.runWith(Sink.seq)
|
||||||
|
|
||||||
|
firstElementArrived.future.map { _ =>
|
||||||
|
flowPromise.success(Flow[Int].map(_.toString))
|
||||||
|
}
|
||||||
|
|
||||||
|
result.futureValue shouldBe List("1")
|
||||||
}
|
}
|
||||||
|
|
||||||
"fail the flow when the future materialization fails" in assertAllStagesStopped {
|
"fail the flow when the future materialization fails" in assertAllStagesStopped {
|
||||||
|
|
@ -170,7 +191,9 @@ class LazyFlowSpec extends StreamSpec("""
|
||||||
val deferredMatVal = result._1
|
val deferredMatVal = result._1
|
||||||
val list = result._2
|
val list = result._2
|
||||||
list.failed.futureValue shouldBe a[TE]
|
list.failed.futureValue shouldBe a[TE]
|
||||||
deferredMatVal.failed.futureValue shouldBe a[TE]
|
//futureFlow's behaviour in case of mat failure (follows flatMapPrefix)
|
||||||
|
deferredMatVal.failed.futureValue shouldBe a[NeverMaterializedException]
|
||||||
|
deferredMatVal.failed.futureValue.getCause shouldEqual TE("mat-failed")
|
||||||
}
|
}
|
||||||
|
|
||||||
"fail the flow when there was elements but the inner flow failed" in assertAllStagesStopped {
|
"fail the flow when there was elements but the inner flow failed" in assertAllStagesStopped {
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,2 @@
|
||||||
|
# Changes to internals
|
||||||
|
ProblemFilters.exclude[MissingClassProblem]("akka.stream.impl.fusing.LazyFlow")
|
||||||
|
|
@ -745,6 +745,7 @@ final class ActorMaterializerSettings @InternalApi private (
|
||||||
// for stream refs and io live with the respective stages
|
// for stream refs and io live with the respective stages
|
||||||
Attributes.InputBuffer(initialInputBufferSize, maxInputBufferSize) ::
|
Attributes.InputBuffer(initialInputBufferSize, maxInputBufferSize) ::
|
||||||
Attributes.CancellationStrategy.Default :: // FIXME: make configurable, see https://github.com/akka/akka/issues/28000
|
Attributes.CancellationStrategy.Default :: // FIXME: make configurable, see https://github.com/akka/akka/issues/28000
|
||||||
|
Attributes.NestedMaterializationCancellationPolicy.Default ::
|
||||||
ActorAttributes.Dispatcher(dispatcher) ::
|
ActorAttributes.Dispatcher(dispatcher) ::
|
||||||
ActorAttributes.SupervisionStrategy(supervisionDecider) ::
|
ActorAttributes.SupervisionStrategy(supervisionDecider) ::
|
||||||
ActorAttributes.DebugLogging(debugLogging) ::
|
ActorAttributes.DebugLogging(debugLogging) ::
|
||||||
|
|
|
||||||
|
|
@ -439,6 +439,78 @@ object Attributes {
|
||||||
strategy: CancellationStrategy.Strategy): CancellationStrategy.Strategy =
|
strategy: CancellationStrategy.Strategy): CancellationStrategy.Strategy =
|
||||||
CancellationStrategy.AfterDelay(delay, strategy)
|
CancellationStrategy.AfterDelay(delay, strategy)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Nested materialization cancellation strategy provides a way to configure the cancellation behavior of stages that materialize a nested flow.
|
||||||
|
*
|
||||||
|
* When cancelled before materializing their nested flows, these stages can either immediately cancel (default behaviour) without materializing the nested flow
|
||||||
|
* or wait for the nested flow to materialize and then propagate the cancellation signal through it.
|
||||||
|
*
|
||||||
|
* This applies to [[akka.stream.scaladsl.FlowOps.flatMapPrefix]], [[akka.stream.scaladsl.Flow.futureFlow]] (and derivations such as [[akka.stream.scaladsl.Flow.lazyFutureFlow]]).
|
||||||
|
* These operators either delay the nested flow's materialization or wait for a future to complete before doing so,
|
||||||
|
* in this period of time they may receive a downstream cancellation signal. When this happens these operators will behave according to
|
||||||
|
* this [[Attribute]]: when set to true they will 'stash' the signal and later deliver it to the materialized nested flow
|
||||||
|
* , otherwise these stages will immediately cancel without materializing the nested flow.
|
||||||
|
*/
|
||||||
|
@ApiMayChange
|
||||||
|
class NestedMaterializationCancellationPolicy private[NestedMaterializationCancellationPolicy] (
|
||||||
|
val propagateToNestedMaterialization: Boolean)
|
||||||
|
extends MandatoryAttribute
|
||||||
|
|
||||||
|
@ApiMayChange
|
||||||
|
object NestedMaterializationCancellationPolicy {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A [[NestedMaterializationCancellationPolicy]] that configures graph stages
|
||||||
|
* delaying nested flow materialization to cancel immediately when downstream cancels before
|
||||||
|
* nested flow materialization.
|
||||||
|
* This applies to [[akka.stream.scaladsl.FlowOps.flatMapPrefix]], [[akka.stream.scaladsl.Flow.futureFlow]] and derived operators.
|
||||||
|
*/
|
||||||
|
val EagerCancellation = new NestedMaterializationCancellationPolicy(false)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A [[NestedMaterializationCancellationPolicy]] that configures graph stages
|
||||||
|
* delaying nested flow materialization to delay cancellation when downstream cancels before
|
||||||
|
* nested flow materialization. Once the nested flow is materialized it will be cancelled immediately.
|
||||||
|
* This applies to [[akka.stream.scaladsl.FlowOps.flatMapPrefix]], [[akka.stream.scaladsl.Flow.futureFlow]] and derived operators.
|
||||||
|
*/
|
||||||
|
val PropagateToNested = new NestedMaterializationCancellationPolicy(true)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Default [[NestedMaterializationCancellationPolicy]],
|
||||||
|
* please see [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.EagerCancellation()]] for details.
|
||||||
|
*/
|
||||||
|
val Default = EagerCancellation
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* JAVA API
|
||||||
|
* A [[NestedMaterializationCancellationPolicy]] that configures graph stages
|
||||||
|
* delaying nested flow materialization to cancel immediately when downstream cancels before
|
||||||
|
* nested flow materialization.
|
||||||
|
* This applies to [[akka.stream.scaladsl.FlowOps.flatMapPrefix]], [[akka.stream.scaladsl.Flow.futureFlow]] and derived operators.
|
||||||
|
*/
|
||||||
|
@ApiMayChange
|
||||||
|
def nestedMaterializationCancellationPolicyEagerCancellation(): NestedMaterializationCancellationPolicy =
|
||||||
|
NestedMaterializationCancellationPolicy.EagerCancellation
|
||||||
|
|
||||||
|
/**
|
||||||
|
* JAVA API
|
||||||
|
* A [[NestedMaterializationCancellationPolicy]] that configures graph stages
|
||||||
|
* delaying nested flow materialization to delay cancellation when downstream cancels before
|
||||||
|
* nested flow materialization. Once the nested flow is materialized it will be cancelled immediately.
|
||||||
|
* This applies to [[akka.stream.scaladsl.FlowOps.flatMapPrefix]], [[akka.stream.scaladsl.Flow.futureFlow]] and derived operators.
|
||||||
|
*/
|
||||||
|
@ApiMayChange
|
||||||
|
def nestedMaterializationCancellationPolicyPropagateToNested(): NestedMaterializationCancellationPolicy =
|
||||||
|
NestedMaterializationCancellationPolicy.PropagateToNested
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Default [[NestedMaterializationCancellationPolicy]],
|
||||||
|
* please see [[akka.stream.Attributes#nestedMaterializationCancellationPolicyEagerCancellation()]] for details.
|
||||||
|
*/
|
||||||
|
def nestedMaterializationCancellationPolicyDefault(): NestedMaterializationCancellationPolicy =
|
||||||
|
NestedMaterializationCancellationPolicy.Default
|
||||||
|
|
||||||
object LogLevels {
|
object LogLevels {
|
||||||
|
|
||||||
/** Use to disable logging on certain operations when configuring [[Attributes#logLevels]] */
|
/** Use to disable logging on certain operations when configuring [[Attributes#logLevels]] */
|
||||||
|
|
|
||||||
|
|
@ -27,7 +27,11 @@ import akka.util.OptionVal
|
||||||
override def initialAttributes: Attributes = DefaultAttributes.flatMapPrefix
|
override def initialAttributes: Attributes = DefaultAttributes.flatMapPrefix
|
||||||
|
|
||||||
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[M]) = {
|
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[M]) = {
|
||||||
val matPromise = Promise[M]()
|
val propagateToNestedMaterialization =
|
||||||
|
inheritedAttributes
|
||||||
|
.mandatoryAttribute[Attributes.NestedMaterializationCancellationPolicy]
|
||||||
|
.propagateToNestedMaterialization
|
||||||
|
val matPromise = Promise[M]
|
||||||
val logic = new GraphStageLogic(shape) with InHandler with OutHandler {
|
val logic = new GraphStageLogic(shape) with InHandler with OutHandler {
|
||||||
val accumulated = collection.mutable.Buffer.empty[In]
|
val accumulated = collection.mutable.Buffer.empty[In]
|
||||||
|
|
||||||
|
|
@ -90,7 +94,10 @@ import akka.util.OptionVal
|
||||||
|
|
||||||
override def onDownstreamFinish(cause: Throwable): Unit = {
|
override def onDownstreamFinish(cause: Throwable): Unit = {
|
||||||
subSink match {
|
subSink match {
|
||||||
case OptionVal.None => downstreamCause = OptionVal.Some(cause)
|
case OptionVal.None if propagateToNestedMaterialization => downstreamCause = OptionVal.Some(cause)
|
||||||
|
case OptionVal.None =>
|
||||||
|
matPromise.failure(new NeverMaterializedException(cause))
|
||||||
|
cancelStage(cause)
|
||||||
case OptionVal.Some(s) => s.cancel(cause)
|
case OptionVal.Some(s) => s.cancel(cause)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,147 @@
|
||||||
|
/*
|
||||||
|
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
|
||||||
|
*/
|
||||||
|
|
||||||
|
package akka.stream.impl.fusing
|
||||||
|
|
||||||
|
import akka.annotation.InternalApi
|
||||||
|
import akka.dispatch.ExecutionContexts
|
||||||
|
import akka.stream.{ AbruptStageTerminationException, Attributes, FlowShape, Inlet, NeverMaterializedException, Outlet }
|
||||||
|
import akka.stream.scaladsl.{ Flow, Keep, Source }
|
||||||
|
import akka.stream.stage.{ GraphStageLogic, GraphStageWithMaterializedValue, InHandler, OutHandler }
|
||||||
|
import akka.util.OptionVal
|
||||||
|
|
||||||
|
import scala.concurrent.{ Future, Promise }
|
||||||
|
import scala.util.{ Failure, Success, Try }
|
||||||
|
|
||||||
|
@InternalApi private[akka] final class FutureFlow[In, Out, M](futureFlow: Future[Flow[In, Out, M]])
|
||||||
|
extends GraphStageWithMaterializedValue[FlowShape[In, Out], Future[M]] {
|
||||||
|
val in = Inlet[In](s"${this}.in")
|
||||||
|
val out = Outlet[Out](s"${this}.out")
|
||||||
|
override val shape: FlowShape[In, Out] = FlowShape(in, out)
|
||||||
|
|
||||||
|
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[M]) = {
|
||||||
|
val propagateToNestedMaterialization =
|
||||||
|
inheritedAttributes
|
||||||
|
.mandatoryAttribute[Attributes.NestedMaterializationCancellationPolicy]
|
||||||
|
.propagateToNestedMaterialization
|
||||||
|
val innerMatValue = Promise[M]
|
||||||
|
val logic = new GraphStageLogic(shape) {
|
||||||
|
|
||||||
|
//seems like we must set handlers BEFORE preStart
|
||||||
|
setHandlers(in, out, Initializing)
|
||||||
|
|
||||||
|
override def preStart(): Unit = {
|
||||||
|
futureFlow.value match {
|
||||||
|
case Some(tryFlow) =>
|
||||||
|
Initializing.onFuture(tryFlow)
|
||||||
|
case None =>
|
||||||
|
val cb = getAsyncCallback(Initializing.onFuture)
|
||||||
|
futureFlow.onComplete(cb.invoke)(ExecutionContexts.parasitic)
|
||||||
|
//in case both ports are closed before future completion
|
||||||
|
setKeepGoing(true)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
override def postStop(): Unit = {
|
||||||
|
if (!innerMatValue.isCompleted) {
|
||||||
|
innerMatValue.failure(new AbruptStageTerminationException(this))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object Initializing extends InHandler with OutHandler {
|
||||||
|
// we don't expect a push since we bever pull upstream during initialization
|
||||||
|
override def onPush(): Unit = throw new IllegalStateException("unexpected push during initialization")
|
||||||
|
|
||||||
|
var upstreamFailure = OptionVal.none[Throwable]
|
||||||
|
|
||||||
|
override def onUpstreamFailure(ex: Throwable): Unit = {
|
||||||
|
upstreamFailure = OptionVal.Some(ex)
|
||||||
|
}
|
||||||
|
|
||||||
|
//will later be propagated to the materialized flow (by examining isClosed(in))
|
||||||
|
override def onUpstreamFinish(): Unit = {}
|
||||||
|
|
||||||
|
//will later be propagated to the materialized flow (by examining isAvailable(out))
|
||||||
|
override def onPull(): Unit = {}
|
||||||
|
|
||||||
|
var downstreamCause = OptionVal.none[Throwable]
|
||||||
|
|
||||||
|
override def onDownstreamFinish(cause: Throwable): Unit =
|
||||||
|
if (propagateToNestedMaterialization) {
|
||||||
|
downstreamCause = OptionVal.Some(cause)
|
||||||
|
} else {
|
||||||
|
innerMatValue.failure(new NeverMaterializedException(cause))
|
||||||
|
cancelStage(cause)
|
||||||
|
}
|
||||||
|
|
||||||
|
def onFuture(futureRes: Try[Flow[In, Out, M]]) = futureRes match {
|
||||||
|
case Failure(exception) =>
|
||||||
|
setKeepGoing(false)
|
||||||
|
innerMatValue.failure(new NeverMaterializedException(exception))
|
||||||
|
failStage(exception)
|
||||||
|
case Success(flow) =>
|
||||||
|
//materialize flow, connect inlet and outlet, feed with potential events and set handlers
|
||||||
|
connect(flow)
|
||||||
|
setKeepGoing(false)
|
||||||
|
}
|
||||||
|
|
||||||
|
def connect(flow: Flow[In, Out, M]): Unit = {
|
||||||
|
val subSource = new SubSourceOutlet[In](s"${FutureFlow.this}.subIn")
|
||||||
|
val subSink = new SubSinkInlet[Out](s"${FutureFlow.this}.subOut")
|
||||||
|
|
||||||
|
subSource.setHandler {
|
||||||
|
new OutHandler {
|
||||||
|
override def onPull(): Unit = if (!isClosed(in)) tryPull(in)
|
||||||
|
|
||||||
|
override def onDownstreamFinish(cause: Throwable): Unit = if (!isClosed(in)) cancel(in, cause)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
subSink.setHandler {
|
||||||
|
new InHandler {
|
||||||
|
override def onPush(): Unit = push(out, subSink.grab())
|
||||||
|
|
||||||
|
override def onUpstreamFinish(): Unit = complete(out)
|
||||||
|
|
||||||
|
override def onUpstreamFailure(ex: Throwable): Unit = fail(out, ex)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
Try {
|
||||||
|
Source.fromGraph(subSource.source).viaMat(flow)(Keep.right).to(subSink.sink).run()(subFusingMaterializer)
|
||||||
|
} match {
|
||||||
|
case Success(matVal) =>
|
||||||
|
innerMatValue.success(matVal)
|
||||||
|
upstreamFailure match {
|
||||||
|
case OptionVal.Some(ex) =>
|
||||||
|
subSource.fail(ex)
|
||||||
|
case OptionVal.None =>
|
||||||
|
if (isClosed(in))
|
||||||
|
subSource.complete()
|
||||||
|
}
|
||||||
|
downstreamCause match {
|
||||||
|
case OptionVal.Some(cause) =>
|
||||||
|
subSink.cancel(cause)
|
||||||
|
case OptionVal.None =>
|
||||||
|
if (isAvailable(out)) subSink.pull()
|
||||||
|
}
|
||||||
|
setHandlers(in, out, new InHandler with OutHandler {
|
||||||
|
override def onPull(): Unit = subSink.pull()
|
||||||
|
|
||||||
|
override def onDownstreamFinish(cause: Throwable): Unit = subSink.cancel(cause)
|
||||||
|
|
||||||
|
override def onPush(): Unit = subSource.push(grab(in))
|
||||||
|
|
||||||
|
override def onUpstreamFinish(): Unit = subSource.complete()
|
||||||
|
|
||||||
|
override def onUpstreamFailure(ex: Throwable): Unit = subSource.fail(ex)
|
||||||
|
})
|
||||||
|
case Failure(ex) =>
|
||||||
|
innerMatValue.failure(new NeverMaterializedException(ex))
|
||||||
|
failStage(ex)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
(logic, innerMatValue.future)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -9,7 +9,7 @@ import java.util.concurrent.TimeUnit.NANOSECONDS
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
import scala.collection.immutable.VectorBuilder
|
import scala.collection.immutable.VectorBuilder
|
||||||
import scala.concurrent.{ Future, Promise }
|
import scala.concurrent.Future
|
||||||
import scala.concurrent.duration.{ FiniteDuration, _ }
|
import scala.concurrent.duration.{ FiniteDuration, _ }
|
||||||
import scala.util.{ Failure, Success, Try }
|
import scala.util.{ Failure, Success, Try }
|
||||||
import scala.util.control.{ NoStackTrace, NonFatal }
|
import scala.util.control.{ NoStackTrace, NonFatal }
|
||||||
|
|
@ -19,7 +19,6 @@ import com.github.ghik.silencer.silent
|
||||||
|
|
||||||
import akka.actor.{ ActorRef, Terminated }
|
import akka.actor.{ ActorRef, Terminated }
|
||||||
import akka.annotation.{ DoNotInherit, InternalApi }
|
import akka.annotation.{ DoNotInherit, InternalApi }
|
||||||
import akka.dispatch.ExecutionContexts
|
|
||||||
import akka.event.{ LogMarker, LogSource, Logging, LoggingAdapter, MarkerLoggingAdapter }
|
import akka.event.{ LogMarker, LogSource, Logging, LoggingAdapter, MarkerLoggingAdapter }
|
||||||
import akka.event.Logging.LogLevel
|
import akka.event.Logging.LogLevel
|
||||||
import akka.stream.{ Supervision, _ }
|
import akka.stream.{ Supervision, _ }
|
||||||
|
|
@ -29,7 +28,7 @@ import akka.stream.OverflowStrategies._
|
||||||
import akka.stream.impl.{ ReactiveStreamsCompliance, Buffer => BufferImpl }
|
import akka.stream.impl.{ ReactiveStreamsCompliance, Buffer => BufferImpl }
|
||||||
import akka.stream.impl.Stages.DefaultAttributes
|
import akka.stream.impl.Stages.DefaultAttributes
|
||||||
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
|
import akka.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
|
||||||
import akka.stream.scaladsl.{ DelayStrategy, Flow, Keep, Source }
|
import akka.stream.scaladsl.{ DelayStrategy, Source }
|
||||||
import akka.stream.stage._
|
import akka.stream.stage._
|
||||||
import akka.util.OptionVal
|
import akka.util.OptionVal
|
||||||
import akka.util.unused
|
import akka.util.unused
|
||||||
|
|
@ -2224,199 +2223,3 @@ private[stream] object Collect {
|
||||||
override def toString = "StatefulMapConcat"
|
override def toString = "StatefulMapConcat"
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* INTERNAL API
|
|
||||||
*/
|
|
||||||
@InternalApi private[akka] final class LazyFlow[I, O, M](flowFactory: I => Future[Flow[I, O, M]])
|
|
||||||
extends GraphStageWithMaterializedValue[FlowShape[I, O], Future[M]] {
|
|
||||||
|
|
||||||
// FIXME: when removing the deprecated I => Flow factories we can remove that complication from this stage
|
|
||||||
|
|
||||||
val in = Inlet[I]("LazyFlow.in")
|
|
||||||
val out = Outlet[O]("LazyFlow.out")
|
|
||||||
|
|
||||||
override def initialAttributes = DefaultAttributes.lazyFlow
|
|
||||||
|
|
||||||
override val shape: FlowShape[I, O] = FlowShape.of(in, out)
|
|
||||||
|
|
||||||
override def toString: String = "LazyFlow"
|
|
||||||
|
|
||||||
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[M]) = {
|
|
||||||
val matPromise = Promise[M]()
|
|
||||||
val stageLogic = new GraphStageLogic(shape) with InHandler with OutHandler {
|
|
||||||
var switching = false
|
|
||||||
|
|
||||||
//
|
|
||||||
// implementation of handler methods in initial state
|
|
||||||
//
|
|
||||||
private def onFlowFutureComplete(firstElement: I)(result: Try[Flow[I, O, M]]) = result match {
|
|
||||||
case Success(flow) =>
|
|
||||||
// check if the stage is still in need for the lazy flow
|
|
||||||
// (there could have been an onUpstreamFailure or onDownstreamFinish in the meantime that has completed the promise)
|
|
||||||
if (!matPromise.isCompleted) {
|
|
||||||
try {
|
|
||||||
val mat = switchTo(flow, firstElement)
|
|
||||||
matPromise.success(mat)
|
|
||||||
} catch {
|
|
||||||
case NonFatal(e) =>
|
|
||||||
matPromise.failure(e)
|
|
||||||
failStage(e)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
case Failure(e) =>
|
|
||||||
matPromise.failure(e)
|
|
||||||
failStage(e)
|
|
||||||
}
|
|
||||||
|
|
||||||
override def onPush(): Unit =
|
|
||||||
try {
|
|
||||||
val element = grab(in)
|
|
||||||
switching = true
|
|
||||||
val futureFlow = flowFactory(element)
|
|
||||||
|
|
||||||
// optimization avoid extra scheduling if already completed
|
|
||||||
futureFlow.value match {
|
|
||||||
case Some(completed) =>
|
|
||||||
onFlowFutureComplete(element)(completed)
|
|
||||||
case None =>
|
|
||||||
val cb = getAsyncCallback[Try[Flow[I, O, M]]](onFlowFutureComplete(element))
|
|
||||||
futureFlow.onComplete(cb.invoke)(ExecutionContexts.parasitic)
|
|
||||||
}
|
|
||||||
} catch {
|
|
||||||
case NonFatal(e) =>
|
|
||||||
matPromise.failure(e)
|
|
||||||
failStage(e)
|
|
||||||
}
|
|
||||||
|
|
||||||
override def onUpstreamFinish(): Unit = {
|
|
||||||
if (!matPromise.isCompleted)
|
|
||||||
matPromise.tryFailure(new NeverMaterializedException)
|
|
||||||
// ignore onUpstreamFinish while the stage is switching but setKeepGoing
|
|
||||||
if (switching) {
|
|
||||||
setKeepGoing(true)
|
|
||||||
} else {
|
|
||||||
super.onUpstreamFinish()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
override def onUpstreamFailure(ex: Throwable): Unit = {
|
|
||||||
super.onUpstreamFailure(ex)
|
|
||||||
if (!matPromise.isCompleted)
|
|
||||||
matPromise.tryFailure(new NeverMaterializedException(ex))
|
|
||||||
}
|
|
||||||
|
|
||||||
override def onPull(): Unit = {
|
|
||||||
pull(in)
|
|
||||||
}
|
|
||||||
|
|
||||||
override def postStop(): Unit = {
|
|
||||||
if (!matPromise.isCompleted)
|
|
||||||
matPromise.tryFailure(new AbruptStageTerminationException(this))
|
|
||||||
}
|
|
||||||
|
|
||||||
setHandler(in, this)
|
|
||||||
setHandler(out, this)
|
|
||||||
|
|
||||||
private def switchTo(flow: Flow[I, O, M], firstElement: I): M = {
|
|
||||||
|
|
||||||
//
|
|
||||||
// ports are wired in the following way:
|
|
||||||
//
|
|
||||||
// in ~> subOutlet ~> lazyFlow ~> subInlet ~> out
|
|
||||||
//
|
|
||||||
|
|
||||||
val subInlet = new SubSinkInlet[O]("LazyFlowSubSink")
|
|
||||||
val subOutlet = new SubSourceOutlet[I]("LazyFlowSubSource")
|
|
||||||
|
|
||||||
val matVal = Source
|
|
||||||
.fromGraph(subOutlet.source)
|
|
||||||
.prepend(Source.single(firstElement))
|
|
||||||
.viaMat(flow)(Keep.right)
|
|
||||||
.toMat(subInlet.sink)(Keep.left)
|
|
||||||
.run()(interpreter.subFusingMaterializer)
|
|
||||||
|
|
||||||
// The lazily materialized flow may be constructed from a sink and a source. Therefore termination
|
|
||||||
// signals (completion, cancellation, and errors) are not guaranteed to pass through the flow. This
|
|
||||||
// means that this stage must not be completed as soon as one side of the flow is finished.
|
|
||||||
//
|
|
||||||
// Invariant: isClosed(out) == subInlet.isClosed after each event because termination signals (i.e.
|
|
||||||
// completion, cancellation, and failure) between these two ports are always forwarded.
|
|
||||||
//
|
|
||||||
// However, isClosed(in) and subOutlet.isClosed may be different. This happens if upstream completes before
|
|
||||||
// the cached element was pushed.
|
|
||||||
def maybeCompleteStage(): Unit = {
|
|
||||||
if (isClosed(in) && subOutlet.isClosed && isClosed(out)) {
|
|
||||||
completeStage()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// The stage must not be shut down automatically; it is completed when maybeCompleteStage decides
|
|
||||||
setKeepGoing(true)
|
|
||||||
|
|
||||||
setHandler(
|
|
||||||
in,
|
|
||||||
new InHandler {
|
|
||||||
override def onPush(): Unit = {
|
|
||||||
subOutlet.push(grab(in))
|
|
||||||
}
|
|
||||||
override def onUpstreamFinish(): Unit = {
|
|
||||||
subOutlet.complete()
|
|
||||||
maybeCompleteStage()
|
|
||||||
}
|
|
||||||
override def onUpstreamFailure(ex: Throwable): Unit = {
|
|
||||||
// propagate exception irrespective if the cached element has been pushed or not
|
|
||||||
subOutlet.fail(ex)
|
|
||||||
maybeCompleteStage()
|
|
||||||
}
|
|
||||||
})
|
|
||||||
|
|
||||||
setHandler(out, new OutHandler {
|
|
||||||
override def onPull(): Unit = {
|
|
||||||
subInlet.pull()
|
|
||||||
}
|
|
||||||
override def onDownstreamFinish(cause: Throwable): Unit = {
|
|
||||||
subInlet.cancel(cause)
|
|
||||||
maybeCompleteStage()
|
|
||||||
}
|
|
||||||
})
|
|
||||||
|
|
||||||
subOutlet.setHandler(new OutHandler {
|
|
||||||
override def onPull(): Unit = {
|
|
||||||
pull(in)
|
|
||||||
}
|
|
||||||
override def onDownstreamFinish(cause: Throwable): Unit = {
|
|
||||||
if (!isClosed(in)) {
|
|
||||||
cancel(in, cause)
|
|
||||||
}
|
|
||||||
maybeCompleteStage()
|
|
||||||
}
|
|
||||||
})
|
|
||||||
|
|
||||||
subInlet.setHandler(new InHandler {
|
|
||||||
override def onPush(): Unit = {
|
|
||||||
push(out, subInlet.grab())
|
|
||||||
}
|
|
||||||
override def onUpstreamFinish(): Unit = {
|
|
||||||
complete(out)
|
|
||||||
maybeCompleteStage()
|
|
||||||
}
|
|
||||||
override def onUpstreamFailure(ex: Throwable): Unit = {
|
|
||||||
fail(out, ex)
|
|
||||||
maybeCompleteStage()
|
|
||||||
}
|
|
||||||
})
|
|
||||||
|
|
||||||
if (isClosed(out)) {
|
|
||||||
// downstream may have been canceled while the stage was switching
|
|
||||||
subInlet.cancel()
|
|
||||||
} else {
|
|
||||||
subInlet.pull()
|
|
||||||
}
|
|
||||||
|
|
||||||
matVal
|
|
||||||
}
|
|
||||||
}
|
|
||||||
(stageLogic, matPromise.future)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
|
||||||
|
|
@ -6,7 +6,6 @@ package akka.stream.javadsl
|
||||||
|
|
||||||
import java.util.Comparator
|
import java.util.Comparator
|
||||||
import java.util.Optional
|
import java.util.Optional
|
||||||
import java.util.concurrent.CompletableFuture
|
|
||||||
import java.util.concurrent.CompletionStage
|
import java.util.concurrent.CompletionStage
|
||||||
import java.util.function.BiFunction
|
import java.util.function.BiFunction
|
||||||
import java.util.function.Supplier
|
import java.util.function.Supplier
|
||||||
|
|
@ -30,7 +29,6 @@ import akka.japi.Util
|
||||||
import akka.japi.function
|
import akka.japi.function
|
||||||
import akka.japi.function.Creator
|
import akka.japi.function.Creator
|
||||||
import akka.stream._
|
import akka.stream._
|
||||||
import akka.stream.impl.fusing.LazyFlow
|
|
||||||
import akka.util.ConstantFun
|
import akka.util.ConstantFun
|
||||||
import akka.util.JavaDurationConverters._
|
import akka.util.JavaDurationConverters._
|
||||||
import akka.util.Timeout
|
import akka.util.Timeout
|
||||||
|
|
@ -263,9 +261,9 @@ object Flow {
|
||||||
flowFactory: function.Function[I, CompletionStage[Flow[I, O, M]]],
|
flowFactory: function.Function[I, CompletionStage[Flow[I, O, M]]],
|
||||||
fallback: function.Creator[M]): Flow[I, O, M] = {
|
fallback: function.Creator[M]): Flow[I, O, M] = {
|
||||||
import scala.compat.java8.FutureConverters._
|
import scala.compat.java8.FutureConverters._
|
||||||
val sflow = scaladsl.Flow
|
val sflow = scaladsl.Flow.lazyInit(
|
||||||
.fromGraph(new LazyFlow[I, O, M](t => flowFactory.apply(t).toScala.map(_.asScala)(ExecutionContexts.parasitic)))
|
(flowFactory.apply(_)).andThen(_.toScala.map(_.asScala)(ExecutionContexts.parasitic)),
|
||||||
.mapMaterializedValue(_ => fallback.create())
|
fallback.create _)
|
||||||
new Flow(sflow)
|
new Flow(sflow)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -304,8 +302,12 @@ object Flow {
|
||||||
* The materialized completion stage value is completed with the materialized value of the future flow or failed with a
|
* The materialized completion stage value is completed with the materialized value of the future flow or failed with a
|
||||||
* [[NeverMaterializedException]] if upstream fails or downstream cancels before the completion stage has completed.
|
* [[NeverMaterializedException]] if upstream fails or downstream cancels before the completion stage has completed.
|
||||||
*/
|
*/
|
||||||
def completionStageFlow[I, O, M](flow: CompletionStage[Flow[I, O, M]]): Flow[I, O, CompletionStage[M]] =
|
def completionStageFlow[I, O, M](flow: CompletionStage[Flow[I, O, M]]): Flow[I, O, CompletionStage[M]] = {
|
||||||
lazyCompletionStageFlow(() => flow)
|
import scala.compat.java8.FutureConverters._
|
||||||
|
val sflow =
|
||||||
|
scaladsl.Flow.futureFlow(flow.toScala.map(_.asScala)(ExecutionContexts.parasitic)).mapMaterializedValue(_.toJava)
|
||||||
|
new javadsl.Flow(sflow)
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Defers invoking the `create` function to create a future flow until there is downstream demand and passing
|
* Defers invoking the `create` function to create a future flow until there is downstream demand and passing
|
||||||
|
|
@ -322,8 +324,15 @@ object Flow {
|
||||||
*
|
*
|
||||||
* '''Cancels when''' downstream cancels
|
* '''Cancels when''' downstream cancels
|
||||||
*/
|
*/
|
||||||
def lazyFlow[I, O, M](create: Creator[Flow[I, O, M]]): Flow[I, O, CompletionStage[M]] =
|
def lazyFlow[I, O, M](create: Creator[Flow[I, O, M]]): Flow[I, O, CompletionStage[M]] = {
|
||||||
lazyCompletionStageFlow(() => CompletableFuture.completedFuture(create.create()))
|
import scala.compat.java8.FutureConverters._
|
||||||
|
val sflow = scaladsl.Flow
|
||||||
|
.lazyFlow { () =>
|
||||||
|
create.create().asScala
|
||||||
|
}
|
||||||
|
.mapMaterializedValue(_.toJava)
|
||||||
|
new javadsl.Flow(sflow)
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Defers invoking the `create` function to create a future flow until there downstream demand has caused upstream
|
* Defers invoking the `create` function to create a future flow until there downstream demand has caused upstream
|
||||||
|
|
|
||||||
|
|
@ -578,13 +578,22 @@ object Flow {
|
||||||
*
|
*
|
||||||
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
|
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
|
||||||
*
|
*
|
||||||
* '''Cancels when''' downstream cancels
|
* '''Cancels when''' downstream cancels (see below)
|
||||||
|
*
|
||||||
|
* The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
|
||||||
|
* This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
|
||||||
|
* this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
|
||||||
*/
|
*/
|
||||||
@deprecated(
|
@deprecated(
|
||||||
"Use 'Flow.futureFlow' in combination with prefixAndTail(1) instead, see `futureFlow` operator docs for details",
|
"Use 'Flow.futureFlow' in combination with prefixAndTail(1) instead, see `futureFlow` operator docs for details",
|
||||||
"2.6.0")
|
"2.6.0")
|
||||||
def lazyInit[I, O, M](flowFactory: I => Future[Flow[I, O, M]], fallback: () => M): Flow[I, O, M] =
|
def lazyInit[I, O, M](flowFactory: I => Future[Flow[I, O, M]], fallback: () => M): Flow[I, O, M] =
|
||||||
Flow.fromGraph(new LazyFlow[I, O, M](flowFactory)).mapMaterializedValue(_ => fallback())
|
Flow[I]
|
||||||
|
.flatMapPrefix(1) {
|
||||||
|
case Seq(a) => futureFlow(flowFactory(a)).mapMaterializedValue(_ => NotUsed)
|
||||||
|
case Nil => Flow[I].asInstanceOf[Flow[I, O, NotUsed]]
|
||||||
|
}
|
||||||
|
.mapMaterializedValue(_ => fallback())
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a real `Flow` upon receiving the first element. Internal `Flow` will not be created
|
* Creates a real `Flow` upon receiving the first element. Internal `Flow` will not be created
|
||||||
|
|
@ -600,13 +609,17 @@ object Flow {
|
||||||
*
|
*
|
||||||
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
|
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
|
||||||
*
|
*
|
||||||
* '''Cancels when''' downstream cancels
|
* '''Cancels when''' downstream cancels (see below)
|
||||||
|
*
|
||||||
|
* The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
|
||||||
|
* This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
|
||||||
|
* this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
|
||||||
*/
|
*/
|
||||||
@deprecated("Use 'Flow.lazyFutureFlow' instead", "2.6.0")
|
@deprecated("Use 'Flow.lazyFutureFlow' instead", "2.6.0")
|
||||||
def lazyInitAsync[I, O, M](flowFactory: () => Future[Flow[I, O, M]]): Flow[I, O, Future[Option[M]]] =
|
def lazyInitAsync[I, O, M](flowFactory: () => Future[Flow[I, O, M]]): Flow[I, O, Future[Option[M]]] =
|
||||||
Flow.fromGraph(new LazyFlow[I, O, M](_ => flowFactory())).mapMaterializedValue { v =>
|
Flow.lazyFutureFlow(flowFactory).mapMaterializedValue {
|
||||||
implicit val ec = akka.dispatch.ExecutionContexts.parasitic
|
implicit val ec = akka.dispatch.ExecutionContexts.parasitic
|
||||||
v.map[Option[M]](Some.apply _).recover { case _: NeverMaterializedException => None }
|
_.map(Some.apply).recover { case _: NeverMaterializedException => None }
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -615,9 +628,13 @@ object Flow {
|
||||||
*
|
*
|
||||||
* The materialized future value is completed with the materialized value of the future flow or failed with a
|
* The materialized future value is completed with the materialized value of the future flow or failed with a
|
||||||
* [[NeverMaterializedException]] if upstream fails or downstream cancels before the future has completed.
|
* [[NeverMaterializedException]] if upstream fails or downstream cancels before the future has completed.
|
||||||
|
*
|
||||||
|
* The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
|
||||||
|
* This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
|
||||||
|
* this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
|
||||||
*/
|
*/
|
||||||
def futureFlow[I, O, M](flow: Future[Flow[I, O, M]]): Flow[I, O, Future[M]] =
|
def futureFlow[I, O, M](flow: Future[Flow[I, O, M]]): Flow[I, O, Future[M]] =
|
||||||
lazyFutureFlow(() => flow)
|
Flow.fromGraph(new FutureFlow(flow))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Defers invoking the `create` function to create a future flow until there is downstream demand and passing
|
* Defers invoking the `create` function to create a future flow until there is downstream demand and passing
|
||||||
|
|
@ -638,7 +655,11 @@ object Flow {
|
||||||
*
|
*
|
||||||
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
|
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
|
||||||
*
|
*
|
||||||
* '''Cancels when''' downstream cancels
|
* '''Cancels when''' downstream cancels (see below)
|
||||||
|
*
|
||||||
|
* The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
|
||||||
|
* This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
|
||||||
|
* this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
|
||||||
*/
|
*/
|
||||||
def lazyFlow[I, O, M](create: () => Flow[I, O, M]): Flow[I, O, Future[M]] =
|
def lazyFlow[I, O, M](create: () => Flow[I, O, M]): Flow[I, O, Future[M]] =
|
||||||
lazyFutureFlow(() => Future.successful(create()))
|
lazyFutureFlow(() => Future.successful(create()))
|
||||||
|
|
@ -662,10 +683,27 @@ object Flow {
|
||||||
*
|
*
|
||||||
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
|
* '''Completes when''' upstream completes and all elements have been emitted from the internal flow
|
||||||
*
|
*
|
||||||
* '''Cancels when''' downstream cancels
|
* '''Cancels when''' downstream cancels (see below)
|
||||||
|
*
|
||||||
|
* The operator's default behaviour in case of downstream cancellation before nested flow materialization (future completion) is to cancel immediately.
|
||||||
|
* This behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy.PropagateToNested]] attribute,
|
||||||
|
* this will delay downstream cancellation until nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
|
||||||
*/
|
*/
|
||||||
def lazyFutureFlow[I, O, M](create: () => Future[Flow[I, O, M]]): Flow[I, O, Future[M]] =
|
def lazyFutureFlow[I, O, M](create: () => Future[Flow[I, O, M]]): Flow[I, O, Future[M]] =
|
||||||
Flow.fromGraph(new LazyFlow(_ => create()))
|
Flow[I]
|
||||||
|
.flatMapPrefixMat(1) {
|
||||||
|
case Seq(a) =>
|
||||||
|
val f: Flow[I, O, Future[M]] =
|
||||||
|
futureFlow(create()
|
||||||
|
.map(Flow[I].prepend(Source.single(a)).viaMat(_)(Keep.right))(akka.dispatch.ExecutionContexts.parasitic))
|
||||||
|
f
|
||||||
|
case Nil =>
|
||||||
|
val f: Flow[I, O, Future[M]] = Flow[I]
|
||||||
|
.asInstanceOf[Flow[I, O, NotUsed]]
|
||||||
|
.mapMaterializedValue(_ => Future.failed[M](new NeverMaterializedException()))
|
||||||
|
f
|
||||||
|
}(Keep.right)
|
||||||
|
.mapMaterializedValue(_.flatten)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -1945,7 +1983,9 @@ trait FlowOps[+Out, +Mat] {
|
||||||
* the resulting flow will be materialized and signalled for upstream completion, it can then complete or continue to emit elements at its own discretion.
|
* the resulting flow will be materialized and signalled for upstream completion, it can then complete or continue to emit elements at its own discretion.
|
||||||
*
|
*
|
||||||
* '''Cancels when''' the materialized flow cancels.
|
* '''Cancels when''' the materialized flow cancels.
|
||||||
* Notice that when downstream cancels prior to prefix completion, the cancellation cause is stashed until prefix completion (or upstream completion) and then handed to the materialized flow.
|
* When downstream cancels before materialization of the nested flow, the operator's default behaviour is to cancel immediately,
|
||||||
|
* this behaviour can be controlled by setting the [[akka.stream.Attributes.NestedMaterializationCancellationPolicy]] attribute on the flow.
|
||||||
|
* When this attribute is configured to true, downstream cancellation is delayed until the nested flow's materialization which is then immediately cancelled (with the original cancellation cause).
|
||||||
*
|
*
|
||||||
* @param n the number of elements to accumulate before materializing the downstream flow.
|
* @param n the number of elements to accumulate before materializing the downstream flow.
|
||||||
* @param f a function that produces the downstream flow based on the upstream's prefix.
|
* @param f a function that produces the downstream flow based on the upstream's prefix.
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue