=str #15402 Split up MultiStream(Input|Output)Processor since SSL will use both
This commit is contained in:
parent
076cdaccbb
commit
8265f92330
9 changed files with 279 additions and 219 deletions
|
|
@ -27,26 +27,20 @@ private[akka] object MultiStreamOutputProcessor {
|
|||
override def toString = "SubstreamSubscription" + System.identityHashCode(this)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] abstract class MultiStreamOutputProcessor(_settings: MaterializerSettings) extends ActorProcessorImpl(_settings) {
|
||||
import MultiStreamOutputProcessor._
|
||||
private var nextId = 0
|
||||
private val substreamOutputs = mutable.Map.empty[SubstreamKey, SubstreamOutputs]
|
||||
|
||||
class SubstreamOutputs(val key: SubstreamKey) extends SimpleOutputs(self, this) with Publisher[Any] {
|
||||
|
||||
object SubstreamOutput {
|
||||
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
|
||||
}
|
||||
|
||||
private val subscription = new SubstreamSubscription(self, key)
|
||||
class SubstreamOutput(val key: SubstreamKey, actor: ActorRef, pump: Pump) extends SimpleOutputs(actor, pump) with Publisher[Any] {
|
||||
|
||||
import SubstreamOutput._
|
||||
|
||||
private val subscription = new SubstreamSubscription(actor, key)
|
||||
private val state = new AtomicReference[PublisherState](Open)
|
||||
|
||||
override def subreceive: SubReceive =
|
||||
|
|
@ -75,7 +69,7 @@ private[akka] abstract class MultiStreamOutputProcessor(_settings: MaterializerS
|
|||
state.getAndSet(withState) match {
|
||||
case Attached(sub) ⇒ closeSubscriber(sub, withState)
|
||||
case _: CompletedState ⇒ throw new IllegalStateException("Attempted to double shutdown publisher")
|
||||
case Open ⇒ // No action needed
|
||||
case Open ⇒ // No action needed
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -85,7 +79,7 @@ private[akka] abstract class MultiStreamOutputProcessor(_settings: MaterializerS
|
|||
}
|
||||
|
||||
override def subscribe(s: Subscriber[Any]): Unit = {
|
||||
if (state.compareAndSet(Open, Attached(s))) self ! SubstreamSubscribe(key, s)
|
||||
if (state.compareAndSet(Open, Attached(s))) actor ! SubstreamSubscribe(key, s)
|
||||
else {
|
||||
state.get() match {
|
||||
case _: Attached ⇒ s.onError(new IllegalStateException("Cannot subscribe two or more Subscribers to this Publisher"))
|
||||
|
|
@ -101,38 +95,64 @@ private[akka] abstract class MultiStreamOutputProcessor(_settings: MaterializerS
|
|||
subscriber.onSubscribe(subscription)
|
||||
} else subscriber.onError(new IllegalStateException("Cannot subscribe two or more Subscribers to this Publisher"))
|
||||
}
|
||||
}
|
||||
|
||||
protected def newSubstream(): SubstreamOutputs = {
|
||||
val id = SubstreamKey(nextId)
|
||||
nextId += 1
|
||||
val outputs = new SubstreamOutputs(id)
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] trait MultiStreamOutputProcessorLike extends Pump { this: Actor ⇒
|
||||
import MultiStreamOutputProcessor._
|
||||
|
||||
protected def nextId(): Long
|
||||
|
||||
private val substreamOutputs = mutable.Map.empty[SubstreamKey, SubstreamOutput]
|
||||
|
||||
protected def createSubstreamOutput(): SubstreamOutput = {
|
||||
val id = SubstreamKey(nextId())
|
||||
val outputs = new SubstreamOutput(id, self, this)
|
||||
substreamOutputs(outputs.key) = outputs
|
||||
outputs
|
||||
}
|
||||
|
||||
protected def invalidateSubstream(substream: SubstreamKey): Unit = {
|
||||
protected def invalidateSubstreamOutput(substream: SubstreamKey): Unit = {
|
||||
substreamOutputs(substream).complete()
|
||||
substreamOutputs -= substream
|
||||
pump()
|
||||
}
|
||||
|
||||
override def fail(e: Throwable): Unit = {
|
||||
protected def failOutputs(e: Throwable): Unit = {
|
||||
substreamOutputs.values foreach (_.cancel(e))
|
||||
}
|
||||
|
||||
protected def finishOutputs(): Unit = {
|
||||
substreamOutputs.values foreach (_.complete())
|
||||
}
|
||||
|
||||
val outputSubstreamManagement: Receive = {
|
||||
case SubstreamRequestMore(key, demand) ⇒ substreamOutputs(key).enqueueOutputDemand(demand)
|
||||
case SubstreamCancel(key) ⇒ invalidateSubstreamOutput(key)
|
||||
case SubstreamSubscribe(key, subscriber) ⇒ substreamOutputs(key).attachSubscriber(subscriber)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 }
|
||||
|
||||
override protected def fail(e: Throwable): Unit = {
|
||||
failOutputs(e)
|
||||
super.fail(e)
|
||||
}
|
||||
|
||||
override def pumpFinished(): Unit = {
|
||||
substreamOutputs.values foreach (_.complete())
|
||||
finishOutputs()
|
||||
super.pumpFinished()
|
||||
}
|
||||
|
||||
val substreamManagement: Receive = {
|
||||
case SubstreamRequestMore(key, demand) ⇒ substreamOutputs(key).enqueueOutputDemand(demand)
|
||||
case SubstreamCancel(key) ⇒ invalidateSubstream(key)
|
||||
case SubstreamSubscribe(key, subscriber) ⇒ substreamOutputs(key).attachSubscriber(subscriber)
|
||||
}
|
||||
|
||||
override def activeReceive = primaryInputs.subreceive orElse primaryOutputs.subreceive orElse substreamManagement
|
||||
override def activeReceive = primaryInputs.subreceive orElse primaryOutputs.subreceive orElse outputSubstreamManagement
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -196,7 +216,7 @@ private[akka] abstract class TwoStreamInputProcessor(_settings: MaterializerSett
|
|||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object MultiStreamInputProcessor {
|
||||
case class SubstreamKey(id: Int)
|
||||
case class SubstreamKey(id: Long)
|
||||
|
||||
class SubstreamSubscriber[T](val impl: ActorRef, key: SubstreamKey) extends Subscriber[T] {
|
||||
override def onError(cause: Throwable): Unit = impl ! SubstreamOnError(key, cause)
|
||||
|
|
@ -209,18 +229,8 @@ private[akka] object MultiStreamInputProcessor {
|
|||
case class SubstreamOnNext(key: SubstreamKey, element: Any)
|
||||
case class SubstreamOnError(key: SubstreamKey, e: Throwable)
|
||||
case class SubstreamStreamOnSubscribe(key: SubstreamKey, subscription: Subscription)
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] abstract class MultiStreamInputProcessor(_settings: MaterializerSettings) extends ActorProcessorImpl(_settings) {
|
||||
import akka.stream.impl.MultiStreamInputProcessor._
|
||||
var nextId = 0
|
||||
|
||||
private val substreamInputs = collection.mutable.Map.empty[SubstreamKey, SubstreamInputs]
|
||||
|
||||
class SubstreamInputs(val key: SubstreamKey) extends BatchingInputBuffer(settings.initialInputBufferSize, pump = this) {
|
||||
class SubstreamInput(val key: SubstreamKey, bufferSize: Int, processor: MultiStreamInputProcessorLike, pump: Pump) extends BatchingInputBuffer(bufferSize, pump) {
|
||||
// Not driven directly
|
||||
override val subreceive = new SubReceive(Actor.emptyBehavior)
|
||||
|
||||
|
|
@ -231,11 +241,25 @@ private[akka] abstract class MultiStreamInputProcessor(_settings: MaterializerSe
|
|||
|
||||
override protected def inputOnError(e: Throwable): Unit = {
|
||||
super.inputOnError(e)
|
||||
invalidateSubstream(key, e)
|
||||
processor.invalidateSubstreamInput(key, e)
|
||||
}
|
||||
}
|
||||
|
||||
val substreamManagement: Receive = {
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 = {
|
||||
case SubstreamStreamOnSubscribe(key, subscription) ⇒ substreamInputs(key).substreamOnSubscribe(subscription)
|
||||
case SubstreamOnNext(key, element) ⇒ substreamInputs(key).substreamOnNext(element)
|
||||
case SubstreamOnComplete(key) ⇒ {
|
||||
|
|
@ -246,30 +270,53 @@ private[akka] abstract class MultiStreamInputProcessor(_settings: MaterializerSe
|
|||
|
||||
}
|
||||
|
||||
def createSubstreamInputs(p: Publisher[Any]): SubstreamInputs = {
|
||||
val key = SubstreamKey(nextId)
|
||||
val inputs = new SubstreamInputs(key)
|
||||
p.subscribe(new SubstreamSubscriber(self, key))
|
||||
def createSubstreamInput(): SubstreamInput = {
|
||||
val key = SubstreamKey(nextId())
|
||||
val inputs = new SubstreamInput(key, inputBufferSize, this, this)
|
||||
substreamInputs(key) = inputs
|
||||
nextId += 1
|
||||
inputs
|
||||
}
|
||||
|
||||
protected def invalidateSubstream(substream: SubstreamKey, e: Throwable): Unit = {
|
||||
def createAndSubscribeSubstreamInput(p: Publisher[Any]): SubstreamInput = {
|
||||
val inputs = createSubstreamInput()
|
||||
p.subscribe(new SubstreamSubscriber(self, inputs.key))
|
||||
inputs
|
||||
}
|
||||
|
||||
def invalidateSubstreamInput(substream: SubstreamKey, e: Throwable): Unit = {
|
||||
substreamInputs(substream).cancel()
|
||||
substreamInputs -= substream
|
||||
pump()
|
||||
}
|
||||
|
||||
override def fail(e: Throwable): Unit = {
|
||||
protected def failInputs(e: Throwable): Unit = {
|
||||
substreamInputs.values foreach (_.cancel())
|
||||
}
|
||||
|
||||
protected def finishInputs(): Unit = {
|
||||
substreamInputs.values foreach (_.cancel())
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* 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(): Unit = {
|
||||
substreamInputs.values foreach (_.cancel())
|
||||
override def pumpFinished() = {
|
||||
finishInputs()
|
||||
super.pumpFinished()
|
||||
}
|
||||
|
||||
override def activeReceive = primaryInputs.subreceive orElse primaryOutputs.subreceive orElse substreamManagement
|
||||
override def activeReceive = primaryInputs.subreceive orElse primaryOutputs.subreceive orElse inputSubstreamManagement
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue