simplify materialized value computation tree, fixes #20015

- also fixes materialized value sources for graphs that import zero or
  one graphs, with and without Fusing
This commit is contained in:
Roland Kuhn 2016-03-11 17:08:30 +01:00
parent b52c498638
commit b255a19374
31 changed files with 582 additions and 279 deletions

View file

@ -19,6 +19,7 @@ import scala.collection.JavaConverters._
import akka.stream.impl.fusing.GraphStageModule
import akka.stream.impl.fusing.GraphStages.MaterializedValueSource
import akka.stream.impl.fusing.GraphModule
import akka.event.Logging
/**
* INTERNAL API
@ -104,9 +105,21 @@ object StreamLayout {
if (problems.nonEmpty && !doPrint) throw new IllegalStateException(s"module inconsistent, found ${problems.size} problems")
}
// TODO: Materialization order
// TODO: Special case linear composites
// TODO: Cycles
object IgnorableMatValComp {
def apply(comp: MaterializedValueNode): Boolean =
comp match {
case Atomic(module) IgnorableMatValComp(module)
case _: Combine | _: Transform false
case Ignore true
}
def apply(module: Module): Boolean =
module match {
case _: AtomicModule | EmptyModule true
case CopiedModule(_, _, module) IgnorableMatValComp(module)
case CompositeModule(_, _, _, _, comp, _) IgnorableMatValComp(comp)
case FusedModule(_, _, _, _, comp, _, _) IgnorableMatValComp(comp)
}
}
sealed trait MaterializedValueNode {
/*
@ -121,14 +134,14 @@ object StreamLayout {
override def toString: String = s"Combine($dep1,$dep2)"
}
case class Atomic(module: Module) extends MaterializedValueNode {
override def toString: String = s"Atomic(${module.attributes.nameOrDefault(module.getClass.getName)}[${module.hashCode}])"
override def toString: String = f"Atomic(${module.attributes.nameOrDefault(module.getClass.getName)}[${System.identityHashCode(module)}%08x])"
}
case class Transform(f: Any Any, dep: MaterializedValueNode) extends MaterializedValueNode {
override def toString: String = s"Transform($dep)"
}
case object Ignore extends MaterializedValueNode
trait Module {
sealed trait Module {
def shape: Shape
/**
@ -241,19 +254,30 @@ object StreamLayout {
require(that ne this, "A module cannot be added to itself. You should pass a separate instance to compose().")
require(!subModules(that), "An existing submodule cannot be added again. All contained modules must be unique.")
val modules1 = if (this.isSealed) Set(this) else this.subModules
val modules2 = if (that.isSealed) Set(that) else that.subModules
val modulesLeft = if (this.isSealed) Set(this) else this.subModules
val modulesRight = if (that.isSealed) Set(that) else that.subModules
val matComputation1 = if (this.isSealed) Atomic(this) else this.materializedValueComputation
val matComputation2 = if (that.isSealed) Atomic(that) else that.materializedValueComputation
val matCompLeft = if (this.isSealed) Atomic(this) else this.materializedValueComputation
val matCompRight = if (that.isSealed) Atomic(that) else that.materializedValueComputation
val mat =
{
val comp =
if (f == scaladsl.Keep.left) {
if (IgnorableMatValComp(matCompRight)) matCompLeft else null
} else if (f == scaladsl.Keep.right) {
if (IgnorableMatValComp(matCompLeft)) matCompRight else null
} else null
if (comp == null) Combine(f.asInstanceOf[(Any, Any) Any], matCompLeft, matCompRight)
else comp
}
CompositeModule(
modules1 ++ modules2,
modulesLeft ++ modulesRight,
AmorphousShape(shape.inlets ++ that.shape.inlets, shape.outlets ++ that.shape.outlets),
downstreams ++ that.downstreams,
upstreams ++ that.upstreams,
// would like to optimize away this allocation for Keep.{left,right} but that breaks side-effecting transformations
Combine(f.asInstanceOf[(Any, Any) Any], matComputation1, matComputation2),
mat,
Attributes.none)
}
@ -314,7 +338,7 @@ object StreamLayout {
final override def equals(obj: scala.Any): Boolean = super.equals(obj)
}
object EmptyModule extends Module {
case object EmptyModule extends Module {
override def shape = ClosedShape
override def replaceShape(s: Shape) =
if (s != shape) throw new UnsupportedOperationException("cannot replace the shape of the EmptyModule")
@ -357,7 +381,7 @@ object StreamLayout {
override def isCopied: Boolean = true
override def toString: String = s"$copyOf (copy)"
override def toString: String = f"[${System.identityHashCode(this)}%08x] copy of $copyOf"
}
final case class CompositeModule(
@ -379,13 +403,13 @@ object StreamLayout {
override def withAttributes(attributes: Attributes): Module = copy(attributes = attributes)
override def toString =
s"""
f"""CompositeModule [${System.identityHashCode(this)}%08x]
| Name: ${this.attributes.nameOrDefault("unnamed")}
| Modules:
| ${subModules.iterator.map(m m.attributes.nameLifted.getOrElse(m.toString.replaceAll("\n", "\n "))).mkString("\n ")}
| ${subModules.iterator.map(m s"(${m.attributes.nameLifted.getOrElse("unnamed")}) ${m.toString.replaceAll("\n", "\n ")}").mkString("\n ")}
| Downstreams: ${downstreams.iterator.map { case (in, out) s"\n $in -> $out" }.mkString("")}
| Upstreams: ${upstreams.iterator.map { case (out, in) s"\n $out -> $in" }.mkString("")}
|""".stripMargin
| MatValue: $materializedValueComputation""".stripMargin
}
object CompositeModule {
@ -414,13 +438,24 @@ object StreamLayout {
override def withAttributes(attributes: Attributes): FusedModule = copy(attributes = attributes)
override def toString =
s"""
| Name: ${this.attributes.nameOrDefault("unnamed")}
| Modules:
| ${subModules.iterator.map(m m.attributes.nameLifted.getOrElse(m.toString.replaceAll("\n", "\n "))).mkString("\n ")}
| Downstreams: ${downstreams.iterator.map { case (in, out) s"\n $in -> $out" }.mkString("")}
| Upstreams: ${upstreams.iterator.map { case (out, in) s"\n $out -> $in" }.mkString("")}
|""".stripMargin
f"""FusedModule [${System.identityHashCode(this)}%08x]
| Name: ${this.attributes.nameOrDefault("unnamed")}
| Modules:
| ${subModules.iterator.map(m m.attributes.nameLifted.getOrElse(m.toString.replaceAll("\n", "\n "))).mkString("\n ")}
| Downstreams: ${downstreams.iterator.map { case (in, out) s"\n $in -> $out" }.mkString("")}
| Upstreams: ${upstreams.iterator.map { case (out, in) s"\n $out -> $in" }.mkString("")}
| MatValue: $materializedValueComputation""".stripMargin
}
/**
* This is the only extension point for the sealed type hierarchy: composition
* (i.e. the module tree) is managed strictly within this file, only leaf nodes
* may be declared elsewhere.
*/
abstract class AtomicModule extends Module {
final override def subModules: Set[Module] = Set.empty
final override def downstreams: Map[OutPort, InPort] = super.downstreams
final override def upstreams: Map[InPort, OutPort] = super.upstreams
}
}
@ -718,6 +753,8 @@ private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Mo
new ju.HashMap[InPort, AnyRef] :: Nil
private var publishersStack: List[ju.Map[OutPort, Publisher[Any]]] =
new ju.HashMap[OutPort, Publisher[Any]] :: Nil
private var matValSrcStack: List[ju.Map[MaterializedValueNode, List[MaterializedValueSource[Any]]]] =
new ju.HashMap[MaterializedValueNode, List[MaterializedValueSource[Any]]] :: Nil
/*
* Please note that this stack keeps track of the scoped modules wrapped in CopiedModule but not the CopiedModule
@ -732,13 +769,16 @@ private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Mo
private def subscribers: ju.Map[InPort, AnyRef] = subscribersStack.head
private def publishers: ju.Map[OutPort, Publisher[Any]] = publishersStack.head
private def currentLayout: Module = moduleStack.head
private def matValSrc: ju.Map[MaterializedValueNode, List[MaterializedValueSource[Any]]] = matValSrcStack.head
// Enters a copied module and establishes a scope that prevents internals to leak out and interfere with copies
// of the same module.
// We don't store the enclosing CopiedModule itself as state since we don't use it anywhere else than exit and enter
private def enterScope(enclosing: CopiedModule): Unit = {
if (MaterializerSession.Debug) println(f"entering scope [${System.identityHashCode(enclosing)}%08x]")
subscribersStack ::= new ju.HashMap
publishersStack ::= new ju.HashMap
matValSrcStack ::= new ju.HashMap
moduleStack ::= enclosing.copyOf
}
@ -747,12 +787,16 @@ private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Mo
// leading to port identity collisions)
// We don't store the enclosing CopiedModule itself as state since we don't use it anywhere else than exit and enter
private def exitScope(enclosing: CopiedModule): Unit = {
if (MaterializerSession.Debug) println(f"exiting scope [${System.identityHashCode(enclosing)}%08x]")
val scopeSubscribers = subscribers
val scopePublishers = publishers
subscribersStack = subscribersStack.tail
publishersStack = publishersStack.tail
matValSrcStack = matValSrcStack.tail
moduleStack = moduleStack.tail
if (MaterializerSession.Debug) println(s" subscribers = $scopeSubscribers\n publishers = $scopePublishers")
// When we exit the scope of a copied module, pick up the Subscribers/Publishers belonging to exposed ports of
// the original module and assign them to the copy ports in the outer scope that we will return to
enclosing.copyOf.shape.inlets.iterator.zip(enclosing.shape.inlets.iterator).foreach {
@ -765,6 +809,7 @@ private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Mo
}
final def materialize(): Any = {
if (MaterializerSession.Debug) println(s"beginning materialization of $topLevel")
require(topLevel ne EmptyModule, "An empty module cannot be materialized (EmptyModule was given)")
require(
topLevel.isRunnable,
@ -789,7 +834,6 @@ private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Mo
protected def mergeAttributes(parent: Attributes, current: Attributes): Attributes =
parent and current
private val matValSrc: ju.Map[MaterializedValueNode, List[MaterializedValueSource[Any]]] = new ju.HashMap
def registerSrc(ms: MaterializedValueSource[Any]): Unit = {
if (MaterializerSession.Debug) println(s"registering source $ms")
matValSrc.get(ms.computation) match {
@ -801,53 +845,60 @@ private[stream] abstract class MaterializerSession(val topLevel: StreamLayout.Mo
protected def materializeModule(module: Module, effectiveAttributes: Attributes): Any = {
val materializedValues: ju.Map[Module, Any] = new ju.HashMap
if (MaterializerSession.Debug) println(f"entering module [${System.identityHashCode(module)}%08x] (${Logging.simpleName(module)})")
for (submodule module.subModules) {
val subEffectiveAttributes = mergeAttributes(effectiveAttributes, submodule.attributes)
submodule match {
case GraphStageModule(shape, attributes, mv: MaterializedValueSource[_])
val copy = mv.copySrc.asInstanceOf[MaterializedValueSource[Any]]
registerSrc(copy)
materializeAtomic(copy.module, subEffectiveAttributes, materializedValues)
case atomic if atomic.isAtomic
case atomic: AtomicModule
materializeAtomic(atomic, subEffectiveAttributes, materializedValues)
case copied: CopiedModule
enterScope(copied)
materializedValues.put(copied, materializeModule(copied, subEffectiveAttributes))
exitScope(copied)
case composite
case composite @ (_: CompositeModule | _: FusedModule)
materializedValues.put(composite, materializeComposite(composite, subEffectiveAttributes))
case EmptyModule => // nothing to do or say
}
}
if (MaterializerSession.Debug) {
println("RESOLVING")
println(s" module = $module")
println(s" computation = ${module.materializedValueComputation}")
println(f"resolving module [${System.identityHashCode(module)}%08x] computation ${module.materializedValueComputation}")
println(s" matValSrc = $matValSrc")
println(s" matVals = $materializedValues")
println(s" matVals =\n ${materializedValues.asScala.map(p "%08x".format(System.identityHashCode(p._1)) -> p._2).mkString("\n ")}")
}
resolveMaterialized(module.materializedValueComputation, materializedValues, " ")
val ret = resolveMaterialized(module.materializedValueComputation, materializedValues, 2)
while (!matValSrc.isEmpty) {
val node = matValSrc.keySet.iterator.next()
if (MaterializerSession.Debug) println(s" delayed computation of $node")
resolveMaterialized(node, materializedValues, 4)
}
if (MaterializerSession.Debug) println(f"exiting module [${System.identityHashCode(module)}%08x]")
ret
}
protected def materializeComposite(composite: Module, effectiveAttributes: Attributes): Any = {
materializeModule(composite, effectiveAttributes)
}
protected def materializeAtomic(atomic: Module, effectiveAttributes: Attributes, matVal: ju.Map[Module, Any]): Unit
protected def materializeAtomic(atomic: AtomicModule, effectiveAttributes: Attributes, matVal: ju.Map[Module, Any]): Unit
private def resolveMaterialized(matNode: MaterializedValueNode, matVal: ju.Map[Module, Any], indent: String): Any = {
if (MaterializerSession.Debug) println(indent + matNode)
private def resolveMaterialized(matNode: MaterializedValueNode, matVal: ju.Map[Module, Any], spaces: Int): Any = {
if (MaterializerSession.Debug) println(" " * spaces + matNode)
val ret = matNode match {
case Atomic(m) matVal.get(m)
case Combine(f, d1, d2) f(resolveMaterialized(d1, matVal, indent + " "), resolveMaterialized(d2, matVal, indent + " "))
case Transform(f, d) f(resolveMaterialized(d, matVal, indent + " "))
case Combine(f, d1, d2) f(resolveMaterialized(d1, matVal, spaces + 2), resolveMaterialized(d2, matVal, spaces + 2))
case Transform(f, d) f(resolveMaterialized(d, matVal, spaces + 2))
case Ignore NotUsed
}
if (MaterializerSession.Debug) println(indent + s"result = $ret")
if (MaterializerSession.Debug) println(" " * spaces + s"result = $ret")
matValSrc.remove(matNode) match {
case null // nothing to do
case srcs
if (MaterializerSession.Debug) println(indent + s"triggering sources $srcs")
if (MaterializerSession.Debug) println(" " * spaces + s"triggering sources $srcs")
srcs.foreach(_.setValue(ret))
}
ret