pekko/akka-stream/src/main/scala/akka/stream/impl/ActorMaterializerImpl.scala

157 lines
4.9 KiB
Scala
Raw Normal View History

/**
2017-01-04 17:37:10 +01:00
* Copyright (C) 2014-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.stream.impl
import java.util.concurrent.atomic.AtomicBoolean
import akka.actor._
2016-07-27 13:29:23 +02:00
import akka.event.LoggingAdapter
import akka.pattern.ask
import akka.stream._
2016-07-27 13:29:23 +02:00
import akka.stream.impl.fusing.GraphInterpreterShell
import scala.concurrent.duration.FiniteDuration
2015-02-26 11:58:29 +01:00
import scala.concurrent.{ Await, ExecutionContextExecutor }
/**
* ExtendedActorMaterializer used by subtypes which materializer using GraphInterpreterShell
*/
abstract class ExtendedActorMaterializer extends ActorMaterializer {
override def withNamePrefix(name: String): ExtendedActorMaterializer
/**
* INTERNAL API
*/
def materialize[Mat](
_runnableGraph: Graph[ClosedShape, Mat],
subflowFuser: GraphInterpreterShell ActorRef): Mat
/**
* INTERNAL API
*/
def materialize[Mat](
_runnableGraph: Graph[ClosedShape, Mat],
subflowFuser: GraphInterpreterShell ActorRef,
initialAttributes: Attributes): Mat
/**
* INTERNAL API
*/
override def actorOf(context: MaterializationContext, props: Props): ActorRef = {
val dispatcher =
if (props.deploy.dispatcher == Deploy.NoDispatcherGiven) effectiveSettings(context.effectiveAttributes).dispatcher
else props.dispatcher
2017-03-07 10:43:18 +01:00
actorOf(props.withDispatcher(dispatcher), context.islandName)
}
/**
* INTERNAL API
*/
2017-03-07 10:43:18 +01:00
def actorOf(props: Props, name: String): ActorRef = {
supervisor match {
case ref: LocalActorRef
2017-03-07 10:43:18 +01:00
ref.underlying.attachChild(props, name, systemService = false)
case ref: RepointableActorRef
if (ref.isStarted)
2017-03-07 10:43:18 +01:00
ref.underlying.asInstanceOf[ActorCell].attachChild(props, name, systemService = false)
else {
implicit val timeout = ref.system.settings.CreationTimeout
2017-03-07 10:43:18 +01:00
val f = (supervisor ? StreamSupervisor.Materialize(props, 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}]")
}
}
/**
* INTERNAL API
*/
override def logger: LoggingAdapter
/**
* INTERNAL API
*/
override def supervisor: ActorRef
}
private[akka] class SubFusingActorMaterializerImpl(val delegate: ExtendedActorMaterializer, registerShell: GraphInterpreterShell ActorRef) extends Materializer {
override def executionContext: ExecutionContextExecutor = delegate.executionContext
override def materialize[Mat](runnable: Graph[ClosedShape, Mat]): Mat = delegate.materialize(runnable, registerShell)
override def materialize[Mat](runnable: Graph[ClosedShape, Mat], initialAttributes: Attributes): Mat =
delegate.materialize(runnable, registerShell, initialAttributes)
override def scheduleOnce(delay: FiniteDuration, task: Runnable): Cancellable = delegate.scheduleOnce(delay, task)
override def schedulePeriodically(initialDelay: FiniteDuration, interval: FiniteDuration, task: Runnable): Cancellable =
delegate.schedulePeriodically(initialDelay, interval, task)
override def withNamePrefix(name: String): SubFusingActorMaterializerImpl =
new SubFusingActorMaterializerImpl(delegate.withNamePrefix(name), registerShell)
}
/**
* INTERNAL API
*/
object FlowNames extends ExtensionId[FlowNames] with ExtensionIdProvider {
2015-11-23 15:50:58 +01:00
override def get(system: ActorSystem): FlowNames = super.get(system)
override def lookup() = FlowNames
override def createExtension(system: ExtendedActorSystem): FlowNames = new FlowNames
}
/**
* INTERNAL API
*/
class FlowNames extends Extension {
val name = SeqActorName("Flow")
}
/**
* INTERNAL API
*/
object StreamSupervisor {
def props(settings: ActorMaterializerSettings, haveShutDown: AtomicBoolean): Props =
Props(new StreamSupervisor(settings, haveShutDown)).withDeploy(Deploy.local)
private[stream] val baseName = "StreamSupervisor"
private val actorName = SeqActorName(baseName)
def nextName(): String = actorName.next()
final case class Materialize(props: Props, name: String)
extends DeadLetterSuppression with NoSerializationVerificationNeeded
/** Testing purpose */
2015-08-01 00:13:14 +02:00
case object GetChildren
/** Testing purpose */
final case class Children(children: Set[ActorRef])
/** Testing purpose */
2015-08-01 00:13:14 +02:00
case object StopChildren
/** Testing purpose */
2015-08-01 00:13:14 +02:00
case object StoppedChildren
/** Testing purpose */
case object PrintDebugDump
}
class StreamSupervisor(settings: ActorMaterializerSettings, haveShutDown: AtomicBoolean) extends Actor {
import akka.stream.impl.StreamSupervisor._
override def supervisorStrategy = SupervisorStrategy.stoppingStrategy
def receive = {
case Materialize(props, name)
val impl = context.actorOf(props, name)
sender() ! impl
case GetChildren sender() ! Children(context.children.toSet)
case StopChildren
context.children.foreach(context.stop)
sender() ! StoppedChildren
}
override def postStop(): Unit = haveShutDown.set(true)
}