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:
eyal farago 2020-05-14 11:28:53 +03:00 committed by GitHub
parent 9a1d5191b9
commit de59bb6803
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
15 changed files with 1435 additions and 739 deletions

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -0,0 +1,2 @@
# Changes to internals
ProblemFilters.exclude[MissingClassProblem]("akka.stream.impl.fusing.LazyFlow")

View file

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

View file

@ -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]] */

View file

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

View file

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

View file

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

View file

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

View file

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