New materializer and layout
This commit is contained in:
parent
1989ef481d
commit
ba63c7af8d
60 changed files with 4420 additions and 3181 deletions
|
|
@ -4,26 +4,15 @@
|
|||
package akka.stream.impl
|
||||
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import java.{ util ⇒ ju }
|
||||
|
||||
import akka.NotUsed
|
||||
import akka.actor._
|
||||
import akka.event.{ Logging, LoggingAdapter }
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.pattern.ask
|
||||
import akka.stream._
|
||||
import akka.stream.impl.StreamLayout.{ AtomicModule, Module }
|
||||
import akka.stream.impl.fusing.{ ActorGraphInterpreter, GraphModule }
|
||||
import akka.stream.impl.io.TLSActor
|
||||
import akka.stream.impl.io.TlsModule
|
||||
import org.reactivestreams._
|
||||
import akka.stream.impl.fusing.GraphInterpreterShell
|
||||
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.concurrent.{ Await, ExecutionContextExecutor }
|
||||
import akka.stream.impl.fusing.GraphStageModule
|
||||
import akka.stream.impl.fusing.GraphInterpreter.GraphAssembly
|
||||
import akka.stream.impl.fusing.Fusing
|
||||
import akka.stream.impl.fusing.GraphInterpreterShell
|
||||
|
||||
/**
|
||||
* ExtendedActorMaterializer used by subtypes which materializer using GraphInterpreterShell
|
||||
|
|
@ -60,7 +49,8 @@ abstract class ExtendedActorMaterializer extends ActorMaterializer {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
protected def actorOf(props: Props, name: String, dispatcher: String): ActorRef = {
|
||||
// TODO: hide it again
|
||||
def actorOf(props: Props, name: String, dispatcher: String): ActorRef = {
|
||||
supervisor match {
|
||||
case ref: LocalActorRef ⇒
|
||||
ref.underlying.attachChild(props.withDispatcher(dispatcher), name, systemService = false)
|
||||
|
|
@ -89,183 +79,6 @@ abstract class ExtendedActorMaterializer extends ActorMaterializer {
|
|||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] case class ActorMaterializerImpl(
|
||||
system: ActorSystem,
|
||||
override val settings: ActorMaterializerSettings,
|
||||
dispatchers: Dispatchers,
|
||||
supervisor: ActorRef,
|
||||
haveShutDown: AtomicBoolean,
|
||||
flowNames: SeqActorName) extends ExtendedActorMaterializer {
|
||||
import akka.stream.impl.Stages._
|
||||
private val _logger = Logging.getLogger(system, this)
|
||||
override def logger = _logger
|
||||
|
||||
if (settings.fuzzingMode && !system.settings.config.hasPath("akka.stream.secret-test-fuzzing-warning-disable")) {
|
||||
_logger.warning("Fuzzing mode is enabled on this system. If you see this warning on your production system then " +
|
||||
"set akka.stream.materializer.debug.fuzzing-mode to off.")
|
||||
}
|
||||
|
||||
override def shutdown(): Unit =
|
||||
if (haveShutDown.compareAndSet(false, true)) supervisor ! PoisonPill
|
||||
|
||||
override def isShutdown: Boolean = haveShutDown.get()
|
||||
|
||||
override def withNamePrefix(name: String): ActorMaterializerImpl = this.copy(flowNames = flowNames.copy(name))
|
||||
|
||||
private[this] def createFlowName(): String = flowNames.next()
|
||||
|
||||
private val defaultInitialAttributes = Attributes(
|
||||
Attributes.InputBuffer(settings.initialInputBufferSize, settings.maxInputBufferSize) ::
|
||||
ActorAttributes.Dispatcher(settings.dispatcher) ::
|
||||
ActorAttributes.SupervisionStrategy(settings.supervisionDecider) ::
|
||||
Nil)
|
||||
|
||||
override def effectiveSettings(opAttr: Attributes): ActorMaterializerSettings = {
|
||||
import Attributes._
|
||||
import ActorAttributes._
|
||||
opAttr.attributeList.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 _ ⇒ s
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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)
|
||||
|
||||
override def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat]): Mat =
|
||||
materialize(_runnableGraph, null, defaultInitialAttributes)
|
||||
|
||||
override def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat], initialAttributes: Attributes): Mat =
|
||||
materialize(_runnableGraph, null, initialAttributes)
|
||||
|
||||
override def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat], subflowFuser: (GraphInterpreterShell) ⇒ ActorRef): Mat =
|
||||
materialize(_runnableGraph, subflowFuser, defaultInitialAttributes)
|
||||
|
||||
override def materialize[Mat](
|
||||
_runnableGraph: Graph[ClosedShape, Mat],
|
||||
subflowFuser: GraphInterpreterShell ⇒ ActorRef,
|
||||
initialAttributes: Attributes
|
||||
): Mat = {
|
||||
val runnableGraph =
|
||||
if (settings.autoFusing) Fusing.aggressive(_runnableGraph)
|
||||
else _runnableGraph
|
||||
|
||||
if (haveShutDown.get())
|
||||
throw new IllegalStateException("Attempted to call materialize() after the ActorMaterializer has been shut down.")
|
||||
if (StreamLayout.Debug) StreamLayout.validate(runnableGraph.module)
|
||||
|
||||
val session = new MaterializerSession(runnableGraph.module, initialAttributes) {
|
||||
private val flowName = createFlowName()
|
||||
private var nextId = 0
|
||||
private def stageName(attr: Attributes): String = {
|
||||
val name = s"$flowName-$nextId-${attr.nameOrDefault()}"
|
||||
nextId += 1
|
||||
name
|
||||
}
|
||||
|
||||
override protected def materializeAtomic(atomic: AtomicModule, effectiveAttributes: Attributes, matVal: ju.Map[Module, Any]): Unit = {
|
||||
if (MaterializerSession.Debug) println(s"materializing $atomic")
|
||||
|
||||
def newMaterializationContext() =
|
||||
new MaterializationContext(ActorMaterializerImpl.this, effectiveAttributes, stageName(effectiveAttributes))
|
||||
atomic match {
|
||||
case sink: SinkModule[_, _] ⇒
|
||||
val (sub, mat) = sink.create(newMaterializationContext())
|
||||
assignPort(sink.shape.in, sub)
|
||||
matVal.put(atomic, mat)
|
||||
case source: SourceModule[_, _] ⇒
|
||||
val (pub, mat) = source.create(newMaterializationContext())
|
||||
assignPort(source.shape.out, pub.asInstanceOf[Publisher[Any]])
|
||||
matVal.put(atomic, mat)
|
||||
|
||||
case stage: ProcessorModule[_, _, _] ⇒
|
||||
val (processor, mat) = stage.createProcessor()
|
||||
assignPort(stage.inPort, processor)
|
||||
assignPort(stage.outPort, processor.asInstanceOf[Publisher[Any]])
|
||||
matVal.put(atomic, mat)
|
||||
|
||||
case tls: TlsModule ⇒ // TODO solve this so TlsModule doesn't need special treatment here
|
||||
val es = effectiveSettings(effectiveAttributes)
|
||||
val props =
|
||||
TLSActor.props(es, tls.createSSLEngine, tls.verifySession, tls.closing)
|
||||
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(TLSActor.UserOut))
|
||||
assignPort(tls.cipherOut, publishers(TLSActor.TransportOut))
|
||||
|
||||
assignPort(tls.plainIn, FanIn.SubInput[Any](impl, TLSActor.UserIn))
|
||||
assignPort(tls.cipherIn, FanIn.SubInput[Any](impl, TLSActor.TransportIn))
|
||||
|
||||
matVal.put(atomic, NotUsed)
|
||||
|
||||
case graph: GraphModule ⇒
|
||||
matGraph(graph, effectiveAttributes, matVal)
|
||||
|
||||
case stage: GraphStageModule ⇒
|
||||
val graph =
|
||||
GraphModule(
|
||||
GraphAssembly(stage.shape.inlets, stage.shape.outlets, stage.stage),
|
||||
stage.shape, stage.attributes, Array(stage))
|
||||
matGraph(graph, effectiveAttributes, matVal)
|
||||
}
|
||||
}
|
||||
|
||||
private def matGraph(graph: GraphModule, effectiveAttributes: Attributes, matVal: ju.Map[Module, Any]): Unit = {
|
||||
val calculatedSettings = effectiveSettings(effectiveAttributes)
|
||||
val (connections, logics) = graph.assembly.materialize(effectiveAttributes, graph.matValIDs, matVal, registerSrc)
|
||||
|
||||
val shell = new GraphInterpreterShell(graph.assembly, connections, logics, graph.shape,
|
||||
calculatedSettings, ActorMaterializerImpl.this)
|
||||
|
||||
val impl =
|
||||
if (subflowFuser != null && !effectiveAttributes.contains(Attributes.AsyncBoundary)) {
|
||||
subflowFuser(shell)
|
||||
} else {
|
||||
val props = ActorGraphInterpreter.props(shell)
|
||||
actorOf(props, stageName(effectiveAttributes), calculatedSettings.dispatcher)
|
||||
}
|
||||
|
||||
for ((inlet, i) ← graph.shape.inlets.iterator.zipWithIndex) {
|
||||
val subscriber = new ActorGraphInterpreter.BoundarySubscriber(impl, shell, i)
|
||||
assignPort(inlet, subscriber)
|
||||
}
|
||||
for ((outlet, i) ← graph.shape.outlets.iterator.zipWithIndex) {
|
||||
val publisher = new ActorGraphInterpreter.BoundaryPublisher(impl, shell, i)
|
||||
impl ! ActorGraphInterpreter.ExposedPublisher(shell, i, publisher)
|
||||
assignPort(outlet, publisher)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
session.materialize().asInstanceOf[Mat]
|
||||
}
|
||||
|
||||
override def makeLogger(logSource: Class[_]): LoggingAdapter =
|
||||
Logging(system, logSource)
|
||||
|
||||
override lazy val executionContext: ExecutionContextExecutor = dispatchers.lookup(settings.dispatcher match {
|
||||
case Deploy.NoDispatcherGiven ⇒ Dispatchers.DefaultDispatcherId
|
||||
case other ⇒ other
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
private[akka] class SubFusingActorMaterializerImpl(val delegate: ExtendedActorMaterializer, registerShell: GraphInterpreterShell ⇒ ActorRef) extends Materializer {
|
||||
override def executionContext: ExecutionContextExecutor = delegate.executionContext
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue