2014-03-30 09:27:19 +02:00
|
|
|
/**
|
2016-02-23 12:58:39 +01:00
|
|
|
* Copyright (C) 2009-2016 Lightbend Inc. <http://www.lightbend.com>
|
2014-03-30 09:27:19 +02:00
|
|
|
*/
|
|
|
|
|
package akka.stream.impl
|
|
|
|
|
|
2014-09-01 13:30:15 +02:00
|
|
|
import java.util.concurrent.atomic.AtomicReference
|
2015-05-29 16:43:02 +02:00
|
|
|
import akka.actor._
|
2015-06-23 18:28:53 +02:00
|
|
|
import akka.stream.ActorMaterializerSettings
|
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-03-30 09:27:19 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] object MultiStreamOutputProcessor {
|
2015-01-23 17:18:09 +01:00
|
|
|
final case class SubstreamKey(id: Long)
|
2015-05-29 16:43:02 +02:00
|
|
|
final case class SubstreamRequestMore(substream: SubstreamKey, demand: Long) extends DeadLetterSuppression with NoSerializationVerificationNeeded
|
|
|
|
|
final case class SubstreamCancel(substream: SubstreamKey) extends DeadLetterSuppression with NoSerializationVerificationNeeded
|
|
|
|
|
final case class SubstreamSubscribe(substream: SubstreamKey, subscriber: Subscriber[Any]) extends DeadLetterSuppression with NoSerializationVerificationNeeded
|
|
|
|
|
final case class SubstreamSubscriptionTimeout(substream: SubstreamKey) extends DeadLetterSuppression with NoSerializationVerificationNeeded
|
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-12-08 23:10:04 +01:00
|
|
|
override def request(elements: Long): Unit = parent ! SubstreamRequestMore(substreamKey, elements)
|
2014-09-01 13:30:15 +02:00
|
|
|
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
|
2015-03-03 10:57:25 +01:00
|
|
|
case object Cancelled extends CompletedState
|
2014-09-01 13:30:15 +02:00
|
|
|
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] {
|
2015-02-03 11:34:11 +01:00
|
|
|
import ReactiveStreamsCompliance._
|
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")
|
|
|
|
|
|
2015-08-01 00:13:14 +02:00
|
|
|
def isAttached = state.get().isInstanceOf[Attached]
|
=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
|
|
|
|
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
|
|
|
}
|
|
|
|
|
|
2015-03-03 10:57:25 +01:00
|
|
|
override def error(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
|
|
|
|
2015-03-03 10:57:25 +01:00
|
|
|
override def cancel(): Unit = {
|
|
|
|
|
if (!downstreamCompleted) {
|
|
|
|
|
closePublisher(Cancelled)
|
|
|
|
|
downstreamCompleted = true
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
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 _: CompletedState ⇒ throw new IllegalStateException("Attempted to double shutdown publisher")
|
2015-05-14 12:21:47 +02:00
|
|
|
case Attached(sub) ⇒
|
|
|
|
|
if (subscriber eq null) tryOnSubscribe(sub, CancelledSubscription)
|
|
|
|
|
closeSubscriber(sub, withState)
|
|
|
|
|
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 {
|
2015-02-03 11:34:11 +01:00
|
|
|
case Completed ⇒ tryOnComplete(s)
|
2015-03-03 10:57:25 +01:00
|
|
|
case Cancelled ⇒ // nothing to do
|
2015-02-03 11:34:11 +01:00
|
|
|
case Failed(e: SpecViolation) ⇒ // nothing to do
|
|
|
|
|
case Failed(e) ⇒ tryOnError(s, e)
|
2014-09-01 13:30:15 +02:00
|
|
|
}
|
|
|
|
|
|
2014-08-21 16:07:09 +02:00
|
|
|
override def subscribe(s: Subscriber[_ >: Any]): Unit = {
|
2015-03-03 10:57:25 +01:00
|
|
|
requireNonNullSubscriber(s)
|
2014-11-12 13:05:57 +01:00
|
|
|
subscriptionTimeout.cancel()
|
|
|
|
|
if (state.compareAndSet(Open, Attached(s))) actor ! SubstreamSubscribe(key, s)
|
|
|
|
|
else {
|
|
|
|
|
state.get() match {
|
2015-03-03 10:57:25 +01:00
|
|
|
case _: Attached | Cancelled ⇒
|
|
|
|
|
rejectAdditionalSubscriber(s, "Substream publisher")
|
2015-02-03 11:34:11 +01:00
|
|
|
case c: CompletedState ⇒
|
2015-03-03 10:57:25 +01:00
|
|
|
tryOnSubscribe(s, CancelledSubscription)
|
2015-02-03 11:34:11 +01:00
|
|
|
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
|
2015-02-03 11:34:11 +01:00
|
|
|
tryOnSubscribe(subscriber, subscription)
|
|
|
|
|
} else
|
2015-03-03 10:57:25 +01:00
|
|
|
rejectAdditionalSubscriber(s, "Substream 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 = {
|
2015-03-03 10:57:25 +01:00
|
|
|
cancelSubstreamOutput(substream)
|
2014-11-12 10:16:22 +01:00
|
|
|
pump()
|
|
|
|
|
}
|
|
|
|
|
|
2015-03-03 10:57:25 +01:00
|
|
|
protected def cancelSubstreamOutput(substream: SubstreamKey): Unit = {
|
|
|
|
|
substreamOutputs.get(substream) match {
|
|
|
|
|
case Some(sub) ⇒
|
|
|
|
|
sub.cancel()
|
|
|
|
|
substreamOutputs -= substream
|
|
|
|
|
case _ ⇒ // ignore, already completed...
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-11-12 10:16:22 +01:00
|
|
|
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 = {
|
2015-03-03 10:57:25 +01:00
|
|
|
substreamOutputs.values foreach (_.error(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 = {
|
2015-03-03 10:57:25 +01:00
|
|
|
case SubstreamRequestMore(key, demand) ⇒
|
|
|
|
|
substreamOutputs.get(key) match {
|
|
|
|
|
case Some(sub) ⇒
|
|
|
|
|
if (demand < 1) // According to Reactive Streams Spec 3.9, with non-positive demand must yield onError
|
|
|
|
|
sub.error(ReactiveStreamsCompliance.numberOfElementsInRequestMustBePositiveException)
|
|
|
|
|
else
|
|
|
|
|
sub.enqueueOutputDemand(demand)
|
|
|
|
|
case _ ⇒ // ignore...
|
|
|
|
|
}
|
=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 SubstreamSubscribe(key, subscriber) ⇒ substreamOutputs.get(key) match {
|
|
|
|
|
case Some(sub) ⇒ sub.attachSubscriber(subscriber)
|
|
|
|
|
case _ ⇒ // ignore...
|
|
|
|
|
}
|
|
|
|
|
case SubstreamSubscriptionTimeout(key) ⇒ substreamOutputs.get(key) match {
|
2015-08-01 00:13:14 +02:00
|
|
|
case Some(sub) if !sub.isAttached ⇒ subscriptionTimedOut(sub)
|
|
|
|
|
case _ ⇒ // ignore...
|
2014-11-12 10:16:22 +01:00
|
|
|
}
|
2015-03-03 10:57:25 +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 ⇒
|
2015-03-03 10:57:25 +01:00
|
|
|
s.error(cause)
|
2014-11-12 13:05:57 +01:00
|
|
|
s.attachSubscriber(CancelingSubscriber)
|
|
|
|
|
case _ ⇒ // ignore
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
2014-09-10 12:14:09 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2015-06-23 18:28:53 +02:00
|
|
|
private[akka] abstract class MultiStreamOutputProcessor(_settings: ActorMaterializerSettings) extends ActorProcessorImpl(_settings) with MultiStreamOutputProcessorLike {
|
2014-09-10 12:14:09 +02:00
|
|
|
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
|
|
|
|
2015-01-23 17:18:09 +01:00
|
|
|
override def activeReceive: Receive = primaryInputs.subreceive orElse primaryOutputs.subreceive orElse outputSubstreamManagement
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
|
|
|
|
|