New materializer and layout
This commit is contained in:
parent
1989ef481d
commit
ba63c7af8d
60 changed files with 4420 additions and 3181 deletions
|
|
@ -3,18 +3,12 @@
|
|||
*/
|
||||
package akka.stream.impl.fusing
|
||||
|
||||
import java.util.Arrays
|
||||
import akka.actor.ActorRef
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.stream.stage._
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import akka.stream._
|
||||
import akka.stream.impl.StreamLayout._
|
||||
import java.util.concurrent.ThreadLocalRandom
|
||||
import scala.util.control.NonFatal
|
||||
import java.{ util ⇒ ju }
|
||||
import akka.stream.impl.fusing.GraphStages.MaterializedValueSource
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -28,7 +22,6 @@ object GraphInterpreter {
|
|||
final val Debug = false
|
||||
|
||||
final val NoEvent = null
|
||||
final val Boundary = -1
|
||||
|
||||
final val InReady = 1
|
||||
final val Pulling = 2
|
||||
|
|
@ -81,11 +74,11 @@ object GraphInterpreter {
|
|||
* @param outHandler The handler that contains the callback for output events.
|
||||
*/
|
||||
final class Connection(
|
||||
val id: Int,
|
||||
val inOwnerId: Int,
|
||||
val inOwner: GraphStageLogic,
|
||||
val outOwnerId: Int,
|
||||
val outOwner: GraphStageLogic,
|
||||
var id: Int,
|
||||
var inOwnerId: Int,
|
||||
var inOwner: GraphStageLogic,
|
||||
var outOwnerId: Int,
|
||||
var outOwner: GraphStageLogic,
|
||||
var inHandler: InHandler,
|
||||
var outHandler: OutHandler
|
||||
) {
|
||||
|
|
@ -95,195 +88,6 @@ object GraphInterpreter {
|
|||
override def toString = s"Connection($id, $portState, $slot, $inHandler, $outHandler)"
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* A GraphAssembly represents a small stream processing graph to be executed by the interpreter. Instances of this
|
||||
* class **must not** be mutated after construction.
|
||||
*
|
||||
* The array ``originalAttributes`` may contain the attribute information of the original atomic module, otherwise
|
||||
* it must contain a none (otherwise the enclosing module could not overwrite attributes defined in this array).
|
||||
*
|
||||
* The arrays [[ins]] and [[outs]] correspond to the notion of a *connection* in the [[GraphInterpreter]]. Each slot
|
||||
* *i* contains the input and output port corresponding to connection *i*. Slots where the graph is not closed (i.e.
|
||||
* ports are exposed to the external world) are marked with *null* values. For example if an input port *p* is
|
||||
* exposed, then outs(p) will contain a *null*.
|
||||
*
|
||||
* The arrays [[inOwners]] and [[outOwners]] are lookup tables from a connection id (the index of the slot)
|
||||
* to a slot in the [[stages]] array, indicating which stage is the owner of the given input or output port.
|
||||
* Slots which would correspond to non-existent stages (where the corresponding port is null since it represents
|
||||
* the currently unknown external context) contain the value [[GraphInterpreter#Boundary]].
|
||||
*
|
||||
* The current assumption by the infrastructure is that the layout of these arrays looks like this:
|
||||
*
|
||||
* +---------------------------------------+-----------------+
|
||||
* inOwners: | index to stages array | Boundary (-1) |
|
||||
* +----------------+----------------------+-----------------+
|
||||
* ins: | exposed inputs | internal connections | nulls |
|
||||
* +----------------+----------------------+-----------------+
|
||||
* outs: | nulls | internal connections | exposed outputs |
|
||||
* +----------------+----------------------+-----------------+
|
||||
* outOwners: | Boundary (-1) | index to stages array |
|
||||
* +----------------+----------------------------------------+
|
||||
*
|
||||
* In addition, it is also assumed by the infrastructure that the order of exposed inputs and outputs in the
|
||||
* corresponding segments of these arrays matches the exact same order of the ports in the [[Shape]].
|
||||
*
|
||||
*/
|
||||
final class GraphAssembly(
|
||||
val stages: Array[GraphStageWithMaterializedValue[Shape, Any]],
|
||||
val originalAttributes: Array[Attributes],
|
||||
val ins: Array[Inlet[_]],
|
||||
val inOwners: Array[Int],
|
||||
val outs: Array[Outlet[_]],
|
||||
val outOwners: Array[Int]) {
|
||||
require(ins.length == inOwners.length && inOwners.length == outs.length && outs.length == outOwners.length)
|
||||
|
||||
def connectionCount: Int = ins.length
|
||||
|
||||
/**
|
||||
* Takes an interpreter and returns three arrays required by the interpreter containing the input, output port
|
||||
* handlers and the stage logic instances.
|
||||
*
|
||||
* Returns a tuple of
|
||||
* - lookup table for Connections
|
||||
* - array of the logics
|
||||
*/
|
||||
def materialize(
|
||||
inheritedAttributes: Attributes,
|
||||
copiedModules: Array[Module],
|
||||
matVal: ju.Map[Module, Any],
|
||||
register: MaterializedValueSource[Any] ⇒ Unit): (Array[Connection], Array[GraphStageLogic]) = {
|
||||
val logics = Array.ofDim[GraphStageLogic](stages.length)
|
||||
|
||||
var i = 0
|
||||
while (i < stages.length) {
|
||||
// Port initialization loops, these must come first
|
||||
val shape = stages(i).shape
|
||||
|
||||
var idx = 0
|
||||
val inletItr = shape.inlets.iterator
|
||||
while (inletItr.hasNext) {
|
||||
val inlet = inletItr.next()
|
||||
require(inlet.id == -1 || inlet.id == idx, s"Inlet $inlet was shared among multiple stages. This is illegal.")
|
||||
inlet.id = idx
|
||||
idx += 1
|
||||
}
|
||||
|
||||
idx = 0
|
||||
val outletItr = shape.outlets.iterator
|
||||
while (outletItr.hasNext) {
|
||||
val outlet = outletItr.next()
|
||||
require(outlet.id == -1 || outlet.id == idx, s"Outlet $outlet was shared among multiple stages. This is illegal.")
|
||||
outlet.id = idx
|
||||
idx += 1
|
||||
}
|
||||
|
||||
val stage = stages(i) match {
|
||||
case mv: MaterializedValueSource[_] ⇒
|
||||
val copy = mv.copySrc.asInstanceOf[MaterializedValueSource[Any]]
|
||||
register(copy)
|
||||
copy
|
||||
case x ⇒ x
|
||||
}
|
||||
|
||||
val logicAndMat = stage.createLogicAndMaterializedValue(inheritedAttributes and originalAttributes(i))
|
||||
matVal.put(copiedModules(i), logicAndMat._2)
|
||||
|
||||
logics(i) = logicAndMat._1
|
||||
i += 1
|
||||
}
|
||||
|
||||
val connections = Array.ofDim[Connection](connectionCount)
|
||||
|
||||
i = 0
|
||||
while (i < connectionCount) {
|
||||
connections(i) = new Connection(
|
||||
id = i,
|
||||
inOwner = if (inOwners(i) == Boundary) null else logics(inOwners(i)),
|
||||
inOwnerId = inOwners(i),
|
||||
outOwner = if (outOwners(i) == Boundary) null else logics(outOwners(i)),
|
||||
outOwnerId = outOwners(i),
|
||||
inHandler = null,
|
||||
outHandler = null
|
||||
)
|
||||
|
||||
if (ins(i) ne null) {
|
||||
val logic = logics(inOwners(i))
|
||||
logic.handlers(ins(i).id) match {
|
||||
case null ⇒ throw new IllegalStateException(s"no handler defined in stage $logic for port ${ins(i)}")
|
||||
case h: InHandler ⇒
|
||||
connections(i).inHandler = h
|
||||
}
|
||||
logics(inOwners(i)).portToConn(ins(i).id) = connections(i)
|
||||
}
|
||||
if (outs(i) ne null) {
|
||||
val logic = logics(outOwners(i))
|
||||
val inCount = logic.inCount
|
||||
logic.handlers(outs(i).id + inCount) match {
|
||||
case null ⇒ throw new IllegalStateException(s"no handler defined in stage $logic for port ${outs(i)}")
|
||||
case h: OutHandler ⇒
|
||||
connections(i).outHandler = h
|
||||
}
|
||||
logic.portToConn(outs(i).id + inCount) = connections(i)
|
||||
}
|
||||
i += 1
|
||||
}
|
||||
|
||||
(connections, logics)
|
||||
}
|
||||
|
||||
override def toString: String = {
|
||||
val stageList = stages.iterator.zip(originalAttributes.iterator).map {
|
||||
case (stage, attr) ⇒ s"${stage.module} [${attr.attributeList.mkString(", ")}]"
|
||||
}
|
||||
"GraphAssembly\n " +
|
||||
stageList.mkString("[ ", "\n ", "\n ]") + "\n " +
|
||||
ins.mkString("[", ",", "]") + "\n " +
|
||||
inOwners.mkString("[", ",", "]") + "\n " +
|
||||
outs.mkString("[", ",", "]") + "\n " +
|
||||
outOwners.mkString("[", ",", "]")
|
||||
}
|
||||
}
|
||||
|
||||
object GraphAssembly {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
final def apply(
|
||||
inlets: immutable.Seq[Inlet[_]],
|
||||
outlets: immutable.Seq[Outlet[_]],
|
||||
stages: GraphStageWithMaterializedValue[Shape, _]*): GraphAssembly = {
|
||||
// add the contents of an iterator to an array starting at idx
|
||||
@tailrec def add[T](i: Iterator[T], a: Array[T], idx: Int): Array[T] =
|
||||
if (i.hasNext) {
|
||||
a(idx) = i.next()
|
||||
add(i, a, idx + 1)
|
||||
} else a
|
||||
|
||||
// fill array slots with Boundary
|
||||
def markBoundary(owners: Array[Int], from: Int, to: Int): Array[Int] = {
|
||||
Arrays.fill(owners, from, to, Boundary)
|
||||
owners
|
||||
}
|
||||
|
||||
val inletsSize = inlets.size
|
||||
val outletsSize = outlets.size
|
||||
val connectionCount = inletsSize + outletsSize
|
||||
require(connectionCount > 0, s"sum of inlets ({$inletsSize}) & outlets ({$outletsSize}) must be > 0")
|
||||
|
||||
val assembly = new GraphAssembly(
|
||||
stages.toArray,
|
||||
GraphInterpreter.singleNoAttribute,
|
||||
add(inlets.iterator, Array.ofDim(connectionCount), 0),
|
||||
markBoundary(Array.ofDim(connectionCount), inletsSize, connectionCount),
|
||||
add(outlets.iterator, Array.ofDim(connectionCount), inletsSize),
|
||||
markBoundary(Array.ofDim(connectionCount), 0, inletsSize))
|
||||
|
||||
assembly
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
|
|
@ -389,14 +193,14 @@ object GraphInterpreter {
|
|||
* edge of a balance is pulled, dissolving the original cycle).
|
||||
*/
|
||||
final class GraphInterpreter(
|
||||
private val assembly: GraphInterpreter.GraphAssembly,
|
||||
val materializer: Materializer,
|
||||
val log: LoggingAdapter,
|
||||
val logics: Array[GraphStageLogic], // Array of stage logics
|
||||
val connections: Array[GraphInterpreter.Connection],
|
||||
val onAsyncInput: (GraphStageLogic, Any, (Any) ⇒ Unit) ⇒ Unit,
|
||||
val fuzzingMode: Boolean,
|
||||
val context: ActorRef) {
|
||||
val materializer: Materializer,
|
||||
val log: LoggingAdapter,
|
||||
val logics: Array[GraphStageLogic], // Array of stage logics
|
||||
val connections: Array[GraphInterpreter.Connection],
|
||||
val onAsyncInput: (GraphStageLogic, Any, (Any) ⇒ Unit) ⇒ Unit,
|
||||
val fuzzingMode: Boolean,
|
||||
val context: ActorRef) {
|
||||
|
||||
import GraphInterpreter._
|
||||
|
||||
private[this] val ChaseLimit = if (fuzzingMode) 0 else 16
|
||||
|
|
@ -408,12 +212,11 @@ final class GraphInterpreter(
|
|||
|
||||
// The number of currently running stages. Once this counter reaches zero, the interpreter is considered to be
|
||||
// completed
|
||||
private[this] var runningStages = assembly.stages.length
|
||||
private[this] var runningStages = logics.length
|
||||
|
||||
// Counts how many active connections a stage has. Once it reaches zero, the stage is automatically stopped.
|
||||
private[this] val shutdownCounter = Array.tabulate(assembly.stages.length) { i ⇒
|
||||
val shape = assembly.stages(i).shape
|
||||
shape.inlets.size + shape.outlets.size
|
||||
private[this] val shutdownCounter = Array.tabulate(logics.length) { i ⇒
|
||||
logics(i).handlers.length
|
||||
}
|
||||
|
||||
private[this] var _subFusingMaterializer: Materializer = _
|
||||
|
|
@ -421,7 +224,7 @@ final class GraphInterpreter(
|
|||
|
||||
// An event queue implemented as a circular buffer
|
||||
// FIXME: This calculates the maximum size ever needed, but most assemblies can run on a smaller queue
|
||||
private[this] val eventQueue = Array.ofDim[Connection](1 << (32 - Integer.numberOfLeadingZeros(assembly.connectionCount - 1)))
|
||||
private[this] val eventQueue = Array.ofDim[Connection](1 << (32 - Integer.numberOfLeadingZeros(connections.length - 1)))
|
||||
private[this] val mask = eventQueue.length - 1
|
||||
private[this] var queueHead: Int = 0
|
||||
private[this] var queueTail: Int = 0
|
||||
|
|
@ -449,32 +252,6 @@ final class GraphInterpreter(
|
|||
*/
|
||||
private[stream] def nonNull: GraphInterpreter = this
|
||||
|
||||
/**
|
||||
* Assign the boundary logic to a given connection. This will serve as the interface to the external world
|
||||
* (outside the interpreter) to process and inject events.
|
||||
*/
|
||||
def attachUpstreamBoundary(connection: Connection, logic: UpstreamBoundaryStageLogic[_]): Unit = {
|
||||
logic.portToConn(logic.out.id + logic.inCount) = connection
|
||||
logic.interpreter = this
|
||||
connection.outHandler = logic.handlers(0).asInstanceOf[OutHandler]
|
||||
}
|
||||
|
||||
def attachUpstreamBoundary(connection: Int, logic: UpstreamBoundaryStageLogic[_]): Unit =
|
||||
attachUpstreamBoundary(connections(connection), logic)
|
||||
|
||||
/**
|
||||
* Assign the boundary logic to a given connection. This will serve as the interface to the external world
|
||||
* (outside the interpreter) to process and inject events.
|
||||
*/
|
||||
def attachDownstreamBoundary(connection: Connection, logic: DownstreamBoundaryStageLogic[_]): Unit = {
|
||||
logic.portToConn(logic.in.id) = connection
|
||||
logic.interpreter = this
|
||||
connection.inHandler = logic.handlers(0).asInstanceOf[InHandler]
|
||||
}
|
||||
|
||||
def attachDownstreamBoundary(connection: Int, logic: DownstreamBoundaryStageLogic[_]): Unit =
|
||||
attachDownstreamBoundary(connections(connection), logic)
|
||||
|
||||
/**
|
||||
* Dynamic handler changes are communicated from a GraphStageLogic by this method.
|
||||
*/
|
||||
|
|
@ -513,14 +290,14 @@ final class GraphInterpreter(
|
|||
var i = 0
|
||||
while (i < logics.length) {
|
||||
val logic = logics(i)
|
||||
logic.stageId = i
|
||||
logic.interpreter = this
|
||||
try {
|
||||
logic.beforePreStart()
|
||||
logic.preStart()
|
||||
} catch {
|
||||
case NonFatal(e) ⇒
|
||||
log.error(e, "Error during preStart in [{}]", assembly.stages(logic.stageId))
|
||||
e.printStackTrace()
|
||||
// log.error(e, "Error during preStart in [{}]", assembly.stages(logic.stageId))
|
||||
logic.failStage(e)
|
||||
}
|
||||
afterStageHasRun(logic)
|
||||
|
|
@ -541,32 +318,16 @@ final class GraphInterpreter(
|
|||
}
|
||||
|
||||
// Debug name for a connections input part
|
||||
private def inOwnerName(connection: Connection): String =
|
||||
assembly.inOwners(connection.id) match {
|
||||
case Boundary ⇒ "DownstreamBoundary"
|
||||
case owner ⇒ assembly.stages(owner).toString
|
||||
}
|
||||
private def inOwnerName(connection: Connection): String = connection.inOwner.toString
|
||||
|
||||
// Debug name for a connections output part
|
||||
private def outOwnerName(connection: Connection): String =
|
||||
assembly.outOwners(connection.id) match {
|
||||
case Boundary ⇒ "UpstreamBoundary"
|
||||
case owner ⇒ assembly.stages(owner).toString
|
||||
}
|
||||
private def outOwnerName(connection: Connection): String = connection.outOwner.toString
|
||||
|
||||
// Debug name for a connections input part
|
||||
private def inLogicName(connection: Connection): String =
|
||||
assembly.inOwners(connection.id) match {
|
||||
case Boundary ⇒ "DownstreamBoundary"
|
||||
case owner ⇒ logics(owner).toString
|
||||
}
|
||||
private def inLogicName(connection: Connection): String = logics(connection.inOwnerId).toString
|
||||
|
||||
// Debug name for a connections output part
|
||||
private def outLogicName(connection: Connection): String =
|
||||
assembly.outOwners(connection.id) match {
|
||||
case Boundary ⇒ "UpstreamBoundary"
|
||||
case owner ⇒ logics(owner).toString
|
||||
}
|
||||
private def outLogicName(connection: Connection): String = logics(connection.outOwnerId).toString
|
||||
|
||||
private def shutdownCounters: String =
|
||||
shutdownCounter.map(x ⇒ if (x >= KeepGoingFlag) s"${x & KeepGoingMask}(KeepGoing)" else x.toString).mkString(",")
|
||||
|
|
@ -590,9 +351,10 @@ final class GraphInterpreter(
|
|||
def reportStageError(e: Throwable): Unit = {
|
||||
if (activeStage == null) throw e
|
||||
else {
|
||||
val stage = assembly.stages(activeStage.stageId)
|
||||
// TODO: Get error reporting back
|
||||
//val stage = stages(activeStage.stageId)
|
||||
|
||||
log.error(e, "Error in stage [{}]: {}", stage, e.getMessage)
|
||||
log.error(e, "Error in stage [{}]: {}", activeStage, e.getMessage)
|
||||
activeStage.failStage(e)
|
||||
|
||||
// Abort chasing
|
||||
|
|
@ -786,10 +548,8 @@ final class GraphInterpreter(
|
|||
// Register that a connection in which the given stage participated has been completed and therefore the stage
|
||||
// itself might stop, too.
|
||||
private def completeConnection(stageId: Int): Unit = {
|
||||
if (stageId != Boundary) {
|
||||
val activeConnections = shutdownCounter(stageId)
|
||||
if (activeConnections > 0) shutdownCounter(stageId) = activeConnections - 1
|
||||
}
|
||||
val activeConnections = shutdownCounter(stageId)
|
||||
if (activeConnections > 0) shutdownCounter(stageId) = activeConnections - 1
|
||||
}
|
||||
|
||||
private[stream] def setKeepGoing(logic: GraphStageLogic, enabled: Boolean): Unit =
|
||||
|
|
@ -802,7 +562,7 @@ final class GraphInterpreter(
|
|||
logic.afterPostStop()
|
||||
} catch {
|
||||
case NonFatal(e) ⇒
|
||||
log.error(e, s"Error during postStop in [{}]: {}", assembly.stages(logic.stageId), e.getMessage)
|
||||
//log.error(e, s"Error during postStop in [{}]: {}", assembly.stages(logic.stageId), e.getMessage)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -877,39 +637,39 @@ final class GraphInterpreter(
|
|||
*/
|
||||
def dumpWaits(): Unit = println(toString)
|
||||
|
||||
override def toString: String = {
|
||||
val builder = new StringBuilder("digraph waits {\n")
|
||||
|
||||
for (i ← assembly.stages.indices)
|
||||
builder.append(s"""N$i [label="${assembly.stages(i)}"]""" + "\n")
|
||||
|
||||
def nameIn(port: Int): String = {
|
||||
val owner = assembly.inOwners(port)
|
||||
if (owner == Boundary) "Out" + port
|
||||
else "N" + owner
|
||||
}
|
||||
|
||||
def nameOut(port: Int): String = {
|
||||
val owner = assembly.outOwners(port)
|
||||
if (owner == Boundary) "In" + port
|
||||
else "N" + owner
|
||||
}
|
||||
|
||||
for (i ← connections.indices) {
|
||||
connections(i).portState match {
|
||||
case InReady ⇒
|
||||
builder.append(s""" ${nameIn(i)} -> ${nameOut(i)} [label=shouldPull; color=blue]""")
|
||||
case OutReady ⇒
|
||||
builder.append(s""" ${nameOut(i)} -> ${nameIn(i)} [label=shouldPush; color=red];""")
|
||||
case x if (x | InClosed | OutClosed) == (InClosed | OutClosed) ⇒
|
||||
builder.append(s""" ${nameIn(i)} -> ${nameOut(i)} [style=dotted; label=closed dir=both];""")
|
||||
case _ ⇒
|
||||
}
|
||||
builder.append("\n")
|
||||
}
|
||||
|
||||
builder.append("}\n")
|
||||
builder.append(s"// $queueStatus (running=$runningStages, shutdown=${shutdownCounter.mkString(",")})")
|
||||
builder.toString()
|
||||
}
|
||||
// override def toString: String = {
|
||||
// val builder = new StringBuilder("digraph waits {\n")
|
||||
//
|
||||
// for (i ← assembly.stages.indices)
|
||||
// builder.append(s"""N$i [label="${assembly.stages(i)}"]""" + "\n")
|
||||
//
|
||||
// def nameIn(port: Int): String = {
|
||||
// val owner = assembly.inOwners(port)
|
||||
// if (owner == Boundary) "Out" + port
|
||||
// else "N" + owner
|
||||
// }
|
||||
//
|
||||
// def nameOut(port: Int): String = {
|
||||
// val owner = assembly.outOwners(port)
|
||||
// if (owner == Boundary) "In" + port
|
||||
// else "N" + owner
|
||||
// }
|
||||
//
|
||||
// for (i ← connections.indices) {
|
||||
// connections(i).portState match {
|
||||
// case InReady ⇒
|
||||
// builder.append(s""" ${nameIn(i)} -> ${nameOut(i)} [label=shouldPull; color=blue]""")
|
||||
// case OutReady ⇒
|
||||
// builder.append(s""" ${nameOut(i)} -> ${nameIn(i)} [label=shouldPush; color=red];""")
|
||||
// case x if (x | InClosed | OutClosed) == (InClosed | OutClosed) ⇒
|
||||
// builder.append(s""" ${nameIn(i)} -> ${nameOut(i)} [style=dotted; label=closed dir=both];""")
|
||||
// case _ ⇒
|
||||
// }
|
||||
// builder.append("\n")
|
||||
// }
|
||||
//
|
||||
// builder.append("}\n")
|
||||
// builder.append(s"// $queueStatus (running=$runningStages, shutdown=${shutdownCounter.mkString(",")})")
|
||||
// builder.toString()
|
||||
// }
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue