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

219 lines
7.4 KiB
Scala
Raw Normal View History

/*
* Copyright (C) 2014-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.stream.impl
import java.util.concurrent.atomic.AtomicBoolean
import akka.actor._
import akka.annotation.{ DoNotInherit, InternalApi }
import akka.dispatch.Dispatchers
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 akka.util.OptionVal
import scala.concurrent.duration.FiniteDuration
import scala.concurrent.{ Await, ExecutionContextExecutor }
/**
* ExtendedActorMaterializer used by subtypes which delegates in-island wiring to [[akka.stream.impl.PhaseIsland]]s
*/
@DoNotInherit private[akka] abstract class ExtendedActorMaterializer extends ActorMaterializer {
override def withNamePrefix(name: String): ExtendedActorMaterializer
/** INTERNAL API */
@InternalApi def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat]): Mat
/** INTERNAL API */
2019-03-11 10:38:24 +01:00
@InternalApi def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat], defaultAttributes: Attributes): Mat
/** INTERNAL API */
2019-03-13 10:56:20 +01:00
@InternalApi private[akka] def materialize[Mat](
graph: Graph[ClosedShape, Mat],
defaultAttributes: Attributes,
defaultPhase: Phase[Any],
phases: Map[IslandTag, Phase[Any]]): Mat
/**
* INTERNAL API
*/
@InternalApi private[akka] override def actorOf(context: MaterializationContext, props: Props): ActorRef = {
val effectiveProps = props.dispatcher match {
case Dispatchers.DefaultDispatcherId =>
props.withDispatcher(context.effectiveAttributes.mandatoryAttribute[ActorAttributes.Dispatcher].dispatcher)
case ActorAttributes.IODispatcher.dispatcher =>
// this one is actually not a dispatcher but a relative config key pointing containing the actual dispatcher name
props.withDispatcher(settings.blockingIoDispatcher)
case _ => props
}
actorOf(effectiveProps, context.islandName)
}
/**
* INTERNAL API
*/
@InternalApi private[akka] 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
*/
@InternalApi private[akka] override def logger: LoggingAdapter
/**
* INTERNAL API
*/
@InternalApi private[akka] override def supervisor: ActorRef
}
/**
* This materializer replaces the default phase with one that will fuse operators into an existing interpreter (via `registerShell`),
* rather than start a new actor for each of them.
*
* The default phases are left in-tact since we still respect `.async` and other tags that were marked within a sub-fused graph.
*/
2019-03-13 10:56:20 +01:00
private[akka] class SubFusingActorMaterializerImpl(
val delegate: ExtendedActorMaterializer,
registerShell: GraphInterpreterShell => ActorRef)
2019-03-11 10:38:24 +01:00
extends Materializer {
val subFusingPhase = new Phase[Any] {
2019-03-13 10:56:20 +01:00
override def apply(
settings: ActorMaterializerSettings,
attributes: Attributes,
materializer: PhasedFusingActorMaterializer,
islandName: String): PhaseIsland[Any] = {
2019-03-11 10:38:24 +01:00
new GraphStageIsland(settings, attributes, materializer, islandName, OptionVal(registerShell))
.asInstanceOf[PhaseIsland[Any]]
}
}
override def executionContext: ExecutionContextExecutor = delegate.executionContext
override def materialize[Mat](runnable: Graph[ClosedShape, Mat]): Mat =
delegate match {
case am: PhasedFusingActorMaterializer =>
materialize(runnable, am.defaultAttributes)
case other =>
2019-03-11 10:38:24 +01:00
throw new IllegalStateException(
s"SubFusing only supported by [PhasedFusingActorMaterializer], " +
s"yet was used with [${other.getClass.getName}]!")
}
override def materialize[Mat](runnable: Graph[ClosedShape, Mat], defaultAttributes: Attributes): Mat = {
if (PhasedFusingActorMaterializer.Debug) println(s"Using [${getClass.getSimpleName}] to materialize [${runnable}]")
val phases = PhasedFusingActorMaterializer.DefaultPhases
delegate.materialize(runnable, defaultAttributes, subFusingPhase, phases)
}
override def scheduleOnce(delay: FiniteDuration, task: Runnable): Cancellable = delegate.scheduleOnce(delay, task)
2019-03-13 10:56:20 +01:00
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
*/
@InternalApi private[akka] 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
*/
@InternalApi private[akka] class FlowNames extends Extension {
val name = SeqActorName("Flow")
}
/**
* INTERNAL API
*/
@InternalApi private[akka] object StreamSupervisor {
def props(settings: ActorMaterializerSettings, haveShutDown: AtomicBoolean): Props =
2019-03-11 10:38:24 +01:00
Props(new StreamSupervisor(haveShutDown)).withDeploy(Deploy.local).withDispatcher(settings.dispatcher)
private[stream] val baseName = "StreamSupervisor"
private val actorName = SeqActorName(baseName)
def nextName(): String = actorName.next()
final case class Materialize(props: Props, name: String)
2019-03-11 10:38:24 +01:00
extends DeadLetterSuppression
with NoSerializationVerificationNeeded
final case class AddFunctionRef(f: (ActorRef, Any) => Unit, name: String)
extends DeadLetterSuppression
with NoSerializationVerificationNeeded
final case class RemoveFunctionRef(ref: FunctionRef)
extends DeadLetterSuppression
with NoSerializationVerificationNeeded
/** Testing purpose */
2015-08-01 00:13:14 +02:00
case object GetChildren
2019-03-11 10:38:24 +01:00
/** Testing purpose */
final case class Children(children: Set[ActorRef])
2019-03-11 10:38:24 +01:00
/** Testing purpose */
2015-08-01 00:13:14 +02:00
case object StopChildren
2019-03-11 10:38:24 +01:00
/** Testing purpose */
2015-08-01 00:13:14 +02:00
case object StoppedChildren
}
/**
* INTERNAL API
*/
@InternalApi private[akka] class StreamSupervisor(haveShutDown: AtomicBoolean) extends Actor {
import akka.stream.impl.StreamSupervisor._
override def supervisorStrategy: SupervisorStrategy = SupervisorStrategy.stoppingStrategy
def receive = {
case Materialize(props, name) =>
val impl = context.actorOf(props, name)
sender() ! impl
case AddFunctionRef(f, name) =>
val ref = context.asInstanceOf[ActorCell].addFunctionRef(f, name)
sender() ! ref
case RemoveFunctionRef(ref) =>
context.asInstanceOf[ActorCell].removeFunctionRef(ref)
case GetChildren =>
sender() ! Children(context.children.toSet)
case StopChildren =>
context.children.foreach(context.stop)
sender() ! StoppedChildren
}
override def postStop(): Unit = haveShutDown.set(true)
}