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-08-21 16:07:09 +02:00
|
|
|
import akka.stream.{ ReactiveStreamsConstants, MaterializerSettings }
|
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-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-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 =
|
|
|
|
|
if (elements <= 0) throw new IllegalArgumentException(ReactiveStreamsConstants.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-03 15:29:02 +01:00
|
|
|
class SubstreamOutput(val key: SubstreamKey, actor: ActorRef, pump: Pump, subscriptionTimeout: SubscriptionTimeout)
|
|
|
|
|
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")
|
|
|
|
|
|
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 = {
|
|
|
|
|
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-03 15:29:02 +01:00
|
|
|
if (subscriptionTimeout.cancelAndHandle(s)) {
|
|
|
|
|
if (state.compareAndSet(Open, Attached(s))) actor ! SubstreamSubscribe(key, s)
|
|
|
|
|
else {
|
|
|
|
|
state.get() match {
|
|
|
|
|
case _: Attached ⇒ s.onError(new IllegalStateException("GroupBy substream publisher " + ReactiveStreamsConstants.SupportsOnlyASingleSubscriber))
|
|
|
|
|
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-03-30 09:27:19 +02:00
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
protected def nextId(): Long
|
|
|
|
|
|
|
|
|
|
private val substreamOutputs = mutable.Map.empty[SubstreamKey, SubstreamOutput]
|
|
|
|
|
|
|
|
|
|
protected def createSubstreamOutput(): SubstreamOutput = {
|
|
|
|
|
val id = SubstreamKey(nextId())
|
2014-11-03 15:29:02 +01:00
|
|
|
val outputs = publisherWithStreamSubscriptionTimeout {
|
|
|
|
|
new SubstreamOutput(id, self, this, _)
|
|
|
|
|
}
|
2014-09-01 13:30:15 +02:00
|
|
|
substreamOutputs(outputs.key) = outputs
|
2014-03-30 09:27:19 +02:00
|
|
|
outputs
|
|
|
|
|
}
|
|
|
|
|
|
2014-09-10 12:14:09 +02:00
|
|
|
protected def invalidateSubstreamOutput(substream: SubstreamKey): Unit = {
|
2014-03-30 09:27:19 +02:00
|
|
|
substreamOutputs(substream).complete()
|
|
|
|
|
substreamOutputs -= substream
|
|
|
|
|
pump()
|
|
|
|
|
}
|
|
|
|
|
|
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-09-01 13:30:15 +02:00
|
|
|
case SubstreamRequestMore(key, demand) ⇒ substreamOutputs(key).enqueueOutputDemand(demand)
|
2014-09-10 12:14:09 +02:00
|
|
|
case SubstreamCancel(key) ⇒ invalidateSubstreamOutput(key)
|
2014-09-01 13:30:15 +02:00
|
|
|
case SubstreamSubscribe(key, subscriber) ⇒ substreamOutputs(key).attachSubscriber(subscriber)
|
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
|
|
|
}
|