2014-03-30 09:27:19 +02:00
|
|
|
/**
|
|
|
|
|
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
|
|
|
|
|
*/
|
|
|
|
|
package akka.stream.impl
|
|
|
|
|
|
2014-09-01 13:30:15 +02:00
|
|
|
import java.util.concurrent.atomic.AtomicReference
|
2014-11-03 15:29:02 +01:00
|
|
|
import akka.actor.ActorLogging
|
|
|
|
|
import akka.actor.Cancellable
|
2014-11-19 12:56:55 +01:00
|
|
|
|
2014-09-01 13:30:15 +02:00
|
|
|
import akka.actor.{ Actor, ActorRef }
|
2014-04-08 13:37:55 +02:00
|
|
|
import akka.stream.MaterializerSettings
|
2014-07-22 12:21:53 +02:00
|
|
|
import org.reactivestreams.{ Publisher, Subscriber, Subscription }
|
2014-09-01 13:30:15 +02:00
|
|
|
import scala.collection.mutable
|
2014-11-12 13:05:57 +01:00
|
|
|
import scala.concurrent.duration.FiniteDuration
|
2014-03-30 09:27:19 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] object MultiStreamOutputProcessor {
|
2014-09-01 13:30:15 +02:00
|
|
|
case class SubstreamKey(id: Long)
|
2014-08-21 16:07:09 +02:00
|
|
|
case class SubstreamRequestMore(substream: SubstreamKey, demand: Long)
|
2014-09-01 13:30:15 +02:00
|
|
|
case class SubstreamCancel(substream: SubstreamKey)
|
|
|
|
|
case class SubstreamSubscribe(substream: SubstreamKey, subscriber: Subscriber[Any])
|
2014-11-12 13:05:57 +01:00
|
|
|
case class SubstreamSubscriptionTimeout(substream: SubstreamKey)
|
2014-03-30 09:27:19 +02:00
|
|
|
|
2014-09-01 13:30:15 +02:00
|
|
|
class SubstreamSubscription(val parent: ActorRef, val substreamKey: SubstreamKey) extends Subscription {
|
2014-08-21 16:07:09 +02:00
|
|
|
override def request(elements: Long): Unit =
|
2014-11-19 12:56:55 +01:00
|
|
|
if (elements <= 0) throw new IllegalArgumentException(ReactiveStreamsCompliance.NumberOfElementsInRequestMustBePositiveMsg)
|
2014-09-01 13:30:15 +02:00
|
|
|
else parent ! SubstreamRequestMore(substreamKey, elements)
|
|
|
|
|
override def cancel(): Unit = parent ! SubstreamCancel(substreamKey)
|
2014-03-30 09:27:19 +02:00
|
|
|
override def toString = "SubstreamSubscription" + System.identityHashCode(this)
|
|
|
|
|
}
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
object SubstreamOutput {
|
2014-09-01 13:30:15 +02:00
|
|
|
sealed trait PublisherState
|
|
|
|
|
sealed trait CompletedState extends PublisherState
|
|
|
|
|
case object Open extends PublisherState
|
|
|
|
|
final case class Attached(sub: Subscriber[Any]) extends PublisherState
|
|
|
|
|
case object Completed extends CompletedState
|
|
|
|
|
final case class Failed(e: Throwable) extends CompletedState
|
2014-09-10 12:14:09 +02:00
|
|
|
}
|
2014-03-30 09:27:19 +02:00
|
|
|
|
2014-11-12 13:05:57 +01:00
|
|
|
class SubstreamOutput(val key: SubstreamKey, actor: ActorRef, pump: Pump, subscriptionTimeout: Cancellable)
|
2014-11-03 15:29:02 +01:00
|
|
|
extends SimpleOutputs(actor, pump) with Publisher[Any] {
|
2014-09-10 12:14:09 +02:00
|
|
|
|
|
|
|
|
import SubstreamOutput._
|
|
|
|
|
|
|
|
|
|
private val subscription = new SubstreamSubscription(actor, key)
|
2014-09-01 13:30:15 +02:00
|
|
|
private val state = new AtomicReference[PublisherState](Open)
|
2014-03-30 09:27:19 +02:00
|
|
|
|
2014-05-16 14:21:15 +02:00
|
|
|
override def subreceive: SubReceive =
|
2014-05-12 16:45:30 +02:00
|
|
|
throw new UnsupportedOperationException("Substream outputs are managed in a dedicated receive block")
|
|
|
|
|
|
=str #16315 fixes assumption that substream key is always present
Additional fix for race when timout put to mailbox, then timeout cancelled and subscribe succeeds.
So it could happen, that JUST BEFORE `subscriptionTimeout.cancel()` the
`SubstreamSubscriptionTimeout` was already put into the actor's mailbox,
the cancelling then kicks in (does nothing) and then the attaching of
the subscriber kicks in, it passes OK. So the actor gets the timeout
first and then the subscription... So the publisher is already
subscribed to by some subsciber. But now the actor gets the
SubstreamSubscriptionTimeout message, and wants to subscribe the
cancelling subscriber to it. The publisher can only handle 1 subscriber,
so attaching of the cancelling subscriber will fail - well this is not
very bad, because it will just onError the cancelling subscriber rigth
away, but it can be missleading because the debug log will contain the
"cancelling... after..." message, while it has not REALLY cancelled it.
The isAttached can not be added to handleSubscriptionTimeout because
there it's "too late" and the log message would already be sent..
So while writing this up I noticed that it's not an "error race", but
it's still a race we could avoid when putting this isAttached check
before cancelling the publisher... do you think it's worth it?
2014-11-17 15:25:24 +01:00
|
|
|
def isAttached() = state.get().isInstanceOf[Attached]
|
|
|
|
|
|
2014-08-21 16:07:09 +02:00
|
|
|
def enqueueOutputDemand(demand: Long): Unit = {
|
2014-09-01 13:30:15 +02:00
|
|
|
downstreamDemand += demand
|
|
|
|
|
pump.pump()
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
|
|
|
|
|
2014-04-08 10:28:27 +02:00
|
|
|
override def cancel(e: Throwable): Unit = {
|
2014-09-01 13:30:15 +02:00
|
|
|
if (!downstreamCompleted) {
|
|
|
|
|
closePublisher(Failed(e))
|
|
|
|
|
downstreamCompleted = true
|
|
|
|
|
}
|
2014-04-08 10:28:27 +02:00
|
|
|
}
|
2014-03-30 09:27:19 +02:00
|
|
|
|
2014-09-01 13:30:15 +02:00
|
|
|
override def complete(): Unit = {
|
|
|
|
|
if (!downstreamCompleted) {
|
|
|
|
|
closePublisher(Completed)
|
|
|
|
|
downstreamCompleted = true
|
|
|
|
|
}
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
2014-04-01 12:45:41 +02:00
|
|
|
|
2014-09-01 13:30:15 +02:00
|
|
|
private def closePublisher(withState: CompletedState): Unit = {
|
2014-11-12 13:05:57 +01:00
|
|
|
subscriptionTimeout.cancel()
|
2014-09-01 13:30:15 +02:00
|
|
|
state.getAndSet(withState) match {
|
|
|
|
|
case Attached(sub) ⇒ closeSubscriber(sub, withState)
|
|
|
|
|
case _: CompletedState ⇒ throw new IllegalStateException("Attempted to double shutdown publisher")
|
2014-09-10 12:14:09 +02:00
|
|
|
case Open ⇒ // No action needed
|
2014-09-01 13:30:15 +02:00
|
|
|
}
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
2014-09-01 13:30:15 +02:00
|
|
|
|
|
|
|
|
private def closeSubscriber(s: Subscriber[Any], withState: CompletedState): Unit = withState match {
|
|
|
|
|
case Completed ⇒ s.onComplete()
|
|
|
|
|
case Failed(e) ⇒ s.onError(e)
|
|
|
|
|
}
|
|
|
|
|
|
2014-08-21 16:07:09 +02:00
|
|
|
override def subscribe(s: Subscriber[_ >: Any]): Unit = {
|
2014-11-12 13:05:57 +01:00
|
|
|
subscriptionTimeout.cancel()
|
|
|
|
|
if (state.compareAndSet(Open, Attached(s))) actor ! SubstreamSubscribe(key, s)
|
|
|
|
|
else {
|
|
|
|
|
state.get() match {
|
2014-11-19 12:56:55 +01:00
|
|
|
case _: Attached ⇒ s.onError(new IllegalStateException("Substream publisher " + ReactiveStreamsCompliance.SupportsOnlyASingleSubscriber))
|
2014-11-12 13:05:57 +01:00
|
|
|
case c: CompletedState ⇒ closeSubscriber(s, c)
|
|
|
|
|
case Open ⇒ throw new IllegalStateException("Publisher cannot become open after being used before")
|
2014-09-01 13:30:15 +02:00
|
|
|
}
|
|
|
|
|
}
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
2014-09-01 13:30:15 +02:00
|
|
|
|
|
|
|
|
def attachSubscriber(s: Subscriber[Any]): Unit =
|
|
|
|
|
if (subscriber eq null) {
|
|
|
|
|
subscriber = s
|
|
|
|
|
subscriber.onSubscribe(subscription)
|
|
|
|
|
} else subscriber.onError(new IllegalStateException("Cannot subscribe two or more Subscribers to this Publisher"))
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
2014-09-10 12:14:09 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2014-11-03 15:29:02 +01:00
|
|
|
private[akka] trait MultiStreamOutputProcessorLike extends Pump with StreamSubscriptionTimeoutSupport {
|
|
|
|
|
this: Actor with ActorLogging ⇒
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
import MultiStreamOutputProcessor._
|
2014-11-12 13:05:57 +01:00
|
|
|
import StreamSubscriptionTimeoutSupport._
|
2014-03-30 09:27:19 +02:00
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
protected def nextId(): Long
|
|
|
|
|
|
=str #16315 fixes assumption that substream key is always present
Additional fix for race when timout put to mailbox, then timeout cancelled and subscribe succeeds.
So it could happen, that JUST BEFORE `subscriptionTimeout.cancel()` the
`SubstreamSubscriptionTimeout` was already put into the actor's mailbox,
the cancelling then kicks in (does nothing) and then the attaching of
the subscriber kicks in, it passes OK. So the actor gets the timeout
first and then the subscription... So the publisher is already
subscribed to by some subsciber. But now the actor gets the
SubstreamSubscriptionTimeout message, and wants to subscribe the
cancelling subscriber to it. The publisher can only handle 1 subscriber,
so attaching of the cancelling subscriber will fail - well this is not
very bad, because it will just onError the cancelling subscriber rigth
away, but it can be missleading because the debug log will contain the
"cancelling... after..." message, while it has not REALLY cancelled it.
The isAttached can not be added to handleSubscriptionTimeout because
there it's "too late" and the log message would already be sent..
So while writing this up I noticed that it's not an "error race", but
it's still a race we could avoid when putting this isAttached check
before cancelling the publisher... do you think it's worth it?
2014-11-17 15:25:24 +01:00
|
|
|
// stream keys will be removed from this map on cancellation/subscription-timeout, never assume a key is present
|
2014-09-10 12:14:09 +02:00
|
|
|
private val substreamOutputs = mutable.Map.empty[SubstreamKey, SubstreamOutput]
|
|
|
|
|
|
|
|
|
|
protected def createSubstreamOutput(): SubstreamOutput = {
|
|
|
|
|
val id = SubstreamKey(nextId())
|
2014-11-12 13:05:57 +01:00
|
|
|
val cancellable = scheduleSubscriptionTimeout(self, SubstreamSubscriptionTimeout(id))
|
|
|
|
|
val output = new SubstreamOutput(id, self, this, cancellable)
|
|
|
|
|
substreamOutputs(output.key) = output
|
|
|
|
|
output
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
protected def invalidateSubstreamOutput(substream: SubstreamKey): Unit = {
|
2014-11-12 10:16:22 +01:00
|
|
|
completeSubstreamOutput(substream)
|
|
|
|
|
pump()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
protected def completeSubstreamOutput(substream: SubstreamKey): Unit = {
|
=str #16315 fixes assumption that substream key is always present
Additional fix for race when timout put to mailbox, then timeout cancelled and subscribe succeeds.
So it could happen, that JUST BEFORE `subscriptionTimeout.cancel()` the
`SubstreamSubscriptionTimeout` was already put into the actor's mailbox,
the cancelling then kicks in (does nothing) and then the attaching of
the subscriber kicks in, it passes OK. So the actor gets the timeout
first and then the subscription... So the publisher is already
subscribed to by some subsciber. But now the actor gets the
SubstreamSubscriptionTimeout message, and wants to subscribe the
cancelling subscriber to it. The publisher can only handle 1 subscriber,
so attaching of the cancelling subscriber will fail - well this is not
very bad, because it will just onError the cancelling subscriber rigth
away, but it can be missleading because the debug log will contain the
"cancelling... after..." message, while it has not REALLY cancelled it.
The isAttached can not be added to handleSubscriptionTimeout because
there it's "too late" and the log message would already be sent..
So while writing this up I noticed that it's not an "error race", but
it's still a race we could avoid when putting this isAttached check
before cancelling the publisher... do you think it's worth it?
2014-11-17 15:25:24 +01:00
|
|
|
substreamOutputs.get(substream) match {
|
|
|
|
|
case Some(sub) ⇒
|
|
|
|
|
sub.complete()
|
|
|
|
|
substreamOutputs -= substream
|
|
|
|
|
case _ ⇒ // ignore, already completed...
|
|
|
|
|
}
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
protected def failOutputs(e: Throwable): Unit = {
|
2014-04-08 10:28:27 +02:00
|
|
|
substreamOutputs.values foreach (_.cancel(e))
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
protected def finishOutputs(): Unit = {
|
2014-09-01 13:30:15 +02:00
|
|
|
substreamOutputs.values foreach (_.complete())
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
val outputSubstreamManagement: Receive = {
|
2014-11-12 10:16:22 +01:00
|
|
|
case SubstreamRequestMore(key, demand) ⇒ substreamOutputs.get(key) match {
|
=str #16315 fixes assumption that substream key is always present
Additional fix for race when timout put to mailbox, then timeout cancelled and subscribe succeeds.
So it could happen, that JUST BEFORE `subscriptionTimeout.cancel()` the
`SubstreamSubscriptionTimeout` was already put into the actor's mailbox,
the cancelling then kicks in (does nothing) and then the attaching of
the subscriber kicks in, it passes OK. So the actor gets the timeout
first and then the subscription... So the publisher is already
subscribed to by some subsciber. But now the actor gets the
SubstreamSubscriptionTimeout message, and wants to subscribe the
cancelling subscriber to it. The publisher can only handle 1 subscriber,
so attaching of the cancelling subscriber will fail - well this is not
very bad, because it will just onError the cancelling subscriber rigth
away, but it can be missleading because the debug log will contain the
"cancelling... after..." message, while it has not REALLY cancelled it.
The isAttached can not be added to handleSubscriptionTimeout because
there it's "too late" and the log message would already be sent..
So while writing this up I noticed that it's not an "error race", but
it's still a race we could avoid when putting this isAttached check
before cancelling the publisher... do you think it's worth it?
2014-11-17 15:25:24 +01:00
|
|
|
case Some(sub) ⇒ sub.enqueueOutputDemand(demand)
|
|
|
|
|
case _ ⇒ // ignore...
|
|
|
|
|
}
|
|
|
|
|
case SubstreamSubscribe(key, subscriber) ⇒ substreamOutputs.get(key) match {
|
|
|
|
|
case Some(sub) ⇒ sub.attachSubscriber(subscriber)
|
|
|
|
|
case _ ⇒ // ignore...
|
|
|
|
|
}
|
|
|
|
|
case SubstreamSubscriptionTimeout(key) ⇒ substreamOutputs.get(key) match {
|
|
|
|
|
case Some(sub) if !sub.isAttached() ⇒ subscriptionTimedOut(sub)
|
|
|
|
|
case _ ⇒ // ignore...
|
2014-11-12 10:16:22 +01:00
|
|
|
}
|
=str #16315 fixes assumption that substream key is always present
Additional fix for race when timout put to mailbox, then timeout cancelled and subscribe succeeds.
So it could happen, that JUST BEFORE `subscriptionTimeout.cancel()` the
`SubstreamSubscriptionTimeout` was already put into the actor's mailbox,
the cancelling then kicks in (does nothing) and then the attaching of
the subscriber kicks in, it passes OK. So the actor gets the timeout
first and then the subscription... So the publisher is already
subscribed to by some subsciber. But now the actor gets the
SubstreamSubscriptionTimeout message, and wants to subscribe the
cancelling subscriber to it. The publisher can only handle 1 subscriber,
so attaching of the cancelling subscriber will fail - well this is not
very bad, because it will just onError the cancelling subscriber rigth
away, but it can be missleading because the debug log will contain the
"cancelling... after..." message, while it has not REALLY cancelled it.
The isAttached can not be added to handleSubscriptionTimeout because
there it's "too late" and the log message would already be sent..
So while writing this up I noticed that it's not an "error race", but
it's still a race we could avoid when putting this isAttached check
before cancelling the publisher... do you think it's worth it?
2014-11-17 15:25:24 +01:00
|
|
|
case SubstreamCancel(key) ⇒ invalidateSubstreamOutput(key)
|
2014-11-12 13:05:57 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override protected def handleSubscriptionTimeout(target: Publisher[_], cause: Exception) = target match {
|
|
|
|
|
case s: SubstreamOutput ⇒
|
|
|
|
|
s.cancel(cause)
|
|
|
|
|
s.attachSubscriber(CancelingSubscriber)
|
|
|
|
|
case _ ⇒ // ignore
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
2014-09-10 12:14:09 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] abstract class MultiStreamOutputProcessor(_settings: MaterializerSettings) extends ActorProcessorImpl(_settings) with MultiStreamOutputProcessorLike {
|
|
|
|
|
private var _nextId = 0L
|
|
|
|
|
protected def nextId(): Long = { _nextId += 1; _nextId }
|
|
|
|
|
|
2014-11-03 15:29:02 +01:00
|
|
|
override val subscriptionTimeoutSettings = _settings.subscriptionTimeoutSettings
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
override protected def fail(e: Throwable): Unit = {
|
|
|
|
|
failOutputs(e)
|
|
|
|
|
super.fail(e)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def pumpFinished(): Unit = {
|
|
|
|
|
finishOutputs()
|
|
|
|
|
super.pumpFinished()
|
|
|
|
|
}
|
2014-05-12 16:45:30 +02:00
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
override def activeReceive = primaryInputs.subreceive orElse primaryOutputs.subreceive orElse outputSubstreamManagement
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] object TwoStreamInputProcessor {
|
2014-04-01 12:45:41 +02:00
|
|
|
class OtherActorSubscriber[T](val impl: ActorRef) extends Subscriber[T] {
|
2014-06-10 14:09:08 +02:00
|
|
|
override def onError(cause: Throwable): Unit = impl ! OtherStreamOnError(cause)
|
2014-03-30 09:27:19 +02:00
|
|
|
override def onComplete(): Unit = impl ! OtherStreamOnComplete
|
|
|
|
|
override def onNext(element: T): Unit = impl ! OtherStreamOnNext(element)
|
|
|
|
|
override def onSubscribe(subscription: Subscription): Unit = impl ! OtherStreamOnSubscribe(subscription)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
case object OtherStreamOnComplete
|
|
|
|
|
case class OtherStreamOnNext(element: Any)
|
|
|
|
|
case class OtherStreamOnSubscribe(subscription: Subscription)
|
2014-06-10 14:09:08 +02:00
|
|
|
case class OtherStreamOnError(ex: Throwable)
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2014-07-22 12:21:53 +02:00
|
|
|
private[akka] abstract class TwoStreamInputProcessor(_settings: MaterializerSettings, val other: Publisher[Any])
|
2014-03-30 09:27:19 +02:00
|
|
|
extends ActorProcessorImpl(_settings) {
|
2014-09-01 13:30:15 +02:00
|
|
|
import akka.stream.impl.TwoStreamInputProcessor._
|
2014-03-30 09:27:19 +02:00
|
|
|
|
2014-05-12 16:45:30 +02:00
|
|
|
val secondaryInputs: Inputs = new BatchingInputBuffer(settings.initialInputBufferSize, this) {
|
|
|
|
|
override val subreceive: SubReceive = new SubReceive(waitingForUpstream)
|
2014-03-30 09:27:19 +02:00
|
|
|
|
2014-05-12 16:45:30 +02:00
|
|
|
override def inputOnError(e: Throwable): Unit = TwoStreamInputProcessor.this.onError(e)
|
2014-03-30 09:27:19 +02:00
|
|
|
|
2014-05-12 16:45:30 +02:00
|
|
|
override def waitingForUpstream: Receive = {
|
|
|
|
|
case OtherStreamOnComplete ⇒ onComplete()
|
|
|
|
|
case OtherStreamOnSubscribe(subscription) ⇒ onSubscribe(subscription)
|
2014-06-10 14:09:08 +02:00
|
|
|
case OtherStreamOnError(e) ⇒ TwoStreamInputProcessor.this.onError(e)
|
2014-05-12 16:45:30 +02:00
|
|
|
}
|
2014-03-30 09:27:19 +02:00
|
|
|
|
2014-05-12 16:45:30 +02:00
|
|
|
override def upstreamRunning: Receive = {
|
|
|
|
|
case OtherStreamOnNext(element) ⇒ enqueueInputElement(element)
|
|
|
|
|
case OtherStreamOnComplete ⇒ onComplete()
|
2014-06-10 14:09:08 +02:00
|
|
|
case OtherStreamOnError(e) ⇒ TwoStreamInputProcessor.this.onError(e)
|
2014-05-12 16:45:30 +02:00
|
|
|
}
|
|
|
|
|
override protected def completed: Actor.Receive = {
|
|
|
|
|
case OtherStreamOnSubscribe(_) ⇒ throw new IllegalStateException("Cannot subscribe shutdown subscriber")
|
|
|
|
|
}
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
|
|
|
|
|
2014-08-20 19:43:31 +02:00
|
|
|
override def activeReceive: Receive =
|
|
|
|
|
secondaryInputs.subreceive orElse primaryInputs.subreceive orElse primaryOutputs.subreceive
|
2014-03-30 09:27:19 +02:00
|
|
|
|
2014-07-22 12:21:53 +02:00
|
|
|
other.subscribe(new OtherActorSubscriber(self))
|
2014-03-30 09:27:19 +02:00
|
|
|
|
2014-09-01 13:30:15 +02:00
|
|
|
override def pumpFinished(): Unit = {
|
2014-05-12 16:45:30 +02:00
|
|
|
secondaryInputs.cancel()
|
2014-09-01 13:30:15 +02:00
|
|
|
super.pumpFinished()
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
2014-09-01 13:30:15 +02:00
|
|
|
|
2014-05-16 14:21:15 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] object MultiStreamInputProcessor {
|
2014-09-10 12:14:09 +02:00
|
|
|
case class SubstreamKey(id: Long)
|
2014-05-16 14:21:15 +02:00
|
|
|
|
|
|
|
|
class SubstreamSubscriber[T](val impl: ActorRef, key: SubstreamKey) extends Subscriber[T] {
|
|
|
|
|
override def onError(cause: Throwable): Unit = impl ! SubstreamOnError(key, cause)
|
|
|
|
|
override def onComplete(): Unit = impl ! SubstreamOnComplete(key)
|
|
|
|
|
override def onNext(element: T): Unit = impl ! SubstreamOnNext(key, element)
|
|
|
|
|
override def onSubscribe(subscription: Subscription): Unit = impl ! SubstreamStreamOnSubscribe(key, subscription)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
case class SubstreamOnComplete(key: SubstreamKey)
|
|
|
|
|
case class SubstreamOnNext(key: SubstreamKey, element: Any)
|
|
|
|
|
case class SubstreamOnError(key: SubstreamKey, e: Throwable)
|
|
|
|
|
case class SubstreamStreamOnSubscribe(key: SubstreamKey, subscription: Subscription)
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
class SubstreamInput(val key: SubstreamKey, bufferSize: Int, processor: MultiStreamInputProcessorLike, pump: Pump) extends BatchingInputBuffer(bufferSize, pump) {
|
2014-05-16 14:21:15 +02:00
|
|
|
// Not driven directly
|
|
|
|
|
override val subreceive = new SubReceive(Actor.emptyBehavior)
|
|
|
|
|
|
|
|
|
|
def substreamOnComplete(): Unit = onComplete()
|
|
|
|
|
def substreamOnSubscribe(subscription: Subscription): Unit = onSubscribe(subscription)
|
|
|
|
|
def substreamOnError(e: Throwable): Unit = onError(e)
|
|
|
|
|
def substreamOnNext(elem: Any): Unit = enqueueInputElement(elem)
|
|
|
|
|
|
|
|
|
|
override protected def inputOnError(e: Throwable): Unit = {
|
|
|
|
|
super.inputOnError(e)
|
2014-09-10 12:14:09 +02:00
|
|
|
processor.invalidateSubstreamInput(key, e)
|
2014-05-16 14:21:15 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] trait MultiStreamInputProcessorLike extends Pump { this: Actor ⇒
|
|
|
|
|
|
|
|
|
|
import MultiStreamInputProcessor._
|
|
|
|
|
|
|
|
|
|
protected def nextId(): Long
|
|
|
|
|
protected def inputBufferSize: Int
|
|
|
|
|
|
|
|
|
|
private val substreamInputs = collection.mutable.Map.empty[SubstreamKey, SubstreamInput]
|
|
|
|
|
|
|
|
|
|
val inputSubstreamManagement: Receive = {
|
2014-05-16 14:21:15 +02:00
|
|
|
case SubstreamStreamOnSubscribe(key, subscription) ⇒ substreamInputs(key).substreamOnSubscribe(subscription)
|
|
|
|
|
case SubstreamOnNext(key, element) ⇒ substreamInputs(key).substreamOnNext(element)
|
|
|
|
|
case SubstreamOnComplete(key) ⇒ {
|
|
|
|
|
substreamInputs(key).substreamOnComplete()
|
|
|
|
|
substreamInputs -= key
|
|
|
|
|
}
|
|
|
|
|
case SubstreamOnError(key, e) ⇒ substreamInputs(key).substreamOnError(e)
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
def createSubstreamInput(): SubstreamInput = {
|
|
|
|
|
val key = SubstreamKey(nextId())
|
|
|
|
|
val inputs = new SubstreamInput(key, inputBufferSize, this, this)
|
2014-05-16 14:21:15 +02:00
|
|
|
substreamInputs(key) = inputs
|
|
|
|
|
inputs
|
|
|
|
|
}
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
def createAndSubscribeSubstreamInput(p: Publisher[Any]): SubstreamInput = {
|
|
|
|
|
val inputs = createSubstreamInput()
|
|
|
|
|
p.subscribe(new SubstreamSubscriber(self, inputs.key))
|
|
|
|
|
inputs
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def invalidateSubstreamInput(substream: SubstreamKey, e: Throwable): Unit = {
|
2014-05-16 14:21:15 +02:00
|
|
|
substreamInputs(substream).cancel()
|
|
|
|
|
substreamInputs -= substream
|
|
|
|
|
pump()
|
|
|
|
|
}
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
protected def failInputs(e: Throwable): Unit = {
|
2014-05-16 14:21:15 +02:00
|
|
|
substreamInputs.values foreach (_.cancel())
|
|
|
|
|
}
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
protected def finishInputs(): Unit = {
|
2014-05-16 14:21:15 +02:00
|
|
|
substreamInputs.values foreach (_.cancel())
|
2014-09-10 12:14:09 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] abstract class MultiStreamInputProcessor(_settings: MaterializerSettings) extends ActorProcessorImpl(_settings) with MultiStreamInputProcessorLike {
|
|
|
|
|
private var _nextId = 0L
|
|
|
|
|
protected def nextId(): Long = { _nextId += 1; _nextId }
|
|
|
|
|
|
|
|
|
|
override protected val inputBufferSize = _settings.initialInputBufferSize
|
|
|
|
|
|
|
|
|
|
override protected def fail(e: Throwable) = {
|
|
|
|
|
failInputs(e)
|
|
|
|
|
super.fail(e)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def pumpFinished() = {
|
|
|
|
|
finishInputs()
|
2014-09-01 13:30:15 +02:00
|
|
|
super.pumpFinished()
|
2014-05-16 14:21:15 +02:00
|
|
|
}
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
override def activeReceive = primaryInputs.subreceive orElse primaryOutputs.subreceive orElse inputSubstreamManagement
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|