2014-03-30 09:27:19 +02:00
|
|
|
/**
|
|
|
|
|
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
|
|
|
|
*/
|
|
|
|
|
package akka.stream.impl
|
|
|
|
|
|
2015-06-19 17:15:50 +02:00
|
|
|
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicLong }
|
2015-01-28 14:19:50 +01:00
|
|
|
|
2014-11-17 22:50:15 +01:00
|
|
|
import akka.actor._
|
2015-01-28 14:19:50 +01:00
|
|
|
import akka.dispatch.Dispatchers
|
|
|
|
|
import akka.pattern.ask
|
2014-10-27 14:35:41 +01:00
|
|
|
import akka.stream.actor.ActorSubscriber
|
2015-01-28 14:19:50 +01:00
|
|
|
import akka.stream.impl.Junctions._
|
|
|
|
|
import akka.stream.impl.StreamLayout.Module
|
2015-08-19 15:22:02 +02:00
|
|
|
import akka.stream.impl.fusing.{ ActorGraphInterpreter, GraphModule, ActorInterpreter }
|
2015-04-20 16:33:57 +02:00
|
|
|
import akka.stream.impl.io.SslTlsCipherActor
|
2015-01-28 14:19:50 +01:00
|
|
|
import akka.stream._
|
2015-04-20 16:33:57 +02:00
|
|
|
import akka.stream.io.SslTls.TlsModule
|
2015-06-14 03:12:30 -04:00
|
|
|
import akka.stream.stage.Stage
|
2015-07-29 17:00:32 -04:00
|
|
|
import akka.stream.Attributes._
|
2015-01-28 14:19:50 +01:00
|
|
|
import org.reactivestreams._
|
2015-01-27 18:29:20 +01:00
|
|
|
|
2015-09-11 15:50:17 +02:00
|
|
|
import scala.concurrent.duration.FiniteDuration
|
2015-02-26 11:58:29 +01:00
|
|
|
import scala.concurrent.{ Await, ExecutionContextExecutor }
|
2014-03-30 09:27:19 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2015-08-01 00:13:14 +02:00
|
|
|
private[akka] case class ActorMaterializerImpl(val system: ActorSystem,
|
|
|
|
|
override val settings: ActorMaterializerSettings,
|
|
|
|
|
dispatchers: Dispatchers,
|
|
|
|
|
val supervisor: ActorRef,
|
|
|
|
|
val haveShutDown: AtomicBoolean,
|
|
|
|
|
flowNameCounter: AtomicLong,
|
|
|
|
|
namePrefix: String,
|
|
|
|
|
optimizations: Optimizations) extends ActorMaterializer {
|
2015-01-28 14:19:50 +01:00
|
|
|
import akka.stream.impl.Stages._
|
2014-10-27 14:35:41 +01:00
|
|
|
|
2015-06-19 17:15:50 +02:00
|
|
|
override def shutdown(): Unit =
|
|
|
|
|
if (haveShutDown.compareAndSet(false, true)) supervisor ! PoisonPill
|
|
|
|
|
|
|
|
|
|
override def isShutdown: Boolean = haveShutDown.get()
|
|
|
|
|
|
2015-06-23 18:28:53 +02:00
|
|
|
override def withNamePrefix(name: String): Materializer = this.copy(namePrefix = name)
|
2014-05-08 19:34:58 +02:00
|
|
|
|
2014-11-09 21:09:50 +01:00
|
|
|
private[this] def nextFlowNameCount(): Long = flowNameCounter.incrementAndGet()
|
2014-05-08 19:34:58 +02:00
|
|
|
|
2014-11-09 21:09:50 +01:00
|
|
|
private[this] def createFlowName(): String = s"$namePrefix-${nextFlowNameCount()}"
|
2014-05-08 19:34:58 +02:00
|
|
|
|
2015-06-23 18:28:53 +02:00
|
|
|
override def effectiveSettings(opAttr: Attributes): ActorMaterializerSettings = {
|
2015-06-23 17:32:55 +02:00
|
|
|
import Attributes._
|
|
|
|
|
import ActorAttributes._
|
|
|
|
|
opAttr.attributeList.foldLeft(settings) { (s, attr) ⇒
|
2015-04-10 14:39:48 +02:00
|
|
|
attr match {
|
|
|
|
|
case InputBuffer(initial, max) ⇒ s.withInputBuffer(initial, max)
|
|
|
|
|
case Dispatcher(dispatcher) ⇒ s.withDispatcher(dispatcher)
|
|
|
|
|
case SupervisionStrategy(decider) ⇒ s.withSupervisionStrategy(decider)
|
2015-04-09 12:21:12 +02:00
|
|
|
case l: LogLevels ⇒ s
|
2015-04-10 14:39:48 +02:00
|
|
|
case Name(_) ⇒ s
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-09-11 15:50:17 +02:00
|
|
|
override def schedulePeriodically(initialDelay: FiniteDuration, interval: FiniteDuration, task: Runnable) =
|
|
|
|
|
system.scheduler.schedule(initialDelay, interval, task)(executionContext)
|
|
|
|
|
|
|
|
|
|
override def scheduleOnce(delay: FiniteDuration, task: Runnable) =
|
|
|
|
|
system.scheduler.scheduleOnce(delay, task)(executionContext)
|
|
|
|
|
|
2015-06-23 18:41:55 +02:00
|
|
|
override def materialize[Mat](runnableGraph: Graph[ClosedShape, Mat]): Mat = {
|
2015-06-19 17:15:50 +02:00
|
|
|
if (haveShutDown.get())
|
|
|
|
|
throw new IllegalStateException("Attempted to call materialize() after the ActorMaterializer has been shut down.")
|
2015-07-06 22:00:21 +02:00
|
|
|
if (StreamLayout.Debug) StreamLayout.validate(runnableGraph.module)
|
2014-11-09 21:09:50 +01:00
|
|
|
|
2015-06-23 18:41:55 +02:00
|
|
|
val session = new MaterializerSession(runnableGraph.module) {
|
2015-01-28 14:19:50 +01:00
|
|
|
private val flowName = createFlowName()
|
|
|
|
|
private var nextId = 0
|
2015-06-23 17:32:55 +02:00
|
|
|
private def stageName(attr: Attributes): String = {
|
2015-04-10 16:49:49 +02:00
|
|
|
val name = s"$flowName-$nextId-${attr.nameOrDefault()}"
|
2015-01-28 14:19:50 +01:00
|
|
|
nextId += 1
|
|
|
|
|
name
|
|
|
|
|
}
|
2014-11-09 21:09:50 +01:00
|
|
|
|
2015-06-23 17:32:55 +02:00
|
|
|
override protected def materializeAtomic(atomic: Module, effectiveAttributes: Attributes): Any = {
|
2015-02-26 22:42:34 +01:00
|
|
|
|
2015-06-14 03:12:30 -04:00
|
|
|
def newMaterializationContext() =
|
|
|
|
|
new MaterializationContext(ActorMaterializerImpl.this, effectiveAttributes, stageName(effectiveAttributes))
|
2015-02-26 22:42:34 +01:00
|
|
|
atomic match {
|
|
|
|
|
case sink: SinkModule[_, _] ⇒
|
2015-04-10 14:39:48 +02:00
|
|
|
val (sub, mat) = sink.create(newMaterializationContext())
|
2015-02-26 22:42:34 +01:00
|
|
|
assignPort(sink.shape.inlet, sub.asInstanceOf[Subscriber[Any]])
|
|
|
|
|
mat
|
|
|
|
|
case source: SourceModule[_, _] ⇒
|
2015-04-10 14:39:48 +02:00
|
|
|
val (pub, mat) = source.create(newMaterializationContext())
|
2015-02-26 22:42:34 +01:00
|
|
|
assignPort(source.shape.outlet, pub.asInstanceOf[Publisher[Any]])
|
|
|
|
|
mat
|
|
|
|
|
|
|
|
|
|
case stage: StageModule ⇒
|
2015-04-10 14:39:48 +02:00
|
|
|
val (processor, mat) = processorFor(stage, effectiveAttributes, effectiveSettings(effectiveAttributes))
|
2015-02-26 22:42:34 +01:00
|
|
|
assignPort(stage.inPort, processor)
|
|
|
|
|
assignPort(stage.outPort, processor)
|
|
|
|
|
mat
|
|
|
|
|
|
2015-06-14 03:12:30 -04:00
|
|
|
case tls: TlsModule ⇒ // TODO solve this so TlsModule doesn't need special treatment here
|
2015-04-20 16:33:57 +02:00
|
|
|
val es = effectiveSettings(effectiveAttributes)
|
2015-06-14 03:12:30 -04:00
|
|
|
val props =
|
2015-08-01 00:13:14 +02:00
|
|
|
SslTlsCipherActor.props(es, tls.sslContext, tls.firstSession, tls.role, tls.closing, tls.hostInfo)
|
2015-04-20 16:33:57 +02:00
|
|
|
val impl = actorOf(props, stageName(effectiveAttributes), es.dispatcher)
|
|
|
|
|
def factory(id: Int) = new ActorPublisher[Any](impl) {
|
|
|
|
|
override val wakeUpMsg = FanOut.SubstreamSubscribePending(id)
|
|
|
|
|
}
|
|
|
|
|
val publishers = Vector.tabulate(2)(factory)
|
|
|
|
|
impl ! FanOut.ExposedPublishers(publishers)
|
|
|
|
|
|
|
|
|
|
assignPort(tls.plainOut, publishers(SslTlsCipherActor.UserOut))
|
|
|
|
|
assignPort(tls.cipherOut, publishers(SslTlsCipherActor.TransportOut))
|
|
|
|
|
|
|
|
|
|
assignPort(tls.plainIn, FanIn.SubInput[Any](impl, SslTlsCipherActor.UserIn))
|
|
|
|
|
assignPort(tls.cipherIn, FanIn.SubInput[Any](impl, SslTlsCipherActor.TransportIn))
|
|
|
|
|
|
2015-08-19 15:22:02 +02:00
|
|
|
case graph: GraphModule ⇒
|
|
|
|
|
val calculatedSettings = effectiveSettings(effectiveAttributes)
|
2015-09-18 14:30:43 +02:00
|
|
|
val (inHandlers, outHandlers, logics, mat) = graph.assembly.materialize()
|
|
|
|
|
|
|
|
|
|
val props = ActorGraphInterpreter.props(
|
|
|
|
|
graph.assembly, inHandlers, outHandlers, logics, graph.shape, calculatedSettings, ActorMaterializerImpl.this)
|
|
|
|
|
|
2015-08-19 15:22:02 +02:00
|
|
|
val impl = actorOf(props, stageName(effectiveAttributes), calculatedSettings.dispatcher)
|
|
|
|
|
for ((inlet, i) ← graph.shape.inlets.iterator.zipWithIndex) {
|
|
|
|
|
val subscriber = new ActorGraphInterpreter.BoundarySubscriber(impl, i)
|
|
|
|
|
assignPort(inlet, subscriber)
|
|
|
|
|
}
|
|
|
|
|
for ((outlet, i) ← graph.shape.outlets.iterator.zipWithIndex) {
|
|
|
|
|
val publisher = new ActorPublisher[Any](impl) { override val wakeUpMsg = ActorGraphInterpreter.SubscribePending(i) }
|
|
|
|
|
impl ! ActorGraphInterpreter.ExposedPublisher(i, publisher)
|
|
|
|
|
assignPort(outlet, publisher)
|
|
|
|
|
}
|
2015-09-18 14:30:43 +02:00
|
|
|
mat
|
2015-08-19 15:22:02 +02:00
|
|
|
|
2015-06-14 03:12:30 -04:00
|
|
|
case junction: JunctionModule ⇒
|
|
|
|
|
materializeJunction(junction, effectiveAttributes, effectiveSettings(effectiveAttributes))
|
2015-02-26 22:42:34 +01:00
|
|
|
}
|
2014-11-09 21:09:50 +01:00
|
|
|
}
|
2014-03-30 09:27:19 +02:00
|
|
|
|
2015-02-26 22:42:34 +01:00
|
|
|
private def processorFor(op: StageModule,
|
2015-06-23 17:32:55 +02:00
|
|
|
effectiveAttributes: Attributes,
|
2015-06-23 18:28:53 +02:00
|
|
|
effectiveSettings: ActorMaterializerSettings): (Processor[Any, Any], Any) = op match {
|
2015-01-28 14:19:50 +01:00
|
|
|
case DirectProcessor(processorFactory, _) ⇒ processorFactory()
|
2015-06-16 15:34:54 +02:00
|
|
|
case Identity(attr) ⇒ (new VirtualProcessor, ())
|
2015-01-28 14:19:50 +01:00
|
|
|
case _ ⇒
|
2015-06-23 18:28:53 +02:00
|
|
|
val (opprops, mat) = ActorProcessorFactory.props(ActorMaterializerImpl.this, op, effectiveAttributes)
|
2015-06-13 16:28:38 -04:00
|
|
|
ActorProcessorFactory[Any, Any](
|
|
|
|
|
actorOf(opprops, stageName(effectiveAttributes), effectiveSettings.dispatcher)) -> mat
|
2015-01-28 14:19:50 +01:00
|
|
|
}
|
2014-10-27 14:35:41 +01:00
|
|
|
|
2015-02-26 22:42:34 +01:00
|
|
|
private def materializeJunction(op: JunctionModule,
|
2015-06-23 17:32:55 +02:00
|
|
|
effectiveAttributes: Attributes,
|
2015-06-23 18:28:53 +02:00
|
|
|
effectiveSettings: ActorMaterializerSettings): Unit = {
|
2015-01-28 14:19:50 +01:00
|
|
|
op match {
|
|
|
|
|
case fanin: FanInModule ⇒
|
|
|
|
|
val (props, inputs, output) = fanin match {
|
|
|
|
|
case f: FlexiMergeModule[t, p] ⇒
|
|
|
|
|
val flexi = f.flexi(f.shape)
|
2015-02-26 22:42:34 +01:00
|
|
|
(FlexiMerge.props(effectiveSettings, f.shape, flexi), f.shape.inlets, f.shape.outlets.head)
|
2015-01-28 14:19:50 +01:00
|
|
|
|
|
|
|
|
}
|
2015-02-26 22:42:34 +01:00
|
|
|
val impl = actorOf(props, stageName(effectiveAttributes), effectiveSettings.dispatcher)
|
2015-01-28 14:19:50 +01:00
|
|
|
val publisher = new ActorPublisher[Any](impl)
|
2015-04-16 16:05:49 +02:00
|
|
|
// Resolve cyclic dependency with actor. This MUST be the first message no matter what.
|
2015-01-28 14:19:50 +01:00
|
|
|
impl ! ExposedPublisher(publisher)
|
|
|
|
|
for ((in, id) ← inputs.zipWithIndex) {
|
|
|
|
|
assignPort(in, FanIn.SubInput[Any](impl, id))
|
|
|
|
|
}
|
|
|
|
|
assignPort(output, publisher)
|
|
|
|
|
|
|
|
|
|
case fanout: FanOutModule ⇒
|
|
|
|
|
val (props, in, outs) = fanout match {
|
2015-02-26 11:58:29 +01:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
case r: FlexiRouteModule[t, p] ⇒
|
|
|
|
|
val flexi = r.flexi(r.shape)
|
2015-02-26 22:42:34 +01:00
|
|
|
(FlexiRoute.props(effectiveSettings, r.shape, flexi), r.shape.inlets.head: InPort, r.shape.outlets)
|
2015-01-28 14:19:50 +01:00
|
|
|
}
|
2015-02-26 22:42:34 +01:00
|
|
|
val impl = actorOf(props, stageName(effectiveAttributes), effectiveSettings.dispatcher)
|
2015-02-26 11:58:29 +01:00
|
|
|
val size = outs.size
|
2015-06-14 03:12:30 -04:00
|
|
|
def factory(id: Int) =
|
|
|
|
|
new ActorPublisher[Any](impl) { override val wakeUpMsg = FanOut.SubstreamSubscribePending(id) }
|
2015-02-26 11:58:29 +01:00
|
|
|
val publishers =
|
|
|
|
|
if (outs.size < 8) Vector.tabulate(size)(factory)
|
|
|
|
|
else List.tabulate(size)(factory)
|
2014-10-27 14:35:41 +01:00
|
|
|
|
2015-06-14 03:12:30 -04:00
|
|
|
impl ! FanOut.ExposedPublishers(publishers)
|
|
|
|
|
publishers.iterator.zip(outs.iterator).foreach { case (pub, out) ⇒ assignPort(out, pub) }
|
|
|
|
|
assignPort(in, ActorSubscriber[Any](impl))
|
2014-10-27 14:35:41 +01:00
|
|
|
}
|
|
|
|
|
}
|
2014-11-28 10:41:57 +01:00
|
|
|
|
|
|
|
|
}
|
2015-01-28 14:19:50 +01:00
|
|
|
|
|
|
|
|
session.materialize().asInstanceOf[Mat]
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
2014-05-07 15:56:02 +02:00
|
|
|
|
2015-04-10 14:39:48 +02:00
|
|
|
override lazy val executionContext: ExecutionContextExecutor = dispatchers.lookup(settings.dispatcher match {
|
2014-11-17 22:50:15 +01:00
|
|
|
case Deploy.NoDispatcherGiven ⇒ Dispatchers.DefaultDispatcherId
|
|
|
|
|
case other ⇒ other
|
|
|
|
|
})
|
|
|
|
|
|
2015-04-10 14:39:48 +02:00
|
|
|
override def actorOf(context: MaterializationContext, props: Props): ActorRef = {
|
|
|
|
|
val dispatcher =
|
|
|
|
|
if (props.deploy.dispatcher == Deploy.NoDispatcherGiven) effectiveSettings(context.effectiveAttributes).dispatcher
|
|
|
|
|
else props.dispatcher
|
|
|
|
|
actorOf(props, context.stageName, dispatcher)
|
|
|
|
|
}
|
2014-12-01 20:07:55 +02:00
|
|
|
|
2015-04-17 09:28:00 +02:00
|
|
|
private[akka] def actorOf(props: Props, name: String, dispatcher: String): ActorRef = {
|
|
|
|
|
supervisor match {
|
|
|
|
|
case ref: LocalActorRef ⇒
|
|
|
|
|
ref.underlying.attachChild(props.withDispatcher(dispatcher), name, systemService = false)
|
|
|
|
|
case ref: RepointableActorRef ⇒
|
|
|
|
|
if (ref.isStarted)
|
|
|
|
|
ref.underlying.asInstanceOf[ActorCell].attachChild(props.withDispatcher(dispatcher), name, systemService = false)
|
|
|
|
|
else {
|
|
|
|
|
implicit val timeout = ref.system.settings.CreationTimeout
|
|
|
|
|
val f = (supervisor ? StreamSupervisor.Materialize(props.withDispatcher(dispatcher), name)).mapTo[ActorRef]
|
|
|
|
|
Await.result(f, timeout.duration)
|
|
|
|
|
}
|
|
|
|
|
case unknown ⇒
|
|
|
|
|
throw new IllegalStateException(s"Stream supervisor must be a local actor, was [${unknown.getClass.getName}]")
|
|
|
|
|
}
|
2014-08-21 12:35:38 +02:00
|
|
|
}
|
2014-05-07 15:56:02 +02:00
|
|
|
|
2014-03-30 09:27:19 +02:00
|
|
|
}
|
2014-05-08 19:34:58 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] object FlowNameCounter extends ExtensionId[FlowNameCounter] with ExtensionIdProvider {
|
|
|
|
|
override def get(system: ActorSystem): FlowNameCounter = super.get(system)
|
2015-08-01 00:13:14 +02:00
|
|
|
override def lookup() = FlowNameCounter
|
2014-05-08 19:34:58 +02:00
|
|
|
override def createExtension(system: ExtendedActorSystem): FlowNameCounter = new FlowNameCounter
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] class FlowNameCounter extends Extension {
|
|
|
|
|
val counter = new AtomicLong(0)
|
2014-08-21 12:35:38 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] object StreamSupervisor {
|
2015-06-25 12:54:29 +02:00
|
|
|
def props(settings: ActorMaterializerSettings, haveShutDown: AtomicBoolean): Props =
|
|
|
|
|
Props(new StreamSupervisor(settings, haveShutDown)).withDeploy(Deploy.local)
|
2014-08-21 12:35:38 +02:00
|
|
|
|
2015-06-14 03:12:30 -04:00
|
|
|
final case class Materialize(props: Props, name: String)
|
|
|
|
|
extends DeadLetterSuppression with NoSerializationVerificationNeeded
|
2015-04-16 02:24:01 +02:00
|
|
|
|
|
|
|
|
/** Testing purpose */
|
2015-08-01 00:13:14 +02:00
|
|
|
case object GetChildren
|
2015-04-16 02:24:01 +02:00
|
|
|
/** Testing purpose */
|
|
|
|
|
final case class Children(children: Set[ActorRef])
|
|
|
|
|
/** Testing purpose */
|
2015-08-01 00:13:14 +02:00
|
|
|
case object StopChildren
|
2015-05-14 12:21:47 +02:00
|
|
|
/** Testing purpose */
|
2015-08-01 00:13:14 +02:00
|
|
|
case object StoppedChildren
|
2014-08-21 12:35:38 +02:00
|
|
|
}
|
|
|
|
|
|
2015-06-25 12:54:29 +02:00
|
|
|
private[akka] class StreamSupervisor(settings: ActorMaterializerSettings, haveShutDown: AtomicBoolean) extends Actor {
|
2015-01-28 14:19:50 +01:00
|
|
|
import akka.stream.impl.StreamSupervisor._
|
2014-08-21 12:35:38 +02:00
|
|
|
|
2014-10-27 14:35:41 +01:00
|
|
|
override def supervisorStrategy = SupervisorStrategy.stoppingStrategy
|
|
|
|
|
|
2014-08-21 12:35:38 +02:00
|
|
|
def receive = {
|
|
|
|
|
case Materialize(props, name) ⇒
|
|
|
|
|
val impl = context.actorOf(props, name)
|
|
|
|
|
sender() ! impl
|
2015-05-14 12:21:47 +02:00
|
|
|
case GetChildren ⇒ sender() ! Children(context.children.toSet)
|
|
|
|
|
case StopChildren ⇒
|
|
|
|
|
context.children.foreach(context.stop)
|
|
|
|
|
sender() ! StoppedChildren
|
2014-08-21 12:35:38 +02:00
|
|
|
}
|
2015-06-25 12:54:29 +02:00
|
|
|
|
|
|
|
|
override def postStop(): Unit = haveShutDown.set(true)
|
2014-10-27 14:35:41 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] object ActorProcessorFactory {
|
2015-01-28 14:19:50 +01:00
|
|
|
import akka.stream.impl.Stages._
|
2014-10-27 14:35:41 +01:00
|
|
|
|
2015-06-23 18:28:53 +02:00
|
|
|
def props(materializer: ActorMaterializer, op: StageModule, parentAttributes: Attributes): (Props, Any) = {
|
2015-01-28 14:19:50 +01:00
|
|
|
val att = parentAttributes and op.attributes
|
|
|
|
|
// USE THIS TO AVOID CLOSING OVER THE MATERIALIZER BELOW
|
|
|
|
|
// Also, otherwise the attributes will not affect the settings properly!
|
2015-04-10 14:39:48 +02:00
|
|
|
val settings = materializer.effectiveSettings(att)
|
2015-06-14 03:12:30 -04:00
|
|
|
def interp(s: Stage[_, _]): (Props, Unit) = (ActorInterpreter.props(settings, List(s), materializer, att), ())
|
2015-07-29 17:00:32 -04:00
|
|
|
def interpAttr(s: Stage[_, _], newAttributes: Attributes): (Props, Unit) = (ActorInterpreter.props(settings, List(s), materializer, newAttributes), ())
|
|
|
|
|
def inputSizeAttr(n: Long) = {
|
|
|
|
|
if (n <= 0)
|
|
|
|
|
inputBuffer(initial = 1, max = 1) and att
|
|
|
|
|
else if (n <= materializer.settings.maxInputBufferSize)
|
|
|
|
|
inputBuffer(initial = n.toInt, max = n.toInt) and att
|
|
|
|
|
else
|
|
|
|
|
att
|
|
|
|
|
}
|
2014-12-01 20:07:55 +02:00
|
|
|
op match {
|
2015-06-14 03:12:30 -04:00
|
|
|
case Map(f, _) ⇒ interp(fusing.Map(f, settings.supervisionDecider))
|
|
|
|
|
case Filter(p, _) ⇒ interp(fusing.Filter(p, settings.supervisionDecider))
|
|
|
|
|
case Drop(n, _) ⇒ interp(fusing.Drop(n))
|
2015-07-29 17:00:32 -04:00
|
|
|
case Take(n, _) ⇒ interpAttr(fusing.Take(n), inputSizeAttr(n))
|
2015-06-14 03:12:30 -04:00
|
|
|
case TakeWhile(p, _) ⇒ interp(fusing.TakeWhile(p, settings.supervisionDecider))
|
|
|
|
|
case DropWhile(p, _) ⇒ interp(fusing.DropWhile(p, settings.supervisionDecider))
|
|
|
|
|
case Collect(pf, _) ⇒ interp(fusing.Collect(pf, settings.supervisionDecider))
|
|
|
|
|
case Scan(z, f, _) ⇒ interp(fusing.Scan(z, f, settings.supervisionDecider))
|
|
|
|
|
case Fold(z, f, _) ⇒ interp(fusing.Fold(z, f, settings.supervisionDecider))
|
2015-08-01 00:13:14 +02:00
|
|
|
case Recover(pf, _) ⇒ interp(fusing.Recover(pf))
|
|
|
|
|
case Expand(s, f, _) ⇒ interp(fusing.Expand(s, f))
|
|
|
|
|
case Conflate(s, f, _) ⇒ interp(fusing.Conflate(s, f, settings.supervisionDecider))
|
|
|
|
|
case Buffer(n, s, _) ⇒ interp(fusing.Buffer(n, s))
|
|
|
|
|
case MapConcat(f, _) ⇒ interp(fusing.MapConcat(f, settings.supervisionDecider))
|
|
|
|
|
case MapAsync(p, f, _) ⇒ interp(fusing.MapAsync(p, f, settings.supervisionDecider))
|
|
|
|
|
case MapAsyncUnordered(p, f, _) ⇒ interp(fusing.MapAsyncUnordered(p, f, settings.supervisionDecider))
|
|
|
|
|
case Grouped(n, _) ⇒ interp(fusing.Grouped(n))
|
|
|
|
|
case Sliding(n, step, _) ⇒ interp(fusing.Sliding(n, step))
|
|
|
|
|
case Log(n, e, l, _) ⇒ interp(fusing.Log(n, e, l))
|
2015-01-28 14:19:50 +01:00
|
|
|
case GroupBy(f, _) ⇒ (GroupByProcessorImpl.props(settings, f), ())
|
|
|
|
|
case PrefixAndTail(n, _) ⇒ (PrefixAndTailImpl.props(settings, n), ())
|
2015-04-14 13:44:24 +02:00
|
|
|
case Split(d, _) ⇒ (SplitWhereProcessorImpl.props(settings, d), ())
|
2015-04-20 15:03:03 +02:00
|
|
|
case ConcatAll(_) ⇒ (ConcatAllImpl.props(materializer), ())
|
2015-06-14 03:12:30 -04:00
|
|
|
case StageFactory(mkStage, _) ⇒ interp(mkStage())
|
2015-01-28 14:19:50 +01:00
|
|
|
case MaterializingStageFactory(mkStageAndMat, _) ⇒
|
2015-06-14 03:12:30 -04:00
|
|
|
val s_m = mkStageAndMat()
|
|
|
|
|
(ActorInterpreter.props(settings, List(s_m._1), materializer, att), s_m._2)
|
2015-02-26 11:58:29 +01:00
|
|
|
case DirectProcessor(p, m) ⇒ throw new AssertionError("DirectProcessor cannot end up in ActorProcessorFactory")
|
2015-07-06 22:00:21 +02:00
|
|
|
case Identity(_) ⇒ throw new AssertionError("Identity cannot end up in ActorProcessorFactory")
|
2014-12-01 20:07:55 +02:00
|
|
|
}
|
2014-10-27 14:35:41 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def apply[I, O](impl: ActorRef): ActorProcessor[I, O] = {
|
|
|
|
|
val p = new ActorProcessor[I, O](impl)
|
2015-04-16 16:05:49 +02:00
|
|
|
// Resolve cyclic dependency with actor. This MUST be the first message no matter what.
|
2014-10-27 14:35:41 +01:00
|
|
|
impl ! ExposedPublisher(p.asInstanceOf[ActorPublisher[Any]])
|
|
|
|
|
p
|
|
|
|
|
}
|
2015-03-06 10:23:26 +01:00
|
|
|
}
|