add fusing
This commit is contained in:
parent
0fb6654f4f
commit
a20bbce433
64 changed files with 1397 additions and 534 deletions
|
|
@ -4,7 +4,7 @@
|
|||
package akka.stream.impl
|
||||
|
||||
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicLong }
|
||||
|
||||
import java.{ util ⇒ ju }
|
||||
import akka.actor._
|
||||
import akka.event.Logging
|
||||
import akka.dispatch.Dispatchers
|
||||
|
|
@ -15,9 +15,11 @@ import akka.stream.impl.fusing.{ ActorGraphInterpreter, GraphModule }
|
|||
import akka.stream.impl.io.SslTlsCipherActor
|
||||
import akka.stream.io.SslTls.TlsModule
|
||||
import org.reactivestreams._
|
||||
|
||||
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
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -71,7 +73,11 @@ private[akka] case class ActorMaterializerImpl(system: ActorSystem,
|
|||
override def scheduleOnce(delay: FiniteDuration, task: Runnable) =
|
||||
system.scheduler.scheduleOnce(delay, task)(executionContext)
|
||||
|
||||
override def materialize[Mat](runnableGraph: Graph[ClosedShape, Mat]): Mat = {
|
||||
override def materialize[Mat](_runnableGraph: Graph[ClosedShape, Mat]): 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)
|
||||
|
|
@ -85,7 +91,7 @@ private[akka] case class ActorMaterializerImpl(system: ActorSystem,
|
|||
name
|
||||
}
|
||||
|
||||
override protected def materializeAtomic(atomic: Module, effectiveAttributes: Attributes): Any = {
|
||||
override protected def materializeAtomic(atomic: Module, effectiveAttributes: Attributes, matVal: ju.Map[Module, Any]): Unit = {
|
||||
|
||||
def newMaterializationContext() =
|
||||
new MaterializationContext(ActorMaterializerImpl.this, effectiveAttributes, stageName(effectiveAttributes))
|
||||
|
|
@ -93,18 +99,18 @@ private[akka] case class ActorMaterializerImpl(system: ActorSystem,
|
|||
case sink: SinkModule[_, _] ⇒
|
||||
val (sub, mat) = sink.create(newMaterializationContext())
|
||||
assignPort(sink.shape.inlet, sub.asInstanceOf[Subscriber[Any]])
|
||||
mat
|
||||
matVal.put(atomic, mat)
|
||||
case source: SourceModule[_, _] ⇒
|
||||
val (pub, mat) = source.create(newMaterializationContext())
|
||||
assignPort(source.shape.outlet, pub.asInstanceOf[Publisher[Any]])
|
||||
mat
|
||||
matVal.put(atomic, mat)
|
||||
|
||||
// FIXME: Remove this, only stream-of-stream ops need it
|
||||
case stage: StageModule ⇒
|
||||
val (processor, mat) = processorFor(stage, effectiveAttributes, effectiveSettings(effectiveAttributes))
|
||||
assignPort(stage.inPort, processor)
|
||||
assignPort(stage.outPort, processor)
|
||||
mat
|
||||
matVal.put(atomic, mat)
|
||||
|
||||
case tls: TlsModule ⇒ // TODO solve this so TlsModule doesn't need special treatment here
|
||||
val es = effectiveSettings(effectiveAttributes)
|
||||
|
|
@ -123,24 +129,35 @@ private[akka] case class ActorMaterializerImpl(system: ActorSystem,
|
|||
assignPort(tls.plainIn, FanIn.SubInput[Any](impl, SslTlsCipherActor.UserIn))
|
||||
assignPort(tls.cipherIn, FanIn.SubInput[Any](impl, SslTlsCipherActor.TransportIn))
|
||||
|
||||
matVal.put(atomic, ())
|
||||
|
||||
case graph: GraphModule ⇒
|
||||
val calculatedSettings = effectiveSettings(effectiveAttributes)
|
||||
val (inHandlers, outHandlers, logics, mat) = graph.assembly.materialize(effectiveAttributes)
|
||||
matGraph(graph, effectiveAttributes, matVal)
|
||||
|
||||
val props = ActorGraphInterpreter.props(
|
||||
graph.assembly, inHandlers, outHandlers, logics, graph.shape, calculatedSettings, ActorMaterializerImpl.this)
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
||||
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)
|
||||
}
|
||||
mat
|
||||
private def matGraph(graph: GraphModule, effectiveAttributes: Attributes, matVal: ju.Map[Module, Any]): Unit = {
|
||||
val calculatedSettings = effectiveSettings(effectiveAttributes)
|
||||
val (inHandlers, outHandlers, logics) = graph.assembly.materialize(effectiveAttributes, graph.matValIDs, matVal, registerSrc)
|
||||
|
||||
val props = ActorGraphInterpreter.props(
|
||||
graph.assembly, inHandlers, outHandlers, logics, graph.shape, calculatedSettings, ActorMaterializerImpl.this)
|
||||
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue