materialized value of Flow.lazyInit must be a Future #24670 (#24685)

* change materialized value of LazyFlow from [M] to Future[Option[M]]

* remove whitespace

* improve docu

* restore old Flow.lazyInit method and add new Flow.lazyInitAsync method

* fix deprecation messages

* add 2.5.11.backwards.excludes because of changed LazyFlow constructor signature

* check switching behaviour

* apply formatting

* improve deprecation message; improve null safety

* prevent premature stage completion by setting keepGoing

* deprecate Sink.lazyInit; add Sink.lazyInitAsync

* apply formatting

* add ProblemFilter.exclude for changed LazySink.this

* Update Sink.scala

* Update Sink.scala

* Update Flow.scala

* Update Flow.scala
This commit is contained in:
Stefan Wachter 2018-03-19 14:42:37 +01:00 committed by Konrad `ktoso` Malawski
parent 770b3a3474
commit 3db145643a
12 changed files with 454 additions and 383 deletions

View file

@ -4,38 +4,31 @@
package akka.stream.impl
import akka.dispatch.ExecutionContexts
import akka.stream.ActorAttributes.SupervisionStrategy
import akka.stream.impl.QueueSink.{ Output, Pull }
import java.util.Optional
import java.util.concurrent.CompletionStage
import akka.NotUsed
import akka.actor.{ ActorRef, Props }
import akka.annotation.{ DoNotInherit, InternalApi }
import akka.dispatch.ExecutionContexts
import akka.event.Logging
import akka.stream.Attributes.InputBuffer
import akka.stream._
import akka.stream.impl.QueueSink.{ Output, Pull }
import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.impl.StreamLayout.AtomicModule
import akka.actor.{ ActorRef, Props }
import akka.stream.Attributes.InputBuffer
import akka.stream._
import akka.stream.scaladsl.{ Sink, SinkQueueWithCancel, Source }
import akka.stream.stage._
import org.reactivestreams.{ Publisher, Subscriber }
import scala.annotation.unchecked.uncheckedVariance
import scala.collection.generic.CanBuildFrom
import scala.collection.immutable
import scala.compat.java8.FutureConverters._
import scala.compat.java8.OptionConverters._
import scala.concurrent.{ Future, Promise }
import scala.util.control.NonFatal
import scala.util.{ Failure, Success, Try }
import akka.stream.scaladsl.{ Sink, SinkQueueWithCancel, Source }
import java.util.concurrent.CompletionStage
import scala.compat.java8.FutureConverters._
import scala.compat.java8.OptionConverters._
import java.util.Optional
import akka.annotation.{ DoNotInherit, InternalApi }
import akka.event.Logging
import scala.collection.generic.CanBuildFrom
/**
* INTERNAL API
@ -456,7 +449,7 @@ import scala.collection.generic.CanBuildFrom
/**
* INTERNAL API
*/
@InternalApi final private[stream] class LazySink[T, M](sinkFactory: T Future[Sink[T, M]], zeroMat: () M) extends GraphStageWithMaterializedValue[SinkShape[T], Future[M]] {
@InternalApi final private[stream] class LazySink[T, M](sinkFactory: T Future[Sink[T, M]]) extends GraphStageWithMaterializedValue[SinkShape[T], Future[Option[M]]] {
val in = Inlet[T]("lazySink.in")
override def initialAttributes = DefaultAttributes.lazySink
override val shape: SinkShape[T] = SinkShape.of(in)
@ -464,102 +457,122 @@ import scala.collection.generic.CanBuildFrom
override def toString: String = "LazySink"
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
lazy val decider = inheritedAttributes.mandatoryAttribute[SupervisionStrategy].decider
var completed = false
val promise = Promise[M]()
val promise = Promise[Option[M]]()
val stageLogic = new GraphStageLogic(shape) with InHandler {
var switching = false
override def preStart(): Unit = pull(in)
override def onPush(): Unit = {
try {
val element = grab(in)
val cb: AsyncCallback[Try[Sink[T, M]]] =
getAsyncCallback {
case Success(sink) initInternalSource(sink, element)
case Failure(e) failure(e)
}
sinkFactory(element).onComplete { cb.invoke }(ExecutionContexts.sameThreadExecutionContext)
setHandler(in, new InHandler {
override def onPush(): Unit = ()
override def onUpstreamFinish(): Unit = gotCompletionEvent()
override def onUpstreamFailure(ex: Throwable): Unit = failure(ex)
})
} catch {
case NonFatal(e) decider(e) match {
case Supervision.Stop failure(e)
case _ pull(in)
val element = grab(in)
switching = true
val cb: AsyncCallback[Try[Sink[T, M]]] =
getAsyncCallback {
case Success(sink)
// check if the stage is still in need for the lazy sink
// (there could have been an onUpstreamFailure in the meantime that has completed the promise)
if (!promise.isCompleted) {
try {
val mat = switchTo(sink, element)
promise.success(Some(mat))
setKeepGoing(true)
} catch {
case NonFatal(e)
promise.failure(e)
failStage(e)
}
}
case Failure(e)
promise.failure(e)
failStage(e)
}
try {
sinkFactory(element).onComplete(cb.invoke)(ExecutionContexts.sameThreadExecutionContext)
} catch {
case NonFatal(e)
promise.failure(e)
failStage(e)
}
}
private def failure(ex: Throwable): Unit = {
failStage(ex)
promise.failure(ex)
}
override def onUpstreamFinish(): Unit = {
completeStage()
promise.tryComplete(Try(zeroMat()))
// ignore onUpstreamFinish while the stage is switching but setKeepGoing
//
if (switching) {
// there is a cached element -> the stage must not be shut down automatically because isClosed(in) is satisfied
setKeepGoing(true)
} else {
promise.success(None)
super.onUpstreamFinish()
}
}
override def onUpstreamFailure(ex: Throwable): Unit = failure(ex)
override def onUpstreamFailure(ex: Throwable): Unit = {
promise.failure(ex)
super.onUpstreamFailure(ex)
}
setHandler(in, this)
private def gotCompletionEvent(): Unit = {
private def switchTo(sink: Sink[T, M], firstElement: T): M = {
var firstElementPushed = false
val subOutlet = new SubSourceOutlet[T]("LazySink")
val matVal = Source.fromGraph(subOutlet.source).runWith(sink)(interpreter.subFusingMaterializer)
def maybeCompleteStage(): Unit = {
if (isClosed(in) && subOutlet.isClosed) {
completeStage()
}
}
// The stage must not be shut down automatically; it is completed when maybeCompleteStage decides
setKeepGoing(true)
completed = true
}
private def initInternalSource(sink: Sink[T, M], firstElement: T): Unit = {
val sourceOut = new SubSourceOutlet[T]("LazySink")
def switchToFirstElementHandlers(): Unit = {
sourceOut.setHandler(new OutHandler {
override def onPull(): Unit = {
sourceOut.push(firstElement)
if (completed) internalSourceComplete() else switchToFinalHandlers()
setHandler(in, new InHandler {
override def onPush(): Unit = {
subOutlet.push(grab(in))
}
override def onUpstreamFinish(): Unit = {
if (firstElementPushed) {
subOutlet.complete()
maybeCompleteStage()
}
override def onDownstreamFinish(): Unit = internalSourceComplete()
})
}
override def onUpstreamFailure(ex: Throwable): Unit = {
// propagate exception irrespective if the cached element has been pushed or not
subOutlet.fail(ex)
maybeCompleteStage()
}
})
setHandler(in, new InHandler {
override def onPush(): Unit = sourceOut.push(grab(in))
override def onUpstreamFinish(): Unit = gotCompletionEvent()
override def onUpstreamFailure(ex: Throwable): Unit = internalSourceFailure(ex)
})
}
subOutlet.setHandler(new OutHandler {
override def onPull(): Unit = {
if (firstElementPushed) {
pull(in)
} else {
// the demand can be satisfied right away by the cached element
firstElementPushed = true
subOutlet.push(firstElement)
// in.onUpstreamFinished was not propagated if it arrived before the cached element was pushed
// -> check if the completion must be propagated now
if (isClosed(in)) {
subOutlet.complete()
maybeCompleteStage()
}
}
}
override def onDownstreamFinish(): Unit = {
if (!isClosed(in)) {
cancel(in)
}
maybeCompleteStage()
}
})
def switchToFinalHandlers(): Unit = {
sourceOut.setHandler(new OutHandler {
override def onPull(): Unit = pull(in)
override def onDownstreamFinish(): Unit = internalSourceComplete()
})
setHandler(in, new InHandler {
override def onPush(): Unit = sourceOut.push(grab(in))
override def onUpstreamFinish(): Unit = internalSourceComplete()
override def onUpstreamFailure(ex: Throwable): Unit = internalSourceFailure(ex)
})
}
def internalSourceComplete(): Unit = {
sourceOut.complete()
completeStage()
}
def internalSourceFailure(ex: Throwable): Unit = {
sourceOut.fail(ex)
failStage(ex)
}
switchToFirstElementHandlers()
try {
val matVal = Source.fromGraph(sourceOut.source).runWith(sink)(interpreter.subFusingMaterializer)
promise.trySuccess(matVal)
} catch {
case NonFatal(ex)
promise.tryFailure(ex)
failStage(ex)
}
matVal
}
}