2015-01-28 14:19:50 +01:00
|
|
|
/**
|
|
|
|
|
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
|
|
|
|
*/
|
|
|
|
|
package akka.stream.impl
|
|
|
|
|
|
2015-12-04 09:37:32 -05:00
|
|
|
import java.util.concurrent.atomic.AtomicReference
|
|
|
|
|
|
2016-01-20 10:00:37 +02:00
|
|
|
import akka.{ Done, NotUsed }
|
2015-11-18 00:09:04 +01:00
|
|
|
import akka.actor.{ ActorRef, Props }
|
2015-12-04 09:37:32 -05:00
|
|
|
import akka.stream.Attributes.InputBuffer
|
2015-08-19 23:04:20 -04:00
|
|
|
import akka.stream._
|
2015-12-04 09:37:32 -05:00
|
|
|
import akka.stream.impl.StreamLayout.Module
|
2016-01-16 12:17:19 -05:00
|
|
|
import akka.stream.stage._
|
2015-11-18 00:09:04 +01:00
|
|
|
import org.reactivestreams.{ Publisher, Subscriber }
|
2015-12-04 09:37:32 -05:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
import scala.annotation.unchecked.uncheckedVariance
|
|
|
|
|
import scala.concurrent.{ Future, Promise }
|
2015-08-19 23:04:20 -04:00
|
|
|
import scala.language.postfixOps
|
2015-12-04 09:37:32 -05:00
|
|
|
import scala.util.{ Failure, Success, Try }
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2015-03-05 12:21:17 +01:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] abstract class SinkModule[-In, Mat](val shape: SinkShape[In]) extends Module {
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2015-04-10 14:39:48 +02:00
|
|
|
def create(context: MaterializationContext): (Subscriber[In] @uncheckedVariance, Mat)
|
2015-01-28 14:19:50 +01:00
|
|
|
|
|
|
|
|
override def replaceShape(s: Shape): Module =
|
|
|
|
|
if (s == shape) this
|
|
|
|
|
else throw new UnsupportedOperationException("cannot replace the shape of a Sink, you need to wrap it in a Graph for that")
|
|
|
|
|
|
|
|
|
|
// This is okay since we the only caller of this method is right below.
|
|
|
|
|
protected def newInstance(s: SinkShape[In] @uncheckedVariance): SinkModule[In, Mat]
|
|
|
|
|
|
2015-12-14 14:52:06 +01:00
|
|
|
override def carbonCopy: Module = newInstance(SinkShape(shape.in.carbonCopy()))
|
2015-01-28 14:19:50 +01:00
|
|
|
|
|
|
|
|
override def subModules: Set[Module] = Set.empty
|
2015-03-05 12:21:17 +01:00
|
|
|
|
2015-07-06 22:00:21 +02:00
|
|
|
protected def amendShape(attr: Attributes): SinkShape[In] = {
|
|
|
|
|
val thisN = attributes.nameOrDefault(null)
|
|
|
|
|
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
|
|
|
}
|
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.
|
|
|
|
|
*/
|
2015-06-23 17:32:55 +02:00
|
|
|
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
|
|
|
|
|
|
|
|
override def toString: String = "PublisherSink"
|
|
|
|
|
|
2015-04-10 14:39:48 +02:00
|
|
|
override def create(context: MaterializationContext): (Subscriber[In], Publisher[In]) = {
|
2015-06-16 15:34:54 +02:00
|
|
|
val proc = new VirtualProcessor[In]
|
|
|
|
|
(proc, proc)
|
2015-01-28 14:19:50 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override protected def newInstance(shape: SinkShape[In]): SinkModule[In, Publisher[In]] = new PublisherSink[In](attributes, shape)
|
2015-06-23 17:32:55 +02:00
|
|
|
override def withAttributes(attr: Attributes): Module = new PublisherSink[In](attr, amendShape(attr))
|
2015-01-28 14:19:50 +01:00
|
|
|
}
|
|
|
|
|
|
2015-03-05 12:21:17 +01:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] final class FanoutPublisherSink[In](
|
2015-06-23 17:32:55 +02:00
|
|
|
val attributes: Attributes,
|
2015-01-28 14:19:50 +01:00
|
|
|
shape: SinkShape[In])
|
|
|
|
|
extends SinkModule[In, Publisher[In]](shape) {
|
|
|
|
|
|
2015-04-10 14:39:48 +02:00
|
|
|
override def create(context: MaterializationContext): (Subscriber[In], Publisher[In]) = {
|
2015-06-23 18:28:53 +02:00
|
|
|
val actorMaterializer = ActorMaterializer.downcast(context.materializer)
|
2015-10-30 16:00:44 +01:00
|
|
|
val fanoutProcessor = ActorProcessorFactory[In, In](
|
|
|
|
|
actorMaterializer.actorOf(
|
|
|
|
|
context,
|
2015-11-03 12:53:24 +01:00
|
|
|
FanoutProcessorImpl.props(actorMaterializer.effectiveSettings(attributes))))
|
2015-01-28 14:19:50 +01:00
|
|
|
(fanoutProcessor, fanoutProcessor)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override protected def newInstance(shape: SinkShape[In]): SinkModule[In, Publisher[In]] =
|
2015-11-03 12:53:24 +01:00
|
|
|
new FanoutPublisherSink[In](attributes, shape)
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2015-06-23 17:32:55 +02:00
|
|
|
override def withAttributes(attr: Attributes): Module =
|
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 which will just discard all received
|
|
|
|
|
* elements.
|
|
|
|
|
*/
|
2016-01-20 10:00:37 +02:00
|
|
|
private[akka] final class SinkholeSink(val attributes: Attributes, shape: SinkShape[Any]) extends SinkModule[Any, Future[Done]](shape) {
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2015-04-10 14:39:48 +02:00
|
|
|
override def create(context: MaterializationContext) = {
|
2015-06-23 18:28:53 +02:00
|
|
|
val effectiveSettings = ActorMaterializer.downcast(context.materializer).effectiveSettings(context.effectiveAttributes)
|
2016-01-20 10:00:37 +02:00
|
|
|
val p = Promise[Done]()
|
2015-11-18 00:09:04 +01:00
|
|
|
(new SinkholeSubscriber[Any](p), p.future)
|
2015-04-10 14:39:48 +02:00
|
|
|
}
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2016-01-20 10:00:37 +02:00
|
|
|
override protected def newInstance(shape: SinkShape[Any]): SinkModule[Any, Future[Done]] = new SinkholeSink(attributes, shape)
|
2015-11-18 00:09:04 +01:00
|
|
|
override def withAttributes(attr: Attributes): Module = new SinkholeSink(attr, amendShape(attr))
|
|
|
|
|
override def toString: String = "SinkholeSink"
|
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.
|
|
|
|
|
*/
|
2016-01-20 10:00:37 +02:00
|
|
|
private[akka] final class SubscriberSink[In](subscriber: Subscriber[In], val attributes: Attributes, shape: SinkShape[In]) 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
|
|
|
|
2016-01-20 10:00:37 +02:00
|
|
|
override protected def newInstance(shape: SinkShape[In]): SinkModule[In, NotUsed] = new SubscriberSink[In](subscriber, attributes, shape)
|
2015-06-23 17:32:55 +02:00
|
|
|
override def withAttributes(attr: Attributes): Module = new SubscriberSink[In](subscriber, attr, amendShape(attr))
|
2015-06-14 03:12:30 -04:00
|
|
|
override def toString: String = "SubscriberSink"
|
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.
|
|
|
|
|
*/
|
2016-01-20 10:00:37 +02:00
|
|
|
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)
|
|
|
|
|
override protected def newInstance(shape: SinkShape[Any]): SinkModule[Any, NotUsed] = new CancelSink(attributes, shape)
|
2015-06-23 17:32:55 +02:00
|
|
|
override def withAttributes(attr: Attributes): Module = new CancelSink(attr, amendShape(attr))
|
2015-06-14 03:12:30 -04:00
|
|
|
override def toString: String = "CancelSink"
|
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
|
|
|
* Creates and wraps an actor into [[org.reactivestreams.Subscriber]] from the given `props`,
|
|
|
|
|
* which should be [[akka.actor.Props]] for an [[akka.stream.actor.ActorSubscriber]].
|
|
|
|
|
*/
|
2015-06-23 17:32:55 +02:00
|
|
|
private[akka] final class ActorSubscriberSink[In](props: Props, val attributes: Attributes, shape: SinkShape[In]) extends SinkModule[In, ActorRef](shape) {
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2015-04-10 14:39:48 +02:00
|
|
|
override def create(context: MaterializationContext) = {
|
2015-06-23 18:28:53 +02:00
|
|
|
val subscriberRef = ActorMaterializer.downcast(context.materializer).actorOf(context, props)
|
2015-01-28 14:19:50 +01:00
|
|
|
(akka.stream.actor.ActorSubscriber[In](subscriberRef), subscriberRef)
|
|
|
|
|
}
|
|
|
|
|
|
2015-03-30 14:42:30 +02:00
|
|
|
override protected def newInstance(shape: SinkShape[In]): SinkModule[In, ActorRef] = new ActorSubscriberSink[In](props, attributes, shape)
|
2015-06-23 17:32:55 +02:00
|
|
|
override def withAttributes(attr: Attributes): Module = new ActorSubscriberSink[In](props, attr, amendShape(attr))
|
2015-06-14 03:12:30 -04:00
|
|
|
override def toString: String = "ActorSubscriberSink"
|
2015-01-28 14:19:50 +01:00
|
|
|
}
|
2015-03-30 14:42:30 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] final class ActorRefSink[In](ref: ActorRef, onCompleteMessage: Any,
|
2015-06-23 17:32:55 +02:00
|
|
|
val attributes: Attributes,
|
2016-01-20 10:00:37 +02:00
|
|
|
shape: SinkShape[In]) extends SinkModule[In, NotUsed](shape) {
|
2015-03-30 14:42:30 +02:00
|
|
|
|
2015-04-10 14:39:48 +02:00
|
|
|
override def create(context: MaterializationContext) = {
|
2015-06-23 18:28:53 +02:00
|
|
|
val actorMaterializer = ActorMaterializer.downcast(context.materializer)
|
2015-04-10 14:39:48 +02:00
|
|
|
val effectiveSettings = actorMaterializer.effectiveSettings(context.effectiveAttributes)
|
|
|
|
|
val subscriberRef = actorMaterializer.actorOf(context,
|
|
|
|
|
ActorRefSinkActor.props(ref, effectiveSettings.maxInputBufferSize, onCompleteMessage))
|
2016-01-20 10:00:37 +02:00
|
|
|
(akka.stream.actor.ActorSubscriber[In](subscriberRef), NotUsed)
|
2015-03-30 14:42:30 +02:00
|
|
|
}
|
|
|
|
|
|
2016-01-20 10:00:37 +02:00
|
|
|
override protected def newInstance(shape: SinkShape[In]): SinkModule[In, NotUsed] =
|
2015-03-30 14:42:30 +02:00
|
|
|
new ActorRefSink[In](ref, onCompleteMessage, attributes, shape)
|
2015-06-23 17:32:55 +02:00
|
|
|
override def withAttributes(attr: Attributes): Module =
|
2015-03-30 14:42:30 +02:00
|
|
|
new ActorRefSink[In](ref, onCompleteMessage, attr, amendShape(attr))
|
2015-06-14 03:12:30 -04:00
|
|
|
override def toString: String = "ActorRefSink"
|
2015-03-30 14:42:30 +02:00
|
|
|
}
|
|
|
|
|
|
2015-12-13 17:02:58 +01:00
|
|
|
private[akka] final class LastOptionStage[T] extends GraphStageWithMaterializedValue[SinkShape[T], Future[Option[T]]] {
|
|
|
|
|
|
|
|
|
|
val in = Inlet[T]("lastOption.in")
|
|
|
|
|
|
|
|
|
|
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()
|
|
|
|
|
(new GraphStageLogic(shape) {
|
|
|
|
|
override def preStart(): Unit = pull(in)
|
|
|
|
|
setHandler(in, new InHandler {
|
|
|
|
|
private[this] var prev: T = null.asInstanceOf[T]
|
|
|
|
|
|
|
|
|
|
override def onPush(): Unit = {
|
|
|
|
|
prev = grab(in)
|
|
|
|
|
pull(in)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def onUpstreamFinish(): Unit = {
|
|
|
|
|
val head = prev
|
|
|
|
|
prev = null.asInstanceOf[T]
|
|
|
|
|
p.trySuccess(Option(head))
|
|
|
|
|
completeStage()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def onUpstreamFailure(ex: Throwable): Unit = {
|
|
|
|
|
prev = null.asInstanceOf[T]
|
|
|
|
|
p.tryFailure(ex)
|
|
|
|
|
failStage(ex)
|
|
|
|
|
}
|
|
|
|
|
})
|
|
|
|
|
}, p.future)
|
|
|
|
|
}
|
2015-12-14 17:02:00 +01:00
|
|
|
|
|
|
|
|
override def toString: String = "LastOptionStage"
|
2015-11-18 00:09:04 +01:00
|
|
|
}
|
|
|
|
|
|
2015-12-13 17:02:58 +01:00
|
|
|
private[akka] final class HeadOptionStage[T] extends GraphStageWithMaterializedValue[SinkShape[T], Future[Option[T]]] {
|
|
|
|
|
|
|
|
|
|
val in = Inlet[T]("headOption.in")
|
|
|
|
|
|
|
|
|
|
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()
|
|
|
|
|
(new GraphStageLogic(shape) {
|
|
|
|
|
override def preStart(): Unit = pull(in)
|
|
|
|
|
setHandler(in, new InHandler {
|
|
|
|
|
override def onPush(): Unit = {
|
|
|
|
|
p.trySuccess(Option(grab(in)))
|
|
|
|
|
completeStage()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def onUpstreamFinish(): Unit = {
|
|
|
|
|
p.trySuccess(None)
|
|
|
|
|
completeStage()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def onUpstreamFailure(ex: Throwable): Unit = {
|
|
|
|
|
p.tryFailure(ex)
|
|
|
|
|
failStage(ex)
|
|
|
|
|
}
|
|
|
|
|
})
|
|
|
|
|
}, 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
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] class QueueSink[T]() extends GraphStageWithMaterializedValue[SinkShape[T], SinkQueue[T]] {
|
2016-01-16 12:17:19 -05:00
|
|
|
type Requested[E] = Promise[Option[E]]
|
2015-12-04 09:37:32 -05:00
|
|
|
|
|
|
|
|
val in = Inlet[T]("queueSink.in")
|
|
|
|
|
override val shape: SinkShape[T] = SinkShape.of(in)
|
|
|
|
|
|
|
|
|
|
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes) = {
|
|
|
|
|
type Received[E] = Try[Option[E]]
|
|
|
|
|
|
|
|
|
|
val maxBuffer = module.attributes.getAttribute(classOf[InputBuffer], InputBuffer(16, 16)).max
|
|
|
|
|
require(maxBuffer > 0, "Buffer size must be greater than 0")
|
|
|
|
|
|
|
|
|
|
val buffer = FixedSizeBuffer[Received[T]](maxBuffer + 1)
|
|
|
|
|
var currentRequest: Option[Requested[T]] = None
|
|
|
|
|
|
2016-01-16 12:17:19 -05:00
|
|
|
val stageLogic = new GraphStageLogic(shape) with CallbackWrapper[Requested[T]] {
|
2015-12-04 09:37:32 -05:00
|
|
|
|
|
|
|
|
override def preStart(): Unit = {
|
2015-12-20 12:54:05 +01:00
|
|
|
setKeepGoing(true)
|
2016-01-16 12:17:19 -05:00
|
|
|
initCallback(callback.invoke)
|
2015-12-04 09:37:32 -05:00
|
|
|
pull(in)
|
|
|
|
|
}
|
|
|
|
|
|
2016-01-16 12:17:19 -05:00
|
|
|
override def postStop(): Unit = stopCallback(promise ⇒
|
|
|
|
|
promise.failure(new IllegalStateException("Stream is terminated. QueueSink is detached")))
|
|
|
|
|
|
2015-12-04 09:37:32 -05:00
|
|
|
private val callback: AsyncCallback[Requested[T]] =
|
|
|
|
|
getAsyncCallback(promise ⇒ currentRequest match {
|
|
|
|
|
case Some(_) ⇒
|
|
|
|
|
promise.failure(new IllegalStateException("You have to wait for previous future to be resolved to send another request"))
|
|
|
|
|
case None ⇒
|
|
|
|
|
if (buffer.isEmpty) currentRequest = Some(promise)
|
|
|
|
|
else sendDownstream(promise)
|
|
|
|
|
})
|
|
|
|
|
|
|
|
|
|
def sendDownstream(promise: Requested[T]): Unit = {
|
|
|
|
|
val e = buffer.dequeue()
|
|
|
|
|
promise.complete(e)
|
|
|
|
|
e match {
|
|
|
|
|
case Success(_: Some[_]) ⇒ //do nothing
|
|
|
|
|
case Success(None) ⇒ completeStage()
|
|
|
|
|
case Failure(t) ⇒ failStage(t)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def enqueueAndNotify(requested: Received[T]): Unit = {
|
|
|
|
|
buffer.enqueue(requested)
|
|
|
|
|
currentRequest match {
|
|
|
|
|
case Some(p) ⇒
|
|
|
|
|
sendDownstream(p)
|
|
|
|
|
currentRequest = None
|
|
|
|
|
case None ⇒ //do nothing
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
setHandler(in, new InHandler {
|
|
|
|
|
override def onPush(): Unit = {
|
|
|
|
|
enqueueAndNotify(Success(Some(grab(in))))
|
|
|
|
|
if (buffer.used < maxBuffer - 1) pull(in)
|
|
|
|
|
}
|
|
|
|
|
override def onUpstreamFinish(): Unit = enqueueAndNotify(Success(None))
|
|
|
|
|
override def onUpstreamFailure(ex: Throwable): Unit = enqueueAndNotify(Failure(ex))
|
|
|
|
|
})
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
(stageLogic, new SinkQueue[T] {
|
|
|
|
|
override def pull(): Future[Option[T]] = {
|
|
|
|
|
val p = Promise[Option[T]]
|
2016-01-16 12:17:19 -05:00
|
|
|
stageLogic.invoke(p)
|
2015-12-04 09:37:32 -05:00
|
|
|
p.future
|
|
|
|
|
}
|
|
|
|
|
})
|
|
|
|
|
}
|
|
|
|
|
}
|