2014-03-30 09:27:19 +02:00
|
|
|
/**
|
|
|
|
|
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
|
|
|
|
*/
|
|
|
|
|
package akka.stream.impl
|
|
|
|
|
|
2014-05-08 19:34:58 +02:00
|
|
|
import java.util.concurrent.atomic.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.GenJunctions.ZipWithModule
|
|
|
|
|
import akka.stream.impl.Junctions._
|
|
|
|
|
import akka.stream.impl.StreamLayout.Module
|
|
|
|
|
import akka.stream.impl.fusing.ActorInterpreter
|
2014-10-27 14:35:41 +01:00
|
|
|
import akka.stream.scaladsl._
|
2015-01-28 14:19:50 +01:00
|
|
|
import akka.stream._
|
|
|
|
|
import org.reactivestreams._
|
2015-01-27 18:29:20 +01:00
|
|
|
|
2015-02-26 11:58:29 +01:00
|
|
|
import scala.concurrent.{ Await, ExecutionContextExecutor }
|
2014-03-30 09:27:19 +02:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
2015-03-06 10:23:26 +01:00
|
|
|
private[akka] case class ActorFlowMaterializerImpl(override val settings: ActorFlowMaterializerSettings,
|
|
|
|
|
dispatchers: Dispatchers,
|
|
|
|
|
supervisor: ActorRef,
|
|
|
|
|
flowNameCounter: AtomicLong,
|
|
|
|
|
namePrefix: String,
|
|
|
|
|
optimizations: Optimizations)
|
2015-01-27 18:29:20 +01:00
|
|
|
extends ActorFlowMaterializer {
|
2015-02-26 22:42:34 +01:00
|
|
|
import ActorFlowMaterializerImpl._
|
2015-01-28 14:19:50 +01:00
|
|
|
import akka.stream.impl.Stages._
|
2014-10-27 14:35:41 +01:00
|
|
|
|
2015-04-10 14:39:48 +02:00
|
|
|
override def withNamePrefix(name: String): FlowMaterializer = 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-04-10 14:39:48 +02:00
|
|
|
override def effectiveSettings(opAttr: OperationAttributes): ActorFlowMaterializerSettings = {
|
|
|
|
|
import OperationAttributes._
|
2015-04-10 16:49:49 +02:00
|
|
|
import ActorOperationAttributes._
|
2015-04-10 14:39:48 +02:00
|
|
|
opAttr.attributes.foldLeft(settings) { (s, attr) ⇒
|
|
|
|
|
attr match {
|
|
|
|
|
case InputBuffer(initial, max) ⇒ s.withInputBuffer(initial, max)
|
|
|
|
|
case Dispatcher(dispatcher) ⇒ s.withDispatcher(dispatcher)
|
|
|
|
|
case SupervisionStrategy(decider) ⇒ s.withSupervisionStrategy(decider)
|
|
|
|
|
case Name(_) ⇒ s
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-02-26 11:58:29 +01:00
|
|
|
override def materialize[Mat](runnableFlow: Graph[ClosedShape, Mat]): Mat = {
|
2015-01-28 14:19:50 +01:00
|
|
|
runnableFlow.module.validate()
|
2014-11-09 21:09:50 +01:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
val session = new MaterializerSession(runnableFlow.module) {
|
|
|
|
|
private val flowName = createFlowName()
|
|
|
|
|
private var nextId = 0
|
|
|
|
|
private def stageName(attr: OperationAttributes): 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-02-26 22:42:34 +01:00
|
|
|
override protected def materializeAtomic(atomic: Module, effectiveAttributes: OperationAttributes): Any = {
|
|
|
|
|
|
2015-04-10 14:39:48 +02:00
|
|
|
def newMaterializationContext() = new MaterializationContext(ActorFlowMaterializerImpl.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-04-10 14:39:48 +02: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,
|
|
|
|
|
effectiveAttributes: OperationAttributes,
|
|
|
|
|
effectiveSettings: ActorFlowMaterializerSettings): (Processor[Any, Any], Any) = op match {
|
2015-01-28 14:19:50 +01:00
|
|
|
case DirectProcessor(processorFactory, _) ⇒ processorFactory()
|
|
|
|
|
case _ ⇒
|
|
|
|
|
val (opprops, mat) = ActorProcessorFactory.props(ActorFlowMaterializerImpl.this, op, effectiveAttributes)
|
|
|
|
|
val processor = ActorProcessorFactory[Any, Any](actorOf(
|
|
|
|
|
opprops,
|
|
|
|
|
stageName(effectiveAttributes),
|
2015-02-26 22:42:34 +01:00
|
|
|
effectiveSettings.dispatcher))
|
2015-01-28 14:19:50 +01:00
|
|
|
processor -> mat
|
|
|
|
|
}
|
2014-10-27 14:35:41 +01:00
|
|
|
|
2015-02-26 22:42:34 +01:00
|
|
|
private def materializeJunction(op: JunctionModule,
|
|
|
|
|
effectiveAttributes: OperationAttributes,
|
|
|
|
|
effectiveSettings: ActorFlowMaterializerSettings): Unit = {
|
2015-01-28 14:19:50 +01:00
|
|
|
op match {
|
|
|
|
|
case fanin: FanInModule ⇒
|
|
|
|
|
val (props, inputs, output) = fanin match {
|
2015-02-26 11:58:29 +01:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
case MergeModule(shape, _) ⇒
|
2015-02-26 22:42:34 +01:00
|
|
|
(FairMerge.props(effectiveSettings, shape.inArray.size), shape.inArray.toSeq, shape.out)
|
2015-01-28 14:19:50 +01:00
|
|
|
|
|
|
|
|
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
|
|
|
|
|
|
|
|
case MergePreferredModule(shape, _) ⇒
|
2015-02-26 22:42:34 +01:00
|
|
|
(UnfairMerge.props(effectiveSettings, shape.inlets.size), shape.preferred +: shape.inArray.toSeq, shape.out)
|
2015-01-28 14:19:50 +01:00
|
|
|
|
|
|
|
|
case ConcatModule(shape, _) ⇒
|
2015-04-20 15:03:03 +02:00
|
|
|
require(shape.inArray.size == 2, "currently only supporting concatenation of exactly two inputs") // TODO
|
2015-02-26 22:42:34 +01:00
|
|
|
(Concat.props(effectiveSettings), shape.inArray.toSeq, shape.out)
|
2015-01-28 14:19:50 +01:00
|
|
|
|
|
|
|
|
case zip: ZipWithModule ⇒
|
2015-02-26 22:42:34 +01:00
|
|
|
(zip.props(effectiveSettings), zip.shape.inlets, zip.outPorts.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-02-26 11:58:29 +01:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
case BroadcastModule(shape, _) ⇒
|
2015-02-26 22:42:34 +01:00
|
|
|
(Broadcast.props(effectiveSettings, shape.outArray.size), shape.in, shape.outArray.toSeq)
|
2015-02-26 11:58:29 +01:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
case BalanceModule(shape, waitForDownstreams, _) ⇒
|
2015-02-26 22:42:34 +01:00
|
|
|
(Balance.props(effectiveSettings, shape.outArray.size, waitForDownstreams), shape.in, shape.outArray.toSeq)
|
2015-02-26 11:58:29 +01:00
|
|
|
|
2015-01-28 14:19:50 +01:00
|
|
|
case UnzipModule(shape, _) ⇒
|
2015-02-26 22:42:34 +01:00
|
|
|
(Unzip.props(effectiveSettings), shape.in, 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
|
|
|
|
|
def factory(id: Int) = new ActorPublisher[Any](impl) {
|
2015-01-28 14:19:50 +01:00
|
|
|
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)
|
2015-01-28 14:19:50 +01:00
|
|
|
impl ! FanOut.ExposedPublishers(publishers)
|
|
|
|
|
|
|
|
|
|
publishers.zip(outs).foreach { case (pub, out) ⇒ assignPort(out, pub) }
|
|
|
|
|
val subscriber = ActorSubscriber[Any](impl)
|
|
|
|
|
assignPort(in, subscriber)
|
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
|
|
|
|
|
|
|
|
private[akka] def actorOf(props: Props, name: String, dispatcher: String): ActorRef = supervisor match {
|
2014-08-21 12:35:38 +02:00
|
|
|
case ref: LocalActorRef ⇒
|
2014-12-01 20:07:55 +02:00
|
|
|
ref.underlying.attachChild(props.withDispatcher(dispatcher), name, systemService = false)
|
2014-08-21 12:35:38 +02:00
|
|
|
case ref: RepointableActorRef ⇒
|
|
|
|
|
if (ref.isStarted)
|
2014-12-01 20:07:55 +02:00
|
|
|
ref.underlying.asInstanceOf[ActorCell].attachChild(props.withDispatcher(dispatcher), name, systemService = false)
|
2014-08-21 12:35:38 +02:00
|
|
|
else {
|
|
|
|
|
implicit val timeout = ref.system.settings.CreationTimeout
|
2014-12-01 20:07:55 +02:00
|
|
|
val f = (supervisor ? StreamSupervisor.Materialize(props.withDispatcher(dispatcher), name)).mapTo[ActorRef]
|
2014-08-21 12:35:38 +02:00
|
|
|
Await.result(f, timeout.duration)
|
|
|
|
|
}
|
2014-11-09 21:09:50 +01:00
|
|
|
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)
|
|
|
|
|
override def lookup = FlowNameCounter
|
|
|
|
|
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-01-27 18:29:20 +01:00
|
|
|
def props(settings: ActorFlowMaterializerSettings): Props = Props(new StreamSupervisor(settings))
|
2014-08-21 12:35:38 +02:00
|
|
|
|
2015-01-23 17:18:09 +01:00
|
|
|
final case class Materialize(props: Props, name: String) extends DeadLetterSuppression
|
2014-08-21 12:35:38 +02:00
|
|
|
}
|
|
|
|
|
|
2015-01-27 18:29:20 +01:00
|
|
|
private[akka] class StreamSupervisor(settings: ActorFlowMaterializerSettings) 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
|
|
|
|
|
}
|
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._
|
2015-02-26 22:42:34 +01:00
|
|
|
import ActorFlowMaterializerImpl._
|
2014-10-27 14:35:41 +01:00
|
|
|
|
2015-04-09 22:28:16 +02:00
|
|
|
private val _identity = (x: Any) ⇒ x
|
|
|
|
|
|
2015-04-10 14:39:48 +02:00
|
|
|
def props(materializer: ActorFlowMaterializer, op: StageModule, parentAttributes: OperationAttributes): (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)
|
2014-12-01 20:07:55 +02:00
|
|
|
op match {
|
2015-04-09 22:28:16 +02:00
|
|
|
case Identity(_) ⇒ (ActorInterpreter.props(settings, List(fusing.Map(_identity, settings.supervisionDecider)), materializer), ())
|
|
|
|
|
case Fused(ops, _) ⇒ (ActorInterpreter.props(settings, ops, materializer), ())
|
|
|
|
|
case Map(f, _) ⇒ (ActorInterpreter.props(settings, List(fusing.Map(f, settings.supervisionDecider)), materializer), ())
|
|
|
|
|
case Filter(p, _) ⇒ (ActorInterpreter.props(settings, List(fusing.Filter(p, settings.supervisionDecider)), materializer), ())
|
|
|
|
|
case Drop(n, _) ⇒ (ActorInterpreter.props(settings, List(fusing.Drop(n)), materializer), ())
|
|
|
|
|
case Take(n, _) ⇒ (ActorInterpreter.props(settings, List(fusing.Take(n)), materializer), ())
|
|
|
|
|
case Collect(pf, _) ⇒ (ActorInterpreter.props(settings, List(fusing.Collect(settings.supervisionDecider)(pf)), materializer), ())
|
|
|
|
|
case Scan(z, f, _) ⇒ (ActorInterpreter.props(settings, List(fusing.Scan(z, f, settings.supervisionDecider)), materializer), ())
|
|
|
|
|
case Expand(s, f, _) ⇒ (ActorInterpreter.props(settings, List(fusing.Expand(s, f)), materializer), ())
|
|
|
|
|
case Conflate(s, f, _) ⇒ (ActorInterpreter.props(settings, List(fusing.Conflate(s, f, settings.supervisionDecider)), materializer), ())
|
|
|
|
|
case Buffer(n, s, _) ⇒ (ActorInterpreter.props(settings, List(fusing.Buffer(n, s)), materializer), ())
|
|
|
|
|
case MapConcat(f, _) ⇒ (ActorInterpreter.props(settings, List(fusing.MapConcat(f, settings.supervisionDecider)), materializer), ())
|
|
|
|
|
case MapAsync(p, f, _) ⇒ (ActorInterpreter.props(settings, List(fusing.MapAsync(p, f, settings.supervisionDecider)), materializer), ())
|
|
|
|
|
case MapAsyncUnordered(p, f, _) ⇒ (ActorInterpreter.props(settings, List(fusing.MapAsyncUnordered(p, f, settings.supervisionDecider)), materializer), ())
|
|
|
|
|
case Grouped(n, _) ⇒ (ActorInterpreter.props(settings, List(fusing.Grouped(n)), materializer), ())
|
2015-01-28 14:19:50 +01:00
|
|
|
case GroupBy(f, _) ⇒ (GroupByProcessorImpl.props(settings, f), ())
|
|
|
|
|
case PrefixAndTail(n, _) ⇒ (PrefixAndTailImpl.props(settings, n), ())
|
|
|
|
|
case SplitWhen(p, _) ⇒ (SplitWhenProcessorImpl.props(settings, p), ())
|
2015-04-20 15:03:03 +02:00
|
|
|
case ConcatAll(_) ⇒ (ConcatAllImpl.props(materializer), ())
|
2015-04-09 22:28:16 +02:00
|
|
|
case StageFactory(mkStage, _) ⇒ (ActorInterpreter.props(settings, List(mkStage()), materializer), ())
|
2015-01-28 14:19:50 +01:00
|
|
|
case TimerTransform(mkStage, _) ⇒ (TimerTransformerProcessorsImpl.props(settings, mkStage()), ())
|
|
|
|
|
case MaterializingStageFactory(mkStageAndMat, _) ⇒
|
2015-04-09 22:28:16 +02:00
|
|
|
val sm = mkStageAndMat()
|
|
|
|
|
(ActorInterpreter.props(settings, List(sm._1), materializer), sm._2)
|
2015-02-26 11:58:29 +01:00
|
|
|
case DirectProcessor(p, m) ⇒ throw new AssertionError("DirectProcessor 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
|
|
|
}
|