* 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:
parent
770b3a3474
commit
3db145643a
12 changed files with 454 additions and 383 deletions
|
|
@ -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
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue