!str,htc replace and remove OneBoundedInterpreter
main work by @drewhk with contributions from @2m and @rkuhn This work uncovered many well-hidden bugs in existing Stages, in particular StatefulStage. These were hidden by the behavior of OneBoundedInterpreter that normally behaves more orderly than it guarantees in general, especially with respect to the timeliness of delivery of upstream termination signals; the bugs were then that internal state was not flushed when onComplete arrived “too early”.
This commit is contained in:
parent
20f54435f1
commit
556012b7ee
107 changed files with 2456 additions and 3061 deletions
|
|
@ -7,9 +7,9 @@ import java.util.Arrays
|
|||
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.stream.stage._
|
||||
import akka.stream.{ Materializer, Shape, Inlet, Outlet }
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import akka.stream._
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
/**
|
||||
|
|
@ -55,12 +55,17 @@ private[stream] object GraphInterpreter {
|
|||
def in: Inlet[T]
|
||||
}
|
||||
|
||||
val singleNoAttribute: Array[Attributes] = Array(Attributes.none)
|
||||
|
||||
/**
|
||||
* 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
|
||||
|
|
@ -88,6 +93,7 @@ private[stream] object GraphInterpreter {
|
|||
*
|
||||
*/
|
||||
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[_]],
|
||||
|
|
@ -106,7 +112,7 @@ private[stream] object GraphInterpreter {
|
|||
* - array of the logics
|
||||
* - materialized value
|
||||
*/
|
||||
def materialize(): (Array[InHandler], Array[OutHandler], Array[GraphStageLogic], Any) = {
|
||||
def materialize(inheritedAttributes: Attributes): (Array[InHandler], Array[OutHandler], Array[GraphStageLogic], Any) = {
|
||||
val logics = Array.ofDim[GraphStageLogic](stages.length)
|
||||
var finalMat: Any = ()
|
||||
|
||||
|
|
@ -134,7 +140,7 @@ private[stream] object GraphInterpreter {
|
|||
}
|
||||
|
||||
// FIXME: Support for materialized values in fused islands is not yet figured out!
|
||||
val logicAndMat = stages(i).createLogicAndMaterializedValue
|
||||
val logicAndMat = stages(i).createLogicAndMaterializedValue(inheritedAttributes and originalAttributes(i))
|
||||
// FIXME: Current temporary hack to support non-fused stages. If there is one stage that will be under index 0.
|
||||
if (i == 0) finalMat = logicAndMat._2
|
||||
|
||||
|
|
@ -148,20 +154,21 @@ private[stream] object GraphInterpreter {
|
|||
i = 0
|
||||
while (i < connectionCount) {
|
||||
if (ins(i) ne null) {
|
||||
val l = logics(inOwners(i))
|
||||
l.inHandlers(ins(i).id) match {
|
||||
case null ⇒ throw new IllegalStateException(s"no handler defined in stage $l for port ${ins(i)}")
|
||||
case h ⇒ inHandlers(i) = h
|
||||
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 ⇒ inHandlers(i) = h
|
||||
}
|
||||
l.inToConn(ins(i).id) = i
|
||||
logics(inOwners(i)).portToConn(ins(i).id) = i
|
||||
}
|
||||
if (outs(i) ne null) {
|
||||
val l = logics(outOwners(i))
|
||||
l.outHandlers(outs(i).id) match {
|
||||
case null ⇒ throw new IllegalStateException(s"no handler defined in stage $l for port ${outs(i)}")
|
||||
case h ⇒ outHandlers(i) = h
|
||||
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 ⇒ outHandlers(i) = h
|
||||
}
|
||||
l.outToConn(outs(i).id) = i
|
||||
logic.portToConn(outs(i).id + inCount) = i
|
||||
}
|
||||
i += 1
|
||||
}
|
||||
|
|
@ -206,6 +213,7 @@ private[stream] object GraphInterpreter {
|
|||
|
||||
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),
|
||||
|
|
@ -288,7 +296,7 @@ private[stream] object GraphInterpreter {
|
|||
*
|
||||
* Because of the FIFO construction of the queue the interpreter is fair, i.e. a pending event is always executed
|
||||
* after a bounded number of other events. This property, together with suspendability means that even infinite cycles can
|
||||
* be modeled, or even dissolved (if preempted and a "stealing" external even is injected; for example the non-cycle
|
||||
* be modeled, or even dissolved (if preempted and a "stealing" external event is injected; for example the non-cycle
|
||||
* edge of a balance is pulled, dissolving the original cycle).
|
||||
*/
|
||||
private[stream] final class GraphInterpreter(
|
||||
|
|
@ -309,7 +317,7 @@ private[stream] final class GraphInterpreter(
|
|||
// of the class for a full description.
|
||||
val portStates = Array.fill[Int](assembly.connectionCount)(InReady)
|
||||
|
||||
private[this] var activeStageId = Boundary
|
||||
private[this] var activeStage: GraphStageLogic = _
|
||||
|
||||
// The number of currently running stages. Once this counter reaches zero, the interpreter is considered to be
|
||||
// completed
|
||||
|
|
@ -323,19 +331,33 @@ private[stream] 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[Int](1 << Integer.highestOneBit(assembly.connectionCount))
|
||||
private[this] val eventQueue = Array.ofDim[Int](1 << (32 - Integer.numberOfLeadingZeros(assembly.connectionCount - 1)))
|
||||
private[this] val mask = eventQueue.length - 1
|
||||
private[this] var queueHead: Int = 0
|
||||
private[this] var queueTail: Int = 0
|
||||
|
||||
private def queueStatus: String = {
|
||||
val contents = (queueHead until queueTail).map(idx ⇒ {
|
||||
val conn = eventQueue(idx & mask)
|
||||
(conn, portStates(conn), connectionSlots(conn))
|
||||
})
|
||||
s"(${eventQueue.length}, $queueHead, $queueTail)(${contents.mkString(", ")})"
|
||||
}
|
||||
private[this] var _Name: String = _
|
||||
def Name: String =
|
||||
if (_Name eq null) {
|
||||
_Name = f"${System.identityHashCode(this)}%08X"
|
||||
_Name
|
||||
} else _Name
|
||||
|
||||
/**
|
||||
* 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: Int, logic: UpstreamBoundaryStageLogic[_]): Unit = {
|
||||
logic.outToConn(logic.out.id) = connection
|
||||
logic.portToConn(logic.out.id + logic.inCount) = connection
|
||||
logic.interpreter = this
|
||||
outHandlers(connection) = logic.outHandlers(0)
|
||||
outHandlers(connection) = logic.handlers(0).asInstanceOf[OutHandler]
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -343,16 +365,16 @@ private[stream] final class GraphInterpreter(
|
|||
* (outside the interpreter) to process and inject events.
|
||||
*/
|
||||
def attachDownstreamBoundary(connection: Int, logic: DownstreamBoundaryStageLogic[_]): Unit = {
|
||||
logic.inToConn(logic.in.id) = connection
|
||||
logic.portToConn(logic.in.id) = connection
|
||||
logic.interpreter = this
|
||||
inHandlers(connection) = logic.inHandlers(0)
|
||||
inHandlers(connection) = logic.handlers(0).asInstanceOf[InHandler]
|
||||
}
|
||||
|
||||
/**
|
||||
* Dynamic handler changes are communicated from a GraphStageLogic by this method.
|
||||
*/
|
||||
def setHandler(connection: Int, handler: InHandler): Unit = {
|
||||
if (GraphInterpreter.Debug) println(s"SETHANDLER ${inOwnerName(connection)} (in) $handler")
|
||||
if (Debug) println(s"$Name SETHANDLER ${inOwnerName(connection)} (in) $handler")
|
||||
inHandlers(connection) = handler
|
||||
}
|
||||
|
||||
|
|
@ -360,7 +382,7 @@ private[stream] final class GraphInterpreter(
|
|||
* Dynamic handler changes are communicated from a GraphStageLogic by this method.
|
||||
*/
|
||||
def setHandler(connection: Int, handler: OutHandler): Unit = {
|
||||
if (GraphInterpreter.Debug) println(s"SETHANDLER ${outOwnerName(connection)} (out) $handler")
|
||||
if (Debug) println(s"$Name SETHANDLER ${outOwnerName(connection)} (out) $handler")
|
||||
outHandlers(connection) = handler
|
||||
}
|
||||
|
||||
|
|
@ -389,6 +411,7 @@ private[stream] final class GraphInterpreter(
|
|||
} catch {
|
||||
case NonFatal(e) ⇒ logic.failStage(e)
|
||||
}
|
||||
afterStageHasRun(logic)
|
||||
i += 1
|
||||
}
|
||||
}
|
||||
|
|
@ -399,7 +422,8 @@ private[stream] final class GraphInterpreter(
|
|||
def finish(): Unit = {
|
||||
var i = 0
|
||||
while (i < logics.length) {
|
||||
if (!isStageCompleted(i)) finalizeStage(logics(i))
|
||||
val logic = logics(i)
|
||||
if (!isStageCompleted(logic)) finalizeStage(logic)
|
||||
i += 1
|
||||
}
|
||||
}
|
||||
|
|
@ -418,84 +442,97 @@ private[stream] final class GraphInterpreter(
|
|||
case owner ⇒ assembly.stages(owner).toString
|
||||
}
|
||||
|
||||
// Debug name for a connections input part
|
||||
private def inLogicName(connection: Int): String =
|
||||
assembly.inOwners(connection) match {
|
||||
case Boundary ⇒ "DownstreamBoundary"
|
||||
case owner ⇒ logics(owner).toString
|
||||
}
|
||||
|
||||
// Debug name for a connections ouput part
|
||||
private def outLogicName(connection: Int): String =
|
||||
assembly.outOwners(connection) match {
|
||||
case Boundary ⇒ "UpstreamBoundary"
|
||||
case owner ⇒ logics(owner).toString
|
||||
}
|
||||
|
||||
/**
|
||||
* Executes pending events until the given limit is met. If there were remaining events, isSuspended will return
|
||||
* true.
|
||||
*/
|
||||
def execute(eventLimit: Int): Unit = {
|
||||
if (GraphInterpreter.Debug) println("---------------- EXECUTE")
|
||||
if (Debug) println(s"$Name ---------------- EXECUTE (running=$runningStages, shutdown=${shutdownCounter.mkString(",")})")
|
||||
var eventsRemaining = eventLimit
|
||||
var connection = dequeue()
|
||||
while (eventsRemaining > 0 && connection != NoEvent) {
|
||||
try processEvent(connection)
|
||||
catch {
|
||||
case NonFatal(e) ⇒
|
||||
if (activeStageId == Boundary) throw e
|
||||
else logics(activeStageId).failStage(e)
|
||||
if (activeStage == null) throw e
|
||||
else activeStage.failStage(e)
|
||||
}
|
||||
afterStageHasRun(activeStage)
|
||||
eventsRemaining -= 1
|
||||
if (eventsRemaining > 0) connection = dequeue()
|
||||
}
|
||||
if (Debug) println(s"$Name ---------------- $queueStatus (running=$runningStages, shutdown=${shutdownCounter.mkString(",")})")
|
||||
// TODO: deadlock detection
|
||||
}
|
||||
|
||||
// Decodes and processes a single event for the given connection
|
||||
private def processEvent(connection: Int): Unit = {
|
||||
def safeLogics(id: Int) =
|
||||
if (id == Boundary) null
|
||||
else logics(id)
|
||||
|
||||
def processElement(elem: Any): Unit = {
|
||||
if (GraphInterpreter.Debug) println(s"PUSH ${outOwnerName(connection)} -> ${inOwnerName(connection)}, $elem (${inHandlers(connection)})")
|
||||
activeStageId = assembly.inOwners(connection)
|
||||
def processElement(): Unit = {
|
||||
if (Debug) println(s"$Name PUSH ${outOwnerName(connection)} -> ${inOwnerName(connection)}, ${connectionSlots(connection)} (${inHandlers(connection)}) [${inLogicName(connection)}]")
|
||||
activeStage = safeLogics(assembly.inOwners(connection))
|
||||
portStates(connection) ^= PushEndFlip
|
||||
inHandlers(connection).onPush()
|
||||
}
|
||||
|
||||
// this must be the state after returning without delivering any signals, to avoid double-finalization of some unlucky stage
|
||||
// (this can happen if a stage completes voluntarily while connection close events are still queued)
|
||||
activeStage = null
|
||||
val code = portStates(connection)
|
||||
|
||||
// Manual fast decoding, fast paths are PUSH and PULL
|
||||
// PUSH
|
||||
if ((code & (Pushing | InClosed | OutClosed)) == Pushing) {
|
||||
processElement(connectionSlots(connection))
|
||||
processElement()
|
||||
|
||||
// PULL
|
||||
} else if ((code & (Pulling | OutClosed | InClosed)) == Pulling) {
|
||||
if (GraphInterpreter.Debug) println(s"PULL ${inOwnerName(connection)} -> ${outOwnerName(connection)} (${outHandlers(connection)})")
|
||||
if (Debug) println(s"$Name PULL ${inOwnerName(connection)} -> ${outOwnerName(connection)} (${outHandlers(connection)}) [${outLogicName(connection)}]")
|
||||
portStates(connection) ^= PullEndFlip
|
||||
activeStageId = assembly.outOwners(connection)
|
||||
activeStage = safeLogics(assembly.outOwners(connection))
|
||||
outHandlers(connection).onPull()
|
||||
|
||||
// CANCEL
|
||||
} else if ((code & (OutClosed | InClosed)) == InClosed) {
|
||||
val stageId = assembly.outOwners(connection)
|
||||
if (GraphInterpreter.Debug) println(s"CANCEL ${inOwnerName(connection)} -> ${outOwnerName(connection)} (${outHandlers(connection)})")
|
||||
activeStage = safeLogics(stageId)
|
||||
if (Debug) println(s"$Name CANCEL ${inOwnerName(connection)} -> ${outOwnerName(connection)} (${outHandlers(connection)}) [${outLogicName(connection)}]")
|
||||
portStates(connection) |= OutClosed
|
||||
activeStageId = assembly.outOwners(connection)
|
||||
outHandlers(connection).onDownstreamFinish()
|
||||
completeConnection(stageId)
|
||||
outHandlers(connection).onDownstreamFinish()
|
||||
} else if ((code & (OutClosed | InClosed)) == OutClosed) {
|
||||
// COMPLETIONS
|
||||
|
||||
val stageId = assembly.inOwners(connection)
|
||||
|
||||
if ((code & Pushing) == 0) {
|
||||
// Normal completion (no push pending)
|
||||
if (GraphInterpreter.Debug) println(s"COMPLETE ${outOwnerName(connection)} -> ${inOwnerName(connection)} (${inHandlers(connection)})")
|
||||
if (Debug) println(s"$Name COMPLETE ${outOwnerName(connection)} -> ${inOwnerName(connection)} (${inHandlers(connection)}) [${inLogicName(connection)}]")
|
||||
portStates(connection) |= InClosed
|
||||
activeStageId = assembly.inOwners(connection)
|
||||
val stageId = assembly.inOwners(connection)
|
||||
activeStage = safeLogics(stageId)
|
||||
completeConnection(stageId)
|
||||
if ((portStates(connection) & InFailed) == 0) inHandlers(connection).onUpstreamFinish()
|
||||
else inHandlers(connection).onUpstreamFailure(connectionSlots(connection).asInstanceOf[Failed].ex)
|
||||
completeConnection(stageId)
|
||||
} else {
|
||||
// Push is pending, first process push, then re-enqueue closing event
|
||||
// Non-failure case
|
||||
val code = portStates(connection) & (InClosed | InFailed)
|
||||
if (code == 0) {
|
||||
processElement(connectionSlots(connection))
|
||||
enqueue(connection)
|
||||
} else if (code == InFailed) {
|
||||
// Failure case
|
||||
processElement(connectionSlots(connection).asInstanceOf[Failed].previousElem)
|
||||
enqueue(connection)
|
||||
}
|
||||
processElement()
|
||||
enqueue(connection)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -513,26 +550,26 @@ private[stream] final class GraphInterpreter(
|
|||
}
|
||||
|
||||
private def enqueue(connection: Int): Unit = {
|
||||
if (Debug) if (queueTail - queueHead > mask) new Exception(s"$Name internal queue full ($queueStatus) + $connection").printStackTrace()
|
||||
eventQueue(queueTail & mask) = connection
|
||||
queueTail += 1
|
||||
}
|
||||
|
||||
def afterStageHasRun(logic: GraphStageLogic): Unit =
|
||||
if (isStageCompleted(logic)) {
|
||||
runningStages -= 1
|
||||
finalizeStage(logic)
|
||||
}
|
||||
|
||||
// Returns true if the given stage is alredy completed
|
||||
def isStageCompleted(stageId: Int): Boolean = stageId != Boundary && shutdownCounter(stageId) == 0
|
||||
def isStageCompleted(stage: GraphStageLogic): Boolean = stage != null && shutdownCounter(stage.stageId) == 0
|
||||
|
||||
// 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
|
||||
// This was the last active connection keeping this stage alive
|
||||
if (activeConnections == 1) {
|
||||
runningStages -= 1
|
||||
finalizeStage(logics(stageId))
|
||||
}
|
||||
}
|
||||
if (activeConnections > 0) shutdownCounter(stageId) = activeConnections - 1
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -564,35 +601,32 @@ private[stream] final class GraphInterpreter(
|
|||
|
||||
private[stream] def complete(connection: Int): Unit = {
|
||||
val currentState = portStates(connection)
|
||||
if (Debug) println(s"$Name complete($connection) [$currentState]")
|
||||
portStates(connection) = currentState | OutClosed
|
||||
|
||||
if ((currentState & (InClosed | Pushing)) == 0) {
|
||||
enqueue(connection)
|
||||
}
|
||||
|
||||
completeConnection(assembly.outOwners(connection))
|
||||
if ((currentState & (InClosed | Pushing | Pulling)) == 0) enqueue(connection)
|
||||
if ((currentState & OutClosed) == 0) completeConnection(assembly.outOwners(connection))
|
||||
}
|
||||
|
||||
private[stream] def fail(connection: Int, ex: Throwable): Unit = {
|
||||
val currentState = portStates(connection)
|
||||
if (Debug) println(s"$Name fail($connection, $ex) [$currentState]")
|
||||
portStates(connection) = currentState | (OutClosed | InFailed)
|
||||
if ((currentState & InClosed) == 0) {
|
||||
connectionSlots(connection) = Failed(ex, connectionSlots(connection))
|
||||
enqueue(connection)
|
||||
if ((currentState & (Pulling | Pushing)) == 0) enqueue(connection)
|
||||
}
|
||||
|
||||
completeConnection(assembly.outOwners(connection))
|
||||
if ((currentState & OutClosed) == 0) completeConnection(assembly.outOwners(connection))
|
||||
}
|
||||
|
||||
private[stream] def cancel(connection: Int): Unit = {
|
||||
val currentState = portStates(connection)
|
||||
if (Debug) println(s"$Name cancel($connection) [$currentState]")
|
||||
portStates(connection) = currentState | InClosed
|
||||
if ((currentState & OutClosed) == 0) {
|
||||
connectionSlots(connection) = Empty
|
||||
enqueue(connection)
|
||||
if ((currentState & (Pulling | Pushing)) == 0) enqueue(connection)
|
||||
}
|
||||
|
||||
completeConnection(assembly.inOwners(connection))
|
||||
if ((currentState & InClosed) == 0) completeConnection(assembly.inOwners(connection))
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue