=str: Optmizing GraphInterpreter
This commit is contained in:
parent
e7a14e53a9
commit
f4b614a186
19 changed files with 996 additions and 273 deletions
|
|
@ -3,6 +3,8 @@
|
|||
*/
|
||||
package akka.stream.impl.fusing
|
||||
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.io.Tcp.Closed
|
||||
import akka.stream.stage._
|
||||
import akka.stream.{ Materializer, Shape, Inlet, Outlet }
|
||||
|
||||
|
|
@ -19,37 +21,35 @@ private[stream] object GraphInterpreter {
|
|||
*/
|
||||
final val Debug = false
|
||||
|
||||
final val NoEvent = -1
|
||||
final val Boundary = -1
|
||||
|
||||
final val InReady = 1
|
||||
final val Pulling = 2
|
||||
final val Pushing = 4
|
||||
final val OutReady = 8
|
||||
|
||||
final val InClosed = 16
|
||||
final val OutClosed = 32
|
||||
final val InFailed = 64
|
||||
|
||||
final val PullStartFlip = 3 // 0011
|
||||
final val PullEndFlip = 10 // 1010
|
||||
final val PushStartFlip = 12 //1100
|
||||
final val PushEndFlip = 5 //0101
|
||||
|
||||
/**
|
||||
* Marker object that indicates that a port holds no element since it was already grabbed. The port is still pullable,
|
||||
* but there is no more element to grab.
|
||||
*/
|
||||
case object Empty
|
||||
final case class Failed(ex: Throwable, previousElem: Any)
|
||||
|
||||
sealed trait ConnectionState
|
||||
case object Pulled extends ConnectionState
|
||||
|
||||
sealed trait HasElementState
|
||||
|
||||
sealed trait CompletingState extends ConnectionState
|
||||
final case class CompletedHasElement(element: Any) extends CompletingState with HasElementState
|
||||
final case class PushCompleted(element: Any) extends CompletingState with HasElementState
|
||||
case object Completed extends CompletingState
|
||||
case object Cancelled extends CompletingState
|
||||
final case class Failed(ex: Throwable) extends CompletingState
|
||||
|
||||
val NoEvent = -1
|
||||
val Boundary = -1
|
||||
|
||||
sealed trait PortState
|
||||
case object InFlight extends PortState
|
||||
case object Available extends PortState
|
||||
case object Closed extends PortState
|
||||
|
||||
abstract class UpstreamBoundaryStageLogic[T] extends GraphStageLogic {
|
||||
abstract class UpstreamBoundaryStageLogic[T] extends GraphStageLogic(inCount = 0, outCount = 1) {
|
||||
def out: Outlet[T]
|
||||
}
|
||||
|
||||
abstract class DownstreamBoundaryStageLogic[T] extends GraphStageLogic {
|
||||
abstract class DownstreamBoundaryStageLogic[T] extends GraphStageLogic(inCount = 1, outCount = 0) {
|
||||
def in: Inlet[T]
|
||||
}
|
||||
|
||||
|
|
@ -107,27 +107,52 @@ private[stream] object GraphInterpreter {
|
|||
val logics = Array.ofDim[GraphStageLogic](stages.length)
|
||||
var finalMat: Any = ()
|
||||
|
||||
for (i ← stages.indices) {
|
||||
// FIXME: Support for materialized values in fused islands is not yet figured out!
|
||||
val (logic, mat) = stages(i).createLogicAndMaterializedValue
|
||||
// FIXME: Current temporary hack to support non-fused stages. If there is one stage that will be under index 0.
|
||||
if (i == 0) finalMat = mat
|
||||
var i = 0
|
||||
while (i < stages.length) {
|
||||
// Port initialization loops, these must come first
|
||||
val shape = stages(i).asInstanceOf[GraphStageWithMaterializedValue[Shape, _]].shape
|
||||
|
||||
logics(i) = logic
|
||||
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
|
||||
}
|
||||
|
||||
// FIXME: Support for materialized values in fused islands is not yet figured out!
|
||||
val logicAndMat = stages(i).createLogicAndMaterializedValue
|
||||
// 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
|
||||
|
||||
logics(i) = logicAndMat._1
|
||||
i += 1
|
||||
}
|
||||
|
||||
val inHandlers = Array.ofDim[InHandler](connectionCount)
|
||||
val outHandlers = Array.ofDim[OutHandler](connectionCount)
|
||||
|
||||
for (i ← 0 until connectionCount) {
|
||||
i = 0
|
||||
while (i < connectionCount) {
|
||||
if (ins(i) ne null) {
|
||||
inHandlers(i) = logics(inOwners(i)).inHandlers(ins(i))
|
||||
logics(inOwners(i)).inToConn += ins(i) -> i
|
||||
inHandlers(i) = logics(inOwners(i)).inHandlers(ins(i).id)
|
||||
logics(inOwners(i)).inToConn(ins(i).id) = i
|
||||
}
|
||||
if (outs(i) ne null) {
|
||||
outHandlers(i) = logics(outOwners(i)).outHandlers(outs(i))
|
||||
logics(outOwners(i)).outToConn += outs(i) -> i
|
||||
outHandlers(i) = logics(outOwners(i)).outHandlers(outs(i).id)
|
||||
logics(outOwners(i)).outToConn(outs(i).id) = i
|
||||
}
|
||||
i += 1
|
||||
}
|
||||
|
||||
(inHandlers, outHandlers, logics, finalMat)
|
||||
|
|
@ -168,12 +193,10 @@ private[stream] object GraphInterpreter {
|
|||
* connection represents an output-input port pair (an analogue for a connected RS Publisher-Subscriber pair),
|
||||
* while in the practical sense a connection is a number which represents slots in certain arrays.
|
||||
* In particular
|
||||
* - connectionStates is a mapping from a connection id to a current (or future) state of the connection
|
||||
* - inStates is a mapping from a connection to a [[akka.stream.impl.fusing.GraphInterpreter.PortState]]
|
||||
* that indicates whether the input corresponding
|
||||
* to the connection is currently pullable or completed
|
||||
* - outStates is a mapping from a connection to a [[akka.stream.impl.fusing.GraphInterpreter.PortState]]
|
||||
* that indicates whether the input corresponding to the connection is currently pushable or completed
|
||||
* - portStates contains a bitfield that tracks the states of the ports (output-input) corresponding to this
|
||||
* connection. This bitfield is used to decode the event that is in-flight.
|
||||
* - connectionSlots is a mapping from a connection id to a potential element or exception that accompanies the
|
||||
* event encoded in the portStates bitfield
|
||||
* - inHandlers is a mapping from a connection id to the [[InHandler]] instance that handles the events corresponding
|
||||
* to the input port of the connection
|
||||
* - outHandlers is a mapping from a connection id to the [[OutHandler]] instance that handles the events corresponding
|
||||
|
|
@ -181,20 +204,39 @@ private[stream] object GraphInterpreter {
|
|||
*
|
||||
* On top of these lookup tables there is an eventQueue, represented as a circular buffer of integers. The integers
|
||||
* it contains represents connections that have pending events to be processed. The pending event itself is encoded
|
||||
* in the connectionStates table. This implies that there can be only one event in flight for a given connection, which
|
||||
* is true in almost all cases, except a complete-after-push which is therefore handled with a special event
|
||||
* [[GraphInterpreter#PushCompleted]].
|
||||
* in the portStates bitfield. This implies that there can be only one event in flight for a given connection, which
|
||||
* is true in almost all cases, except a complete-after-push or fail-after-push.
|
||||
*
|
||||
* The layout of the portStates bitfield is the following:
|
||||
*
|
||||
* |- state machn.-| Only one bit is hot among these bits
|
||||
* 64 32 16 | 8 4 2 1 |
|
||||
* +---+---+---|---+---+---+---|
|
||||
* | | | | | | |
|
||||
* | | | | | | | From the following flags only one is active in any given time. These bits encode
|
||||
* | | | | | | | state machine states, and they are "moved" around using XOR masks to keep other bits
|
||||
* | | | | | | | intact.
|
||||
* | | | | | | |
|
||||
* | | | | | | +- InReady: The input port is ready to be pulled
|
||||
* | | | | | +----- Pulling: A pull is active, but have not arrived yet (queued)
|
||||
* | | | | +--------- Pushing: A push is active, but have not arrived yet (queued)
|
||||
* | | | +------------- OutReady: The output port is ready to be pushed
|
||||
* | | |
|
||||
* | | +----------------- InClosed: The input port is closed and will not receive any events.
|
||||
* | | A push might be still in flight which will be then processed first.
|
||||
* | +--------------------- OutClosed: The output port is closed and will not receive any events.
|
||||
* +------------------------- InFailed: Always set in conjunction with InClosed. Indicates that the close event
|
||||
* is a failure
|
||||
*
|
||||
* Sending an event is usually the following sequence:
|
||||
* - An action is requested by a stage logic (push, pull, complete, etc.)
|
||||
* - the availability of the port is set on the sender side to Limbo (inStates or outStates)
|
||||
* - the scheduled event is put in the slot of the connection in the connectionStates table
|
||||
* - the state machine in portStates is transitioned from a ready state to a pending event
|
||||
* - the id of the affected connection is enqueued
|
||||
*
|
||||
* Receiving an event is usually the following sequence:
|
||||
* - id of connection to be processed is dequeued
|
||||
* - the type of the event is determined by the object in the corresponding connectionStates slot
|
||||
* - the availability of the port is set on the receiver side to be Available (inStates or outStates)
|
||||
* - the type of the event is determined from the bits set on portStates
|
||||
* - the state machine in portStates is transitioned to a ready state
|
||||
* - using the inHandlers/outHandlers table the corresponding callback is called on the stage logic.
|
||||
*
|
||||
* Because of the FIFO construction of the queue the interpreter is fair, i.e. a pending event is always executed
|
||||
|
|
@ -205,52 +247,48 @@ private[stream] object GraphInterpreter {
|
|||
private[stream] final class GraphInterpreter(
|
||||
private val assembly: GraphInterpreter.GraphAssembly,
|
||||
val materializer: Materializer,
|
||||
val log: LoggingAdapter,
|
||||
val inHandlers: Array[InHandler], // Lookup table for the InHandler of a connection
|
||||
val outHandlers: Array[OutHandler], // Lookup table for the outHandler of the connection
|
||||
val logics: Array[GraphStageLogic], // Array of stage logics
|
||||
val onAsyncInput: (GraphStageLogic, Any, (Any) ⇒ Unit) ⇒ Unit) {
|
||||
import GraphInterpreter._
|
||||
|
||||
// Maintains the next event (and state) of the connection.
|
||||
// Technically the connection cannot be considered being in the state that is encoded here before the enqueued
|
||||
// connection event has been processed. The inStates and outStates arrays usually protect access to this
|
||||
// field while it is in transient state.
|
||||
val connectionStates = Array.fill[Any](assembly.connectionCount)(Empty)
|
||||
// Maintains additional information for events, basically elements in-flight, or failure.
|
||||
// Other events are encoded in the portStates bitfield.
|
||||
val connectionSlots = Array.fill[Any](assembly.connectionCount)(Empty)
|
||||
|
||||
// Indicates whether the input port is pullable. After pulling it becomes false
|
||||
// Be aware that when inAvailable goes to false outAvailable does not become true immediately, only after
|
||||
// the corresponding event in the queue has been processed
|
||||
val inStates = Array.fill[PortState](assembly.connectionCount)(Available)
|
||||
// Bitfield encoding pending events and various states for efficient querying and updates. See the documentation
|
||||
// of the class for a full description.
|
||||
val portStates = Array.fill[Int](assembly.connectionCount)(InReady)
|
||||
|
||||
// Indicates whether the output port is pushable. After pushing it becomes false
|
||||
// Be aware that when inAvailable goes to false outAvailable does not become true immediately, only after
|
||||
// the corresponding event in the queue has been processed
|
||||
val outStates = Array.fill[PortState](assembly.connectionCount)(InFlight)
|
||||
private[this] var activeStageId = Boundary
|
||||
|
||||
// The number of currently running stages. Once this counter reaches zero, the interpreter is considered to be
|
||||
// completed
|
||||
private var runningStages = assembly.stages.length
|
||||
private[this] var runningStages = assembly.stages.length
|
||||
|
||||
// Counts how many active connections a stage has. Once it reaches zero, the stage is automatically stopped.
|
||||
private val shutdownCounter = Array.tabulate(assembly.stages.length) { i ⇒
|
||||
private[this] val shutdownCounter = Array.tabulate(assembly.stages.length) { i ⇒
|
||||
val shape = assembly.stages(i).shape.asInstanceOf[Shape]
|
||||
shape.inlets.size + shape.outlets.size
|
||||
}
|
||||
|
||||
// An event queue implemented as a circular buffer
|
||||
private val eventQueue = Array.ofDim[Int](256)
|
||||
private val mask = eventQueue.length - 1
|
||||
private var queueHead: Int = 0
|
||||
private var queueTail: Int = 0
|
||||
// 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 mask = eventQueue.length - 1
|
||||
private[this] var queueHead: Int = 0
|
||||
private[this] var queueTail: Int = 0
|
||||
|
||||
/**
|
||||
* 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 -> connection
|
||||
logic.outToConn(logic.out.id) = connection
|
||||
logic.interpreter = this
|
||||
outHandlers(connection) = logic.outHandlers.head._2
|
||||
outHandlers(connection) = logic.outHandlers(0)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -258,9 +296,9 @@ private[stream] final class GraphInterpreter(
|
|||
* (outside the interpreter) to process and inject events.
|
||||
*/
|
||||
def attachDownstreamBoundary(connection: Int, logic: DownstreamBoundaryStageLogic[_]): Unit = {
|
||||
logic.inToConn += logic.in -> connection
|
||||
logic.inToConn(logic.in.id) = connection
|
||||
logic.interpreter = this
|
||||
inHandlers(connection) = logic.inHandlers.head._2
|
||||
inHandlers(connection) = logic.inHandlers(0)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -279,10 +317,15 @@ private[stream] final class GraphInterpreter(
|
|||
def init(): Unit = {
|
||||
var i = 0
|
||||
while (i < logics.length) {
|
||||
logics(i).stageId = i
|
||||
logics(i).interpreter = this
|
||||
logics(i).beforePreStart()
|
||||
logics(i).preStart()
|
||||
val logic = logics(i)
|
||||
logic.stageId = i
|
||||
logic.interpreter = this
|
||||
try {
|
||||
logic.beforePreStart()
|
||||
logic.preStart()
|
||||
} catch {
|
||||
case NonFatal(e) ⇒ logic.failStage(e)
|
||||
}
|
||||
i += 1
|
||||
}
|
||||
}
|
||||
|
|
@ -293,10 +336,7 @@ private[stream] final class GraphInterpreter(
|
|||
def finish(): Unit = {
|
||||
var i = 0
|
||||
while (i < logics.length) {
|
||||
if (!isStageCompleted(i)) {
|
||||
logics(i).postStop()
|
||||
logics(i).afterPostStop()
|
||||
}
|
||||
if (!isStageCompleted(i)) finalizeStage(logics(i))
|
||||
i += 1
|
||||
}
|
||||
}
|
||||
|
|
@ -323,16 +363,8 @@ private[stream] final class GraphInterpreter(
|
|||
try processEvent(connection)
|
||||
catch {
|
||||
case NonFatal(e) ⇒
|
||||
val stageId = connectionStates(connection) match {
|
||||
case Failed(ex) ⇒ throw new IllegalStateException("Double fault. Failure while handling failure.", e)
|
||||
case Pulled ⇒ assembly.outOwners(connection)
|
||||
case Completed ⇒ assembly.inOwners(connection)
|
||||
case Cancelled ⇒ assembly.outOwners(connection)
|
||||
case PushCompleted(elem) ⇒ assembly.inOwners(connection)
|
||||
case pushedElem ⇒ assembly.inOwners(connection)
|
||||
}
|
||||
if (stageId == Boundary) throw e
|
||||
else logics(stageId).failStage(e)
|
||||
if (activeStageId == Boundary) throw e
|
||||
else logics(activeStageId).failStage(e)
|
||||
}
|
||||
eventsRemaining -= 1
|
||||
if (eventsRemaining > 0) connection = dequeue()
|
||||
|
|
@ -344,61 +376,62 @@ private[stream] final class GraphInterpreter(
|
|||
private def processEvent(connection: Int): Unit = {
|
||||
|
||||
def processElement(elem: Any): Unit = {
|
||||
if (!isStageCompleted(assembly.inOwners(connection))) {
|
||||
if (GraphInterpreter.Debug) println(s"PUSH ${outOwnerName(connection)} -> ${inOwnerName(connection)}, $elem")
|
||||
inStates(connection) = Available
|
||||
inHandlers(connection).onPush()
|
||||
if (GraphInterpreter.Debug) println(s"PUSH ${outOwnerName(connection)} -> ${inOwnerName(connection)}, $elem")
|
||||
activeStageId = assembly.inOwners(connection)
|
||||
portStates(connection) ^= PushEndFlip
|
||||
inHandlers(connection).onPush()
|
||||
}
|
||||
|
||||
val code = portStates(connection)
|
||||
|
||||
// Manual fast decoding, fast paths are PUSH and PULL
|
||||
// PUSH
|
||||
if ((code & (Pushing | InClosed | OutClosed)) == Pushing) {
|
||||
processElement(connectionSlots(connection))
|
||||
|
||||
// PULL
|
||||
} else if ((code & (Pulling | OutClosed | InClosed)) == Pulling) {
|
||||
if (GraphInterpreter.Debug) println(s"PULL ${inOwnerName(connection)} -> ${outOwnerName(connection)}")
|
||||
portStates(connection) ^= PullEndFlip
|
||||
activeStageId = 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)}")
|
||||
portStates(connection) |= OutClosed
|
||||
activeStageId = assembly.outOwners(connection)
|
||||
outHandlers(connection).onDownstreamFinish()
|
||||
completeConnection(stageId)
|
||||
} 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)}")
|
||||
portStates(connection) |= InClosed
|
||||
activeStageId = assembly.inOwners(connection)
|
||||
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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
connectionStates(connection) match {
|
||||
case Pulled ⇒
|
||||
if (!isStageCompleted(assembly.outOwners(connection))) {
|
||||
if (GraphInterpreter.Debug) println(s"PULL ${inOwnerName(connection)} -> ${outOwnerName(connection)}")
|
||||
outStates(connection) = Available
|
||||
outHandlers(connection).onPull()
|
||||
}
|
||||
case Completed | CompletedHasElement(_) ⇒
|
||||
val stageId = assembly.inOwners(connection)
|
||||
if (!isStageCompleted(stageId) && inStates(connection) != Closed) {
|
||||
if (GraphInterpreter.Debug) println(s"COMPLETE ${outOwnerName(connection)} -> ${inOwnerName(connection)}")
|
||||
inStates(connection) = Closed
|
||||
inHandlers(connection).onUpstreamFinish()
|
||||
completeConnection(stageId)
|
||||
}
|
||||
case Failed(ex) ⇒
|
||||
val stageId = assembly.inOwners(connection)
|
||||
if (!isStageCompleted(stageId) && inStates(connection) != Closed) {
|
||||
if (GraphInterpreter.Debug) println(s"FAIL ${outOwnerName(connection)} -> ${inOwnerName(connection)}")
|
||||
inStates(connection) = Closed
|
||||
inHandlers(connection).onUpstreamFailure(ex)
|
||||
completeConnection(stageId)
|
||||
}
|
||||
case Cancelled ⇒
|
||||
val stageId = assembly.outOwners(connection)
|
||||
if (!isStageCompleted(stageId) && outStates(connection) != Closed) {
|
||||
if (GraphInterpreter.Debug) println(s"CANCEL ${inOwnerName(connection)} -> ${outOwnerName(connection)}")
|
||||
outStates(connection) = Closed
|
||||
outHandlers(connection).onDownstreamFinish()
|
||||
completeConnection(stageId)
|
||||
}
|
||||
case PushCompleted(elem) ⇒
|
||||
val stageId = assembly.inOwners(connection)
|
||||
if (!isStageCompleted(stageId) && inStates(connection) != Closed) {
|
||||
inStates(connection) = Available
|
||||
connectionStates(connection) = elem
|
||||
processElement(elem)
|
||||
val elemAfter = connectionStates(connection)
|
||||
if (elemAfter == Empty) enqueue(connection, Completed)
|
||||
else enqueue(connection, CompletedHasElement(elemAfter))
|
||||
} else {
|
||||
connectionStates(connection) = Completed
|
||||
}
|
||||
|
||||
case pushedElem ⇒ processElement(pushedElem)
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private def dequeue(): Int = {
|
||||
|
|
@ -412,29 +445,14 @@ private[stream] final class GraphInterpreter(
|
|||
}
|
||||
}
|
||||
|
||||
private def enqueue(connection: Int, event: Any): Unit = {
|
||||
connectionStates(connection) = event
|
||||
private def enqueue(connection: Int): Unit = {
|
||||
eventQueue(queueTail & mask) = connection
|
||||
queueTail += 1
|
||||
}
|
||||
|
||||
// Returns true if a connection has been completed *or if the completion event is already enqueued*. This is useful
|
||||
// to prevent redundant completion events in case of concurrent invocation on both sides of the connection.
|
||||
// I.e. when one side already enqueued the completion event, then the other side will not enqueue the event since
|
||||
// there is noone to process it anymore.
|
||||
def isConnectionCompleting(connection: Int): Boolean = connectionStates(connection).isInstanceOf[CompletingState]
|
||||
|
||||
// Returns true if the given stage is alredy completed
|
||||
def isStageCompleted(stageId: Int): Boolean = stageId != Boundary && shutdownCounter(stageId) == 0
|
||||
|
||||
private def isPushInFlight(connection: Int): Boolean =
|
||||
(inStates(connection) == InFlight) && // Other side has not been notified
|
||||
hasElement(connection)
|
||||
|
||||
private def hasElement(connection: Int): Boolean =
|
||||
!connectionStates(connection).isInstanceOf[ConnectionState] &&
|
||||
connectionStates(connection) != Empty
|
||||
|
||||
// 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 = {
|
||||
|
|
@ -445,53 +463,66 @@ private[stream] final class GraphInterpreter(
|
|||
// This was the last active connection keeping this stage alive
|
||||
if (activeConnections == 1) {
|
||||
runningStages -= 1
|
||||
logics(stageId).postStop()
|
||||
logics(stageId).afterPostStop()
|
||||
finalizeStage(logics(stageId))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def finalizeStage(logic: GraphStageLogic): Unit = {
|
||||
try {
|
||||
logic.postStop()
|
||||
logic.afterPostStop()
|
||||
} catch {
|
||||
case NonFatal(e) ⇒
|
||||
log.error(s"Error during postStop in [${assembly.stages(logic.stageId)}]", e)
|
||||
}
|
||||
}
|
||||
|
||||
private[stream] def push(connection: Int, elem: Any): Unit = {
|
||||
if (!(inStates(connection) eq Closed)) {
|
||||
outStates(connection) = InFlight
|
||||
enqueue(connection, elem)
|
||||
if ((portStates(connection) & InClosed) == 0) {
|
||||
portStates(connection) ^= PushStartFlip
|
||||
connectionSlots(connection) = elem
|
||||
enqueue(connection)
|
||||
}
|
||||
}
|
||||
|
||||
private[stream] def pull(connection: Int): Unit = {
|
||||
if (!(outStates(connection) eq Closed)) {
|
||||
inStates(connection) = InFlight
|
||||
enqueue(connection, Pulled)
|
||||
if ((portStates(connection) & OutClosed) == 0) {
|
||||
portStates(connection) ^= PullStartFlip
|
||||
enqueue(connection)
|
||||
}
|
||||
}
|
||||
|
||||
private[stream] def complete(connection: Int): Unit = {
|
||||
outStates(connection) = Closed
|
||||
if (!isConnectionCompleting(connection) && (inStates(connection) ne Closed)) {
|
||||
if (hasElement(connection)) {
|
||||
// There is a pending push, we change the signal to be a PushCompleted (there can be only one signal in flight
|
||||
// for a connection)
|
||||
if (inStates(connection) == InFlight)
|
||||
connectionStates(connection) = PushCompleted(connectionStates(connection))
|
||||
else enqueue(connection, CompletedHasElement(connectionStates(connection)))
|
||||
} else enqueue(connection, Completed)
|
||||
val currentState = portStates(connection)
|
||||
portStates(connection) = portStates(connection) | OutClosed
|
||||
if ((currentState & InClosed) == 0) {
|
||||
if ((currentState & Pushing) != 0) {} // FIXME: Fold into previous condition
|
||||
else if (connectionSlots(connection) != Empty)
|
||||
enqueue(connection)
|
||||
else
|
||||
enqueue(connection)
|
||||
}
|
||||
completeConnection(assembly.outOwners(connection))
|
||||
}
|
||||
|
||||
private[stream] def fail(connection: Int, ex: Throwable): Unit = {
|
||||
outStates(connection) = Closed
|
||||
if (!isConnectionCompleting(connection) && (inStates(connection) ne Closed))
|
||||
enqueue(connection, Failed(ex))
|
||||
portStates(connection) |= (OutClosed | InFailed)
|
||||
if ((portStates(connection) & InClosed) == 0) {
|
||||
connectionSlots(connection) = Failed(ex, connectionSlots(connection))
|
||||
enqueue(connection)
|
||||
}
|
||||
|
||||
completeConnection(assembly.outOwners(connection))
|
||||
}
|
||||
|
||||
private[stream] def cancel(connection: Int): Unit = {
|
||||
inStates(connection) = Closed
|
||||
if (!isConnectionCompleting(connection) && (outStates(connection) ne Closed))
|
||||
enqueue(connection, Cancelled)
|
||||
portStates(connection) |= InClosed
|
||||
if ((portStates(connection) & OutClosed) == 0) {
|
||||
connectionSlots(connection) = Empty
|
||||
enqueue(connection)
|
||||
}
|
||||
|
||||
completeConnection(assembly.inOwners(connection))
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue