=str: Optmizing GraphInterpreter

This commit is contained in:
Endre Sándor Varga 2015-09-25 16:36:53 +02:00
parent e7a14e53a9
commit f4b614a186
19 changed files with 996 additions and 273 deletions

View file

@ -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))
}