2018-10-29 17:19:37 +08:00
|
|
|
/*
|
2021-01-08 17:55:38 +01:00
|
|
|
* Copyright (C) 2014-2021 Lightbend Inc. <https://www.lightbend.com>
|
2015-01-28 14:19:50 +01:00
|
|
|
*/
|
2018-03-13 23:45:55 +09:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
package akka.stream.impl
|
|
|
|
|
|
2019-07-09 13:58:26 +02:00
|
|
|
import java.util.function.BinaryOperator
|
2020-04-27 20:32:18 +08:00
|
|
|
import scala.collection.immutable
|
|
|
|
|
import scala.collection.mutable
|
|
|
|
|
import scala.concurrent.Future
|
|
|
|
|
import scala.concurrent.Promise
|
|
|
|
|
import scala.util.Failure
|
|
|
|
|
import scala.util.Success
|
|
|
|
|
import scala.util.Try
|
|
|
|
|
import scala.util.control.NonFatal
|
|
|
|
|
import org.reactivestreams.Publisher
|
|
|
|
|
import org.reactivestreams.Subscriber
|
2018-03-16 19:08:29 +08:00
|
|
|
import akka.NotUsed
|
2018-05-14 12:11:20 +01:00
|
|
|
import akka.annotation.DoNotInherit
|
|
|
|
|
import akka.annotation.InternalApi
|
2018-03-19 14:42:37 +01:00
|
|
|
import akka.dispatch.ExecutionContexts
|
|
|
|
|
import akka.event.Logging
|
2020-04-27 20:32:18 +08:00
|
|
|
import akka.stream._
|
2019-09-04 13:37:06 +02:00
|
|
|
import akka.stream.ActorAttributes.StreamSubscriptionTimeout
|
2015-12-04 09:37:32 -05:00
|
|
|
import akka.stream.Attributes.InputBuffer
|
2021-01-11 15:17:50 +01:00
|
|
|
import akka.stream.Attributes.SourceLocation
|
2018-05-14 12:11:20 +01:00
|
|
|
import akka.stream.impl.QueueSink.Output
|
|
|
|
|
import akka.stream.impl.QueueSink.Pull
|
2016-02-12 01:36:21 +08:00
|
|
|
import akka.stream.impl.Stages.DefaultAttributes
|
2016-03-11 17:08:30 +01:00
|
|
|
import akka.stream.impl.StreamLayout.AtomicModule
|
2021-06-02 03:48:14 -07:00
|
|
|
import akka.stream.scaladsl.{ Keep, Sink, SinkQueueWithCancel, Source }
|
2016-01-16 12:17:19 -05:00
|
|
|
import akka.stream.stage._
|
2018-11-22 16:18:10 +01:00
|
|
|
import akka.util.ccompat._
|
2019-09-04 13:37:06 +02:00
|
|
|
|
2015-03-05 12:21:17 +01:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
@DoNotInherit private[akka] abstract class SinkModule[-In, Mat](val shape: SinkShape[In])
|
|
|
|
|
extends AtomicModule[SinkShape[In], Mat] {
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2016-02-24 11:55:28 +01:00
|
|
|
/**
|
|
|
|
|
* Create the Subscriber or VirtualPublisher that consumes the incoming
|
|
|
|
|
* stream, plus the materialized value. Since Subscriber and VirtualPublisher
|
|
|
|
|
* do not share a common supertype apart from AnyRef this is what the type
|
|
|
|
|
* union devolves into; unfortunately we do not have union types at our
|
|
|
|
|
* disposal at this point.
|
|
|
|
|
*/
|
|
|
|
|
def create(context: MaterializationContext): (AnyRef, Mat)
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2017-03-07 14:26:18 +01:00
|
|
|
def attributes: Attributes
|
|
|
|
|
|
2016-07-27 13:29:23 +02:00
|
|
|
override def traversalBuilder: TraversalBuilder =
|
2017-03-07 14:26:18 +01:00
|
|
|
LinearTraversalBuilder.fromModule(this, attributes).makeIsland(SinkModuleIslandTag)
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2015-07-06 22:00:21 +02:00
|
|
|
protected def amendShape(attr: Attributes): SinkShape[In] = {
|
2016-07-27 13:29:23 +02:00
|
|
|
val thisN = traversalBuilder.attributes.nameOrDefault(null)
|
2015-07-06 22:00:21 +02:00
|
|
|
val thatN = attr.nameOrDefault(null)
|
|
|
|
|
|
|
|
|
|
if ((thatN eq null) || thisN == thatN) shape
|
2015-12-14 14:52:06 +01:00
|
|
|
else shape.copy(in = Inlet(thatN + ".in"))
|
2015-03-05 12:21:17 +01:00
|
|
|
}
|
2016-03-11 17:08:30 +01:00
|
|
|
|
|
|
|
|
protected def label: String = Logging.simpleName(this)
|
|
|
|
|
final override def toString: String = f"$label [${System.identityHashCode(this)}%08x]"
|
|
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2015-03-05 12:21:17 +01:00
|
|
|
* INTERNAL API
|
2015-01-28 14:19:50 +01:00
|
|
|
* Holds the downstream-most [[org.reactivestreams.Publisher]] interface of the materialized flow.
|
|
|
|
|
* The stream will not have any subscribers attached at this point, which means that after prefetching
|
|
|
|
|
* elements to fill the internal buffers it will assert back-pressure until
|
|
|
|
|
* a subscriber connects and creates demand for elements to be emitted.
|
|
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
@InternalApi private[akka] class PublisherSink[In](val attributes: Attributes, shape: SinkShape[In])
|
|
|
|
|
extends SinkModule[In, Publisher[In]](shape) {
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2016-02-24 11:55:28 +01:00
|
|
|
/*
|
|
|
|
|
* This method is the reason why SinkModule.create may return something that is
|
|
|
|
|
* not a Subscriber: a VirtualPublisher is used in order to avoid the immediate
|
|
|
|
|
* subscription a VirtualProcessor would perform (and it also saves overhead).
|
|
|
|
|
*/
|
|
|
|
|
override def create(context: MaterializationContext): (AnyRef, Publisher[In]) = {
|
2019-07-05 08:19:46 +02:00
|
|
|
|
2016-02-24 11:55:28 +01:00
|
|
|
val proc = new VirtualPublisher[In]
|
2019-07-05 08:19:46 +02:00
|
|
|
context.materializer match {
|
|
|
|
|
case am: ActorMaterializer =>
|
2019-09-04 13:37:06 +02:00
|
|
|
val StreamSubscriptionTimeout(timeout, mode) =
|
|
|
|
|
context.effectiveAttributes.mandatoryAttribute[StreamSubscriptionTimeout]
|
|
|
|
|
if (mode != StreamSubscriptionTimeoutTerminationMode.noop) {
|
|
|
|
|
am.scheduleOnce(timeout, new Runnable {
|
|
|
|
|
def run(): Unit = proc.onSubscriptionTimeout(am, mode)
|
2019-07-05 08:19:46 +02:00
|
|
|
})
|
2019-09-04 13:37:06 +02:00
|
|
|
}
|
2019-07-05 08:19:46 +02:00
|
|
|
case _ => // not possible to setup timeout
|
|
|
|
|
}
|
2015-06-16 15:34:54 +02:00
|
|
|
(proc, proc)
|
2015-01-28 14:19:50 +01:00
|
|
|
}
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
override def withAttributes(attr: Attributes): SinkModule[In, Publisher[In]] =
|
|
|
|
|
new PublisherSink[In](attr, amendShape(attr))
|
2015-01-28 14:19:50 +01:00
|
|
|
}
|
|
|
|
|
|
2015-03-05 12:21:17 +01:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
@InternalApi private[akka] final class FanoutPublisherSink[In](val attributes: Attributes, shape: SinkShape[In])
|
|
|
|
|
extends SinkModule[In, Publisher[In]](shape) {
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2015-04-10 14:39:48 +02:00
|
|
|
override def create(context: MaterializationContext): (Subscriber[In], Publisher[In]) = {
|
2019-09-05 16:08:37 +02:00
|
|
|
val impl = context.materializer.actorOf(context, FanoutProcessorImpl.props(context.effectiveAttributes))
|
2016-05-23 11:31:49 +03:00
|
|
|
val fanoutProcessor = new ActorProcessor[In, In](impl)
|
|
|
|
|
// Resolve cyclic dependency with actor. This MUST be the first message no matter what.
|
2019-07-05 08:19:46 +02:00
|
|
|
impl ! ExposedPublisher(fanoutProcessor.asInstanceOf[ActorPublisher[Any]])
|
2015-01-28 14:19:50 +01:00
|
|
|
(fanoutProcessor, fanoutProcessor)
|
|
|
|
|
}
|
|
|
|
|
|
2016-07-27 13:29:23 +02:00
|
|
|
override def withAttributes(attr: Attributes): SinkModule[In, Publisher[In]] =
|
2015-11-03 12:53:24 +01:00
|
|
|
new FanoutPublisherSink[In](attr, amendShape(attr))
|
2015-01-28 14:19:50 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2015-03-05 12:21:17 +01:00
|
|
|
* INTERNAL API
|
2015-01-28 14:19:50 +01:00
|
|
|
* Attaches a subscriber to this stream.
|
|
|
|
|
*/
|
2019-03-13 10:56:20 +01:00
|
|
|
@InternalApi private[akka] final class SubscriberSink[In](
|
|
|
|
|
subscriber: Subscriber[In],
|
|
|
|
|
val attributes: Attributes,
|
|
|
|
|
shape: SinkShape[In])
|
2019-03-11 10:38:24 +01:00
|
|
|
extends SinkModule[In, NotUsed](shape) {
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2016-01-20 10:00:37 +02:00
|
|
|
override def create(context: MaterializationContext) = (subscriber, NotUsed)
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
override def withAttributes(attr: Attributes): SinkModule[In, NotUsed] =
|
|
|
|
|
new SubscriberSink[In](subscriber, attr, amendShape(attr))
|
2015-01-28 14:19:50 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2015-03-05 12:21:17 +01:00
|
|
|
* INTERNAL API
|
2015-01-28 14:19:50 +01:00
|
|
|
* A sink that immediately cancels its upstream upon materialization.
|
|
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
@InternalApi private[akka] final class CancelSink(val attributes: Attributes, shape: SinkShape[Any])
|
|
|
|
|
extends SinkModule[Any, NotUsed](shape) {
|
|
|
|
|
override def create(context: MaterializationContext): (Subscriber[Any], NotUsed) =
|
|
|
|
|
(new CancellingSubscriber[Any], NotUsed)
|
2016-07-27 13:29:23 +02:00
|
|
|
override def withAttributes(attr: Attributes): SinkModule[Any, NotUsed] = new CancelSink(attr, amendShape(attr))
|
2015-01-28 14:19:50 +01:00
|
|
|
}
|
|
|
|
|
|
2018-07-15 07:57:55 -05:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
@InternalApi private[akka] final class TakeLastStage[T](n: Int)
|
|
|
|
|
extends GraphStageWithMaterializedValue[SinkShape[T], Future[immutable.Seq[T]]] {
|
2018-08-03 12:58:46 +09:00
|
|
|
if (n <= 0)
|
|
|
|
|
throw new IllegalArgumentException("requirement failed: n must be greater than 0")
|
2018-07-15 07:57:55 -05:00
|
|
|
|
2018-08-03 12:58:46 +09:00
|
|
|
val in: Inlet[T] = Inlet("takeLast.in")
|
2018-07-15 07:57:55 -05:00
|
|
|
|
|
|
|
|
override val shape: SinkShape[T] = SinkShape.of(in)
|
|
|
|
|
|
|
|
|
|
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
|
|
|
|
|
val p: Promise[immutable.Seq[T]] = Promise()
|
|
|
|
|
(new GraphStageLogic(shape) with InHandler {
|
|
|
|
|
private[this] val buffer = mutable.Queue.empty[T]
|
|
|
|
|
private[this] var count = 0
|
|
|
|
|
|
|
|
|
|
override def preStart(): Unit = pull(in)
|
|
|
|
|
|
|
|
|
|
override def onPush(): Unit = {
|
|
|
|
|
buffer.enqueue(grab(in))
|
|
|
|
|
if (count < n)
|
|
|
|
|
count += 1
|
|
|
|
|
else
|
|
|
|
|
buffer.dequeue()
|
|
|
|
|
pull(in)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def onUpstreamFinish(): Unit = {
|
2018-11-21 15:00:29 +01:00
|
|
|
val elements = buffer.toList
|
2018-07-15 07:57:55 -05:00
|
|
|
buffer.clear()
|
|
|
|
|
p.trySuccess(elements)
|
|
|
|
|
completeStage()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def onUpstreamFailure(ex: Throwable): Unit = {
|
|
|
|
|
p.tryFailure(ex)
|
|
|
|
|
failStage(ex)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
setHandler(in, this)
|
|
|
|
|
}, p.future)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def toString: String = "TakeLastStage"
|
|
|
|
|
}
|
|
|
|
|
|
2017-03-16 21:04:07 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
@InternalApi private[akka] final class HeadOptionStage[T]
|
|
|
|
|
extends GraphStageWithMaterializedValue[SinkShape[T], Future[Option[T]]] {
|
2015-12-13 17:02:58 +01:00
|
|
|
|
2016-01-14 15:22:25 +01:00
|
|
|
val in: Inlet[T] = Inlet("headOption.in")
|
2015-12-13 17:02:58 +01:00
|
|
|
|
|
|
|
|
override val shape: SinkShape[T] = SinkShape.of(in)
|
|
|
|
|
|
2015-11-18 00:09:04 +01:00
|
|
|
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
|
|
|
|
|
val p: Promise[Option[T]] = Promise()
|
2016-08-29 14:00:48 +02:00
|
|
|
(new GraphStageLogic(shape) with InHandler {
|
2015-11-18 00:09:04 +01:00
|
|
|
override def preStart(): Unit = pull(in)
|
|
|
|
|
|
2016-08-29 14:00:48 +02:00
|
|
|
def onPush(): Unit = {
|
|
|
|
|
p.trySuccess(Option(grab(in)))
|
|
|
|
|
completeStage()
|
|
|
|
|
}
|
2015-11-18 00:09:04 +01:00
|
|
|
|
2016-08-29 14:00:48 +02:00
|
|
|
override def onUpstreamFinish(): Unit = {
|
|
|
|
|
p.trySuccess(None)
|
|
|
|
|
completeStage()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def onUpstreamFailure(ex: Throwable): Unit = {
|
|
|
|
|
p.tryFailure(ex)
|
|
|
|
|
failStage(ex)
|
|
|
|
|
}
|
|
|
|
|
|
2017-04-28 11:11:50 +02:00
|
|
|
override def postStop(): Unit = {
|
|
|
|
|
if (!p.isCompleted) p.failure(new AbruptStageTerminationException(this))
|
|
|
|
|
}
|
|
|
|
|
|
2016-08-29 14:00:48 +02:00
|
|
|
setHandler(in, this)
|
2015-11-18 00:09:04 +01:00
|
|
|
}, p.future)
|
|
|
|
|
}
|
2015-12-14 17:02:00 +01:00
|
|
|
|
|
|
|
|
override def toString: String = "HeadOptionStage"
|
2015-11-18 00:09:04 +01:00
|
|
|
}
|
2015-12-04 09:37:32 -05:00
|
|
|
|
2017-03-16 21:04:07 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
@InternalApi private[akka] final class SeqStage[T, That](implicit cbf: Factory[T, That with immutable.Iterable[_]])
|
|
|
|
|
extends GraphStageWithMaterializedValue[SinkShape[T], Future[That]] {
|
2016-02-12 01:36:21 +08:00
|
|
|
val in = Inlet[T]("seq.in")
|
|
|
|
|
|
2016-03-11 17:08:30 +01:00
|
|
|
override def toString: String = "SeqStage"
|
|
|
|
|
|
2016-02-12 01:36:21 +08:00
|
|
|
override val shape: SinkShape[T] = SinkShape.of(in)
|
|
|
|
|
|
|
|
|
|
override protected def initialAttributes: Attributes = DefaultAttributes.seqSink
|
|
|
|
|
|
|
|
|
|
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
|
2017-11-29 13:49:31 -03:00
|
|
|
val p: Promise[That] = Promise()
|
2016-08-29 14:00:48 +02:00
|
|
|
val logic = new GraphStageLogic(shape) with InHandler {
|
2018-11-21 15:00:29 +01:00
|
|
|
val buf = cbf.newBuilder
|
2016-02-12 01:36:21 +08:00
|
|
|
|
|
|
|
|
override def preStart(): Unit = pull(in)
|
|
|
|
|
|
2016-08-29 14:00:48 +02:00
|
|
|
def onPush(): Unit = {
|
|
|
|
|
buf += grab(in)
|
|
|
|
|
pull(in)
|
|
|
|
|
}
|
2016-02-12 01:36:21 +08:00
|
|
|
|
2016-08-29 14:00:48 +02:00
|
|
|
override def onUpstreamFinish(): Unit = {
|
|
|
|
|
val result = buf.result()
|
|
|
|
|
p.trySuccess(result)
|
|
|
|
|
completeStage()
|
|
|
|
|
}
|
2016-02-12 01:36:21 +08:00
|
|
|
|
2016-08-29 14:00:48 +02:00
|
|
|
override def onUpstreamFailure(ex: Throwable): Unit = {
|
|
|
|
|
p.tryFailure(ex)
|
|
|
|
|
failStage(ex)
|
|
|
|
|
}
|
2016-02-12 01:36:21 +08:00
|
|
|
|
2017-04-28 11:11:50 +02:00
|
|
|
override def postStop(): Unit = {
|
|
|
|
|
if (!p.isCompleted) p.failure(new AbruptStageTerminationException(this))
|
|
|
|
|
}
|
|
|
|
|
|
2016-08-29 14:00:48 +02:00
|
|
|
setHandler(in, this)
|
2016-02-12 01:36:21 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
(logic, p.future)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2017-03-16 21:04:07 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
@InternalApi private[akka] object QueueSink {
|
2016-01-14 15:22:25 +01:00
|
|
|
sealed trait Output[+T]
|
|
|
|
|
final case class Pull[T](promise: Promise[Option[T]]) extends Output[T]
|
|
|
|
|
case object Cancel extends Output[Nothing]
|
|
|
|
|
}
|
|
|
|
|
|
2015-12-04 09:37:32 -05:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2020-01-25 03:33:39 +07:00
|
|
|
@InternalApi private[akka] final class QueueSink[T](maxConcurrentPulls: Int)
|
2019-03-11 10:38:24 +01:00
|
|
|
extends GraphStageWithMaterializedValue[SinkShape[T], SinkQueueWithCancel[T]] {
|
2020-01-25 03:33:39 +07:00
|
|
|
|
|
|
|
|
require(maxConcurrentPulls > 0, "Max concurrent pulls must be greater than 0")
|
|
|
|
|
|
2015-12-04 09:37:32 -05:00
|
|
|
val in = Inlet[T]("queueSink.in")
|
2016-02-15 13:38:37 +01:00
|
|
|
override def initialAttributes = DefaultAttributes.queueSink
|
2015-12-04 09:37:32 -05:00
|
|
|
override val shape: SinkShape[T] = SinkShape.of(in)
|
|
|
|
|
|
2016-03-11 17:08:30 +01:00
|
|
|
override def toString: String = "QueueSink"
|
|
|
|
|
|
2015-12-04 09:37:32 -05:00
|
|
|
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
|
2017-06-17 22:51:34 +03:00
|
|
|
val stageLogic = new GraphStageLogic(shape) with InHandler with SinkQueueWithCancel[T] {
|
2015-12-04 09:37:32 -05:00
|
|
|
|
2018-08-20 16:43:46 +02:00
|
|
|
val maxBuffer = inheritedAttributes.get[InputBuffer](InputBuffer(16, 16)).max
|
2016-02-07 14:54:48 +01:00
|
|
|
require(maxBuffer > 0, "Buffer size must be greater than 0")
|
2015-12-04 09:37:32 -05:00
|
|
|
|
2020-01-25 03:33:39 +07:00
|
|
|
// Allocates one additional element to hold stream closed/failure indicators
|
2021-07-06 18:01:41 +02:00
|
|
|
val buffer: Buffer[Try[Option[T]]] = Buffer(maxBuffer + 1, inheritedAttributes)
|
|
|
|
|
val currentRequests: Buffer[Promise[Option[T]]] = Buffer(maxConcurrentPulls, inheritedAttributes)
|
2015-12-04 09:37:32 -05:00
|
|
|
|
|
|
|
|
override def preStart(): Unit = {
|
2015-12-20 12:54:05 +01:00
|
|
|
setKeepGoing(true)
|
2015-12-04 09:37:32 -05:00
|
|
|
pull(in)
|
|
|
|
|
}
|
|
|
|
|
|
2017-06-17 22:51:34 +03:00
|
|
|
private val callback = getAsyncCallback[Output[T]] {
|
2021-07-06 18:01:41 +02:00
|
|
|
case QueueSink.Pull(pullPromise: Promise[Option[T]] @unchecked) =>
|
2020-01-25 03:33:39 +07:00
|
|
|
if (currentRequests.isFull)
|
|
|
|
|
pullPromise.failure(
|
|
|
|
|
new IllegalStateException(s"Too many concurrent pulls. Specified maximum is $maxConcurrentPulls. " +
|
|
|
|
|
"You have to wait for one previous future to be resolved to send another request"))
|
|
|
|
|
else if (buffer.isEmpty) currentRequests.enqueue(pullPromise)
|
|
|
|
|
else {
|
|
|
|
|
if (buffer.used == maxBuffer) tryPull(in)
|
|
|
|
|
sendDownstream(pullPromise)
|
2019-03-11 10:38:24 +01:00
|
|
|
}
|
2019-02-09 15:25:39 +01:00
|
|
|
case QueueSink.Cancel => completeStage()
|
2017-06-17 22:51:34 +03:00
|
|
|
}
|
2015-12-04 09:37:32 -05:00
|
|
|
|
2021-07-06 18:01:41 +02:00
|
|
|
def sendDownstream(promise: Promise[Option[T]]): Unit = {
|
2015-12-04 09:37:32 -05:00
|
|
|
val e = buffer.dequeue()
|
|
|
|
|
promise.complete(e)
|
|
|
|
|
e match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case Success(_: Some[_]) => //do nothing
|
|
|
|
|
case Success(None) => completeStage()
|
|
|
|
|
case Failure(t) => failStage(t)
|
2015-12-04 09:37:32 -05:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-08-29 14:00:48 +02:00
|
|
|
def onPush(): Unit = {
|
2020-01-25 03:33:39 +07:00
|
|
|
buffer.enqueue(Success(Some(grab(in))))
|
|
|
|
|
if (currentRequests.nonEmpty) currentRequests.dequeue().complete(buffer.dequeue())
|
2016-08-29 14:00:48 +02:00
|
|
|
if (buffer.used < maxBuffer) pull(in)
|
|
|
|
|
}
|
|
|
|
|
|
2020-01-25 03:33:39 +07:00
|
|
|
override def onUpstreamFinish(): Unit = {
|
|
|
|
|
buffer.enqueue(Success(None))
|
|
|
|
|
while (currentRequests.nonEmpty && buffer.nonEmpty) currentRequests.dequeue().complete(buffer.dequeue())
|
|
|
|
|
while (currentRequests.nonEmpty) currentRequests.dequeue().complete(Success(None))
|
|
|
|
|
if (buffer.isEmpty) completeStage()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def onUpstreamFailure(ex: Throwable): Unit = {
|
|
|
|
|
buffer.enqueue(Failure(ex))
|
|
|
|
|
while (currentRequests.nonEmpty && buffer.nonEmpty) currentRequests.dequeue().complete(buffer.dequeue())
|
|
|
|
|
while (currentRequests.nonEmpty) currentRequests.dequeue().complete(Failure(ex))
|
|
|
|
|
if (buffer.isEmpty) failStage(ex)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def postStop(): Unit =
|
|
|
|
|
while (currentRequests.nonEmpty) currentRequests.dequeue().failure(new AbruptStageTerminationException(this))
|
2016-08-29 14:00:48 +02:00
|
|
|
|
|
|
|
|
setHandler(in, this)
|
2015-12-04 09:37:32 -05:00
|
|
|
|
2017-06-17 22:51:34 +03:00
|
|
|
// SinkQueueWithCancel impl
|
2015-12-04 09:37:32 -05:00
|
|
|
override def pull(): Future[Option[T]] = {
|
2020-04-27 17:31:16 +07:00
|
|
|
val p = Promise[Option[T]]()
|
2019-03-11 10:38:24 +01:00
|
|
|
callback
|
|
|
|
|
.invokeWithFeedback(Pull(p))
|
|
|
|
|
.failed
|
|
|
|
|
.foreach {
|
2019-02-09 15:25:39 +01:00
|
|
|
case NonFatal(e) => p.tryFailure(e)
|
|
|
|
|
case _ => ()
|
2020-03-10 15:39:30 +01:00
|
|
|
}(akka.dispatch.ExecutionContexts.parasitic)
|
2015-12-04 09:37:32 -05:00
|
|
|
p.future
|
|
|
|
|
}
|
2016-01-14 15:22:25 +01:00
|
|
|
override def cancel(): Unit = {
|
2017-12-04 16:34:16 +01:00
|
|
|
callback.invoke(QueueSink.Cancel)
|
2016-01-14 15:22:25 +01:00
|
|
|
}
|
2017-06-17 22:51:34 +03:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
(stageLogic, stageLogic)
|
2015-12-04 09:37:32 -05:00
|
|
|
}
|
|
|
|
|
}
|
2016-01-21 16:37:26 +01:00
|
|
|
|
2019-07-09 13:58:26 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*
|
|
|
|
|
* Helper class to be able to express collection as a fold using mutable data without
|
|
|
|
|
* accidentally sharing state between materializations
|
|
|
|
|
*/
|
|
|
|
|
@InternalApi private[akka] trait CollectorState[T, R] {
|
|
|
|
|
def accumulated(): Any
|
|
|
|
|
def update(elem: T): CollectorState[T, R]
|
|
|
|
|
def finish(): R
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*
|
|
|
|
|
* Helper class to be able to express collection as a fold using mutable data
|
|
|
|
|
*/
|
|
|
|
|
@InternalApi private[akka] final class FirstCollectorState[T, R](
|
|
|
|
|
collectorFactory: () => java.util.stream.Collector[T, Any, R])
|
|
|
|
|
extends CollectorState[T, R] {
|
|
|
|
|
|
|
|
|
|
override def update(elem: T): CollectorState[T, R] = {
|
|
|
|
|
// on first update, return a new mutable collector to ensure not
|
|
|
|
|
// sharing collector between streams
|
|
|
|
|
val collector = collectorFactory()
|
|
|
|
|
val accumulator = collector.accumulator()
|
|
|
|
|
val accumulated = collector.supplier().get()
|
|
|
|
|
accumulator.accept(accumulated, elem)
|
|
|
|
|
new MutableCollectorState(collector, accumulator, accumulated)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def accumulated(): Any = {
|
|
|
|
|
// only called if it is asked about accumulated before accepting a first element
|
|
|
|
|
val collector = collectorFactory()
|
|
|
|
|
collector.supplier().get()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def finish(): R = {
|
|
|
|
|
// only called if completed without elements
|
|
|
|
|
val collector = collectorFactory()
|
|
|
|
|
collector.finisher().apply(collector.supplier().get())
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-01-14 15:22:25 +01:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*
|
|
|
|
|
* Helper class to be able to express collection as a fold using mutable data
|
|
|
|
|
*/
|
2019-07-09 13:58:26 +02:00
|
|
|
@InternalApi private[akka] final class MutableCollectorState[T, R](
|
|
|
|
|
collector: java.util.stream.Collector[T, Any, R],
|
|
|
|
|
accumulator: java.util.function.BiConsumer[Any, T],
|
2021-07-06 18:01:41 +02:00
|
|
|
_accumulated: Any)
|
2019-07-09 13:58:26 +02:00
|
|
|
extends CollectorState[T, R] {
|
2016-01-14 15:22:25 +01:00
|
|
|
|
2021-07-06 18:01:41 +02:00
|
|
|
override def accumulated(): Any = _accumulated
|
|
|
|
|
|
2019-07-09 13:58:26 +02:00
|
|
|
override def update(elem: T): CollectorState[T, R] = {
|
2021-07-06 18:01:41 +02:00
|
|
|
accumulator.accept(_accumulated, elem)
|
2016-01-14 15:22:25 +01:00
|
|
|
this
|
|
|
|
|
}
|
|
|
|
|
|
2019-07-09 13:58:26 +02:00
|
|
|
override def finish(): R = {
|
|
|
|
|
// only called if completed without elements
|
2021-07-06 18:01:41 +02:00
|
|
|
collector.finisher().apply(_accumulated)
|
2019-07-09 13:58:26 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*
|
|
|
|
|
* Helper class to be able to express reduce as a fold for parallel collector without
|
|
|
|
|
* accidentally sharing state between materializations
|
|
|
|
|
*/
|
|
|
|
|
@InternalApi private[akka] trait ReducerState[T, R] {
|
|
|
|
|
def update(batch: Any): ReducerState[T, R]
|
|
|
|
|
def finish(): R
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*
|
|
|
|
|
* Helper class to be able to express reduce as a fold for parallel collector
|
|
|
|
|
*/
|
|
|
|
|
@InternalApi private[akka] final class FirstReducerState[T, R](
|
|
|
|
|
collectorFactory: () => java.util.stream.Collector[T, Any, R])
|
|
|
|
|
extends ReducerState[T, R] {
|
|
|
|
|
|
|
|
|
|
def update(batch: Any): ReducerState[T, R] = {
|
|
|
|
|
// on first update, return a new mutable collector to ensure not
|
|
|
|
|
// sharing collector between streams
|
|
|
|
|
val collector = collectorFactory()
|
|
|
|
|
val combiner = collector.combiner()
|
|
|
|
|
new MutableReducerState(collector, combiner, batch)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def finish(): R = {
|
|
|
|
|
// only called if completed without elements
|
|
|
|
|
val collector = collectorFactory()
|
|
|
|
|
collector.finisher().apply(null)
|
|
|
|
|
}
|
2016-01-21 16:37:26 +01:00
|
|
|
}
|
2016-01-14 15:22:25 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*
|
|
|
|
|
* Helper class to be able to express reduce as a fold for parallel collector
|
|
|
|
|
*/
|
2019-07-09 13:58:26 +02:00
|
|
|
@InternalApi private[akka] final class MutableReducerState[T, R](
|
|
|
|
|
val collector: java.util.stream.Collector[T, Any, R],
|
|
|
|
|
val combiner: BinaryOperator[Any],
|
|
|
|
|
var reduced: Any)
|
|
|
|
|
extends ReducerState[T, R] {
|
2016-01-14 15:22:25 +01:00
|
|
|
|
|
|
|
|
def update(batch: Any): ReducerState[T, R] = {
|
2019-07-09 13:58:26 +02:00
|
|
|
reduced = combiner(reduced, batch)
|
2016-01-14 15:22:25 +01:00
|
|
|
this
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def finish(): R = collector.finisher().apply(reduced)
|
|
|
|
|
}
|
|
|
|
|
|
2016-07-07 07:01:28 -04:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2019-03-11 10:38:24 +01:00
|
|
|
@InternalApi final private[stream] class LazySink[T, M](sinkFactory: T => Future[Sink[T, M]])
|
2019-10-16 17:02:12 +02:00
|
|
|
extends GraphStageWithMaterializedValue[SinkShape[T], Future[M]] {
|
2016-07-07 07:01:28 -04:00
|
|
|
val in = Inlet[T]("lazySink.in")
|
2021-01-11 15:17:50 +01:00
|
|
|
override def initialAttributes = DefaultAttributes.lazySink and SourceLocation.forLambda(sinkFactory)
|
2016-07-07 07:01:28 -04:00
|
|
|
override val shape: SinkShape[T] = SinkShape.of(in)
|
|
|
|
|
|
|
|
|
|
override def toString: String = "LazySink"
|
|
|
|
|
|
2019-10-16 17:02:12 +02:00
|
|
|
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[M]) = {
|
2016-07-07 07:01:28 -04:00
|
|
|
|
2019-10-16 17:02:12 +02:00
|
|
|
val promise = Promise[M]()
|
2016-07-07 07:01:28 -04:00
|
|
|
val stageLogic = new GraphStageLogic(shape) with InHandler {
|
2018-03-19 14:42:37 +01:00
|
|
|
var switching = false
|
2016-07-07 07:01:28 -04:00
|
|
|
override def preStart(): Unit = pull(in)
|
|
|
|
|
|
|
|
|
|
override def onPush(): Unit = {
|
2018-03-19 14:42:37 +01:00
|
|
|
val element = grab(in)
|
|
|
|
|
switching = true
|
|
|
|
|
val cb: AsyncCallback[Try[Sink[T, M]]] =
|
|
|
|
|
getAsyncCallback {
|
2019-02-09 15:25:39 +01:00
|
|
|
case Success(sink) =>
|
2018-03-19 14:42:37 +01:00
|
|
|
// 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)
|
2019-10-16 17:02:12 +02:00
|
|
|
promise.success(mat)
|
2018-03-19 14:42:37 +01:00
|
|
|
setKeepGoing(true)
|
|
|
|
|
} catch {
|
2019-02-09 15:25:39 +01:00
|
|
|
case NonFatal(e) =>
|
2018-03-19 14:42:37 +01:00
|
|
|
promise.failure(e)
|
|
|
|
|
failStage(e)
|
|
|
|
|
}
|
|
|
|
|
}
|
2019-02-09 15:25:39 +01:00
|
|
|
case Failure(e) =>
|
2018-03-19 14:42:37 +01:00
|
|
|
promise.failure(e)
|
|
|
|
|
failStage(e)
|
|
|
|
|
}
|
2016-07-07 07:01:28 -04:00
|
|
|
try {
|
2020-03-10 15:39:30 +01:00
|
|
|
sinkFactory(element).onComplete(cb.invoke)(ExecutionContexts.parasitic)
|
2016-07-07 07:01:28 -04:00
|
|
|
} catch {
|
2019-02-09 15:25:39 +01:00
|
|
|
case NonFatal(e) =>
|
2018-03-19 14:42:37 +01:00
|
|
|
promise.failure(e)
|
|
|
|
|
failStage(e)
|
2016-07-07 07:01:28 -04:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2018-03-19 14:42:37 +01:00
|
|
|
override def onUpstreamFinish(): Unit = {
|
|
|
|
|
// 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 {
|
2019-10-16 17:02:12 +02:00
|
|
|
promise.failure(new NeverMaterializedException)
|
2018-03-19 14:42:37 +01:00
|
|
|
super.onUpstreamFinish()
|
|
|
|
|
}
|
2016-07-07 07:01:28 -04:00
|
|
|
}
|
|
|
|
|
|
2018-03-19 14:42:37 +01:00
|
|
|
override def onUpstreamFailure(ex: Throwable): Unit = {
|
|
|
|
|
promise.failure(ex)
|
|
|
|
|
super.onUpstreamFailure(ex)
|
2016-07-07 07:01:28 -04:00
|
|
|
}
|
2018-03-19 14:42:37 +01:00
|
|
|
|
2016-07-07 07:01:28 -04:00
|
|
|
setHandler(in, this)
|
|
|
|
|
|
2018-03-19 14:42:37 +01:00
|
|
|
private def switchTo(sink: Sink[T, M], firstElement: T): M = {
|
2017-02-16 03:19:25 -05:00
|
|
|
|
2018-03-19 14:42:37 +01:00
|
|
|
var firstElementPushed = false
|
2016-07-07 07:01:28 -04:00
|
|
|
|
2018-03-19 14:42:37 +01:00
|
|
|
val subOutlet = new SubSourceOutlet[T]("LazySink")
|
2016-07-07 07:01:28 -04:00
|
|
|
|
2021-06-02 03:48:14 -07:00
|
|
|
val matVal = interpreter.subFusingMaterializer
|
|
|
|
|
.materialize(Source.fromGraph(subOutlet.source).toMat(sink)(Keep.right), inheritedAttributes)
|
2016-07-07 07:01:28 -04:00
|
|
|
|
2018-03-19 14:42:37 +01:00
|
|
|
def maybeCompleteStage(): Unit = {
|
|
|
|
|
if (isClosed(in) && subOutlet.isClosed) {
|
|
|
|
|
completeStage()
|
|
|
|
|
}
|
2016-07-07 07:01:28 -04:00
|
|
|
}
|
|
|
|
|
|
2018-03-19 14:42:37 +01:00
|
|
|
// The stage must not be shut down automatically; it is completed when maybeCompleteStage decides
|
|
|
|
|
setKeepGoing(true)
|
2016-07-07 07:01:28 -04:00
|
|
|
|
2019-03-13 10:56:20 +01:00
|
|
|
setHandler(
|
|
|
|
|
in,
|
|
|
|
|
new InHandler {
|
|
|
|
|
override def onPush(): Unit = {
|
|
|
|
|
subOutlet.push(grab(in))
|
|
|
|
|
}
|
|
|
|
|
override def onUpstreamFinish(): Unit = {
|
|
|
|
|
if (firstElementPushed) {
|
|
|
|
|
subOutlet.complete()
|
|
|
|
|
maybeCompleteStage()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
override def onUpstreamFailure(ex: Throwable): Unit = {
|
|
|
|
|
// propagate exception irrespective if the cached element has been pushed or not
|
|
|
|
|
subOutlet.fail(ex)
|
2019-03-18 14:28:31 +01:00
|
|
|
// #25410 if we fail the stage here directly, the SubSource may not have been started yet,
|
|
|
|
|
// which can happen if upstream fails immediately after emitting a first value.
|
|
|
|
|
// The SubSource won't be started until the stream shuts down, which means downstream won't see the failure,
|
|
|
|
|
// scheduling it lets the interpreter first start the substream
|
|
|
|
|
getAsyncCallback[Throwable](failStage).invoke(ex)
|
2019-03-13 10:56:20 +01:00
|
|
|
}
|
|
|
|
|
})
|
2018-03-19 14:42:37 +01:00
|
|
|
|
|
|
|
|
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()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2019-08-16 10:53:14 +02:00
|
|
|
|
|
|
|
|
override def onDownstreamFinish(cause: Throwable): Unit = {
|
|
|
|
|
if (!isClosed(in)) cancel(in, cause)
|
2018-03-19 14:42:37 +01:00
|
|
|
maybeCompleteStage()
|
|
|
|
|
}
|
|
|
|
|
})
|
|
|
|
|
|
|
|
|
|
matVal
|
2016-07-07 07:01:28 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
(stageLogic, promise.future)
|
|
|
|
|
}
|
|
|
|
|
}
|