Low level interpreter tests working without GraphAssembly #22423

This commit is contained in:
Johan Andrén 2017-03-07 17:06:41 +01:00 committed by GitHub
parent 44260fe5d3
commit c028b550f2
26 changed files with 354 additions and 575 deletions

View file

@ -1,103 +1,268 @@
/**
* Copyright (C) 2015-2017 Lightbend Inc. <http://www.lightbend.com>
*/
/* FIXME enable again
package akka.stream.impl.fusing
import akka.event.Logging
import akka.stream.ActorAttributes.SupervisionStrategy
import akka.stream.Supervision.Decider
import akka.stream._
import akka.stream.impl.fusing.GraphInterpreter.{ DownstreamBoundaryStageLogic, Failed, GraphAssembly, UpstreamBoundaryStageLogic }
import akka.stream.impl.fusing.GraphInterpreter.{ Connection, DownstreamBoundaryStageLogic, Failed, UpstreamBoundaryStageLogic }
import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler, _ }
import akka.stream.testkit.StreamSpec
import akka.stream.testkit.Utils.TE
import akka.stream.impl.fusing.GraphInterpreter.GraphAssembly
import scala.collection.{ Map SMap }
import scala.language.existentials
object GraphInterpreterSpecKit {
/**
* Create logics and enumerate stages and ports
*
* @param stages Stages to "materialize" into graph stage logic instances
* @param upstreams Upstream boundary logics that are already instances of graph stage logic and should be
* part of the graph, is placed before the rest of the stages
* @param downstreams Downstream boundary logics, is placed after the other stages
* @param attributes Optional set of attributes to pass to the stages when creating the logics
* @return Created logics and the maps of all inlets respective outlets to those logics
*/
private[stream] def createLogics(
stages: Array[GraphStageWithMaterializedValue[_ <: Shape, _]],
upstreams: Array[UpstreamBoundaryStageLogic[_]],
downstreams: Array[DownstreamBoundaryStageLogic[_]],
attributes: Array[Attributes] = Array.empty): (Array[GraphStageLogic], SMap[Inlet[_], GraphStageLogic], SMap[Outlet[_], GraphStageLogic]) = {
if (attributes.nonEmpty && attributes.length != stages.length)
throw new IllegalArgumentException("Attributes must be either empty or one per stage")
var inOwners = SMap.empty[Inlet[_], GraphStageLogic]
var outOwners = SMap.empty[Outlet[_], GraphStageLogic]
val logics = Array.ofDim[GraphStageLogic](upstreams.length + stages.length + downstreams.length)
var idx = 0
while (idx < upstreams.length) {
val upstream = upstreams(idx)
upstream.stageId = idx
logics(idx) = upstream
upstream.out.id = 0
outOwners = outOwners + (upstream.out upstream)
idx += 1
}
var stageIdx = 0
while (stageIdx < stages.length) {
val stage = stages(stageIdx)
setPortIds(stage.shape)
val stageAttributes =
if (attributes.nonEmpty) stage.traversalBuilder.attributes and attributes(stageIdx)
else stage.traversalBuilder.attributes
val logic = stage.createLogicAndMaterializedValue(stageAttributes)._1
logic.stageId = idx
var inletIdx = 0
while (inletIdx < stage.shape.inlets.length) {
val inlet = stage.shape.inlets(inletIdx)
inlet.id = inletIdx
inOwners = inOwners + (inlet logic)
inletIdx += 1
}
var outletIdx = 0
while (outletIdx < stage.shape.outlets.length) {
val outlet = stage.shape.outlets(outletIdx)
outlet.id = outletIdx
outOwners = outOwners + (outlet logic)
outletIdx += 1
}
logics(idx) = logic
idx += 1
stageIdx += 1
}
var downstreamIdx = 0
while (downstreamIdx < downstreams.length) {
val downstream = downstreams(downstreamIdx)
downstream.stageId = idx
logics(idx) = downstream
downstream.in.id = 0
inOwners = inOwners + (downstream.in downstream)
idx += 1
downstreamIdx += 1
}
(logics, inOwners, outOwners)
}
/**
* Create connections given a list of flow logics where each one has one connection to the next one
*/
private[stream] def createLinearFlowConnections(logics: Seq[GraphStageLogic]): Array[Connection] = {
require(logics.length >= 2, s"$logics is too short to create a linear flow")
logics.sliding(2).zipWithIndex.map {
case (window, idx)
val outOwner = window(0)
val inOwner = window(1)
val connection = new Connection(
id = idx,
outOwnerId = outOwner.stageId,
outOwner = outOwner,
outHandler = outOwner.outHandler(0),
inOwner = inOwner,
inOwnerId = inOwner.stageId,
inHandler = inOwner.inHandler(0)
)
outOwner.portToConn(outOwner.inCount) = connection
inOwner.portToConn(0) = connection
connection
}.toArray
}
/**
* Create interpreter connections for all the given `connectedPorts`.
*/
private[stream] def createConnections(
logics: Seq[GraphStageLogic],
connectedPorts: Seq[(Outlet[_], Inlet[_])],
inOwners: SMap[Inlet[_], GraphStageLogic],
outOwners: SMap[Outlet[_], GraphStageLogic]): Array[Connection] = {
val connections = Array.ofDim[Connection](connectedPorts.size)
connectedPorts.zipWithIndex.foreach {
case ((outlet, inlet), idx)
val outOwner = outOwners(outlet)
val inOwner = inOwners(inlet)
val connection = new Connection(
id = idx,
outOwnerId = outOwner.stageId,
outOwner = outOwner,
outHandler = outOwner.outHandler(outlet.id),
inOwnerId = inOwner.stageId,
inOwner = inOwner,
inHandler = inOwner.inHandler(inlet.id)
)
connections(idx) = connection
inOwner.portToConn(inlet.id) = connection
outOwner.portToConn(outOwner.inCount + outlet.id) = connection
}
connections
}
private def setPortIds(shape: Shape): Unit = {
shape.inlets.zipWithIndex.foreach {
case (inlet, idx) inlet.id = idx
}
shape.outlets.zipWithIndex.foreach {
case (outlet, idx) outlet.id = idx
}
}
private def setPortIds(stage: GraphStageWithMaterializedValue[_ <: Shape, _]): Unit = {
stage.shape.inlets.zipWithIndex.foreach { case (inlet, idx) inlet.id = idx }
stage.shape.outlets.zipWithIndex.foreach { case (inlet, idx) inlet.id = idx }
}
private def setLogicIds(logics: Array[GraphStageLogic]): Unit = {
logics.zipWithIndex.foreach { case (logic, idx) logic.stageId = idx }
}
}
trait GraphInterpreterSpecKit extends StreamSpec {
import GraphInterpreterSpecKit._
val logger = Logging(system, "InterpreterSpecKit")
abstract class Builder {
private var _interpreter: GraphInterpreter = _
protected def interpreter: GraphInterpreter = _interpreter
def stepAll(): Unit = interpreter.execute(eventLimit = Int.MaxValue)
def step(): Unit = interpreter.execute(eventLimit = 1)
object Upstream extends UpstreamBoundaryStageLogic[Int] {
override val out = Outlet[Int]("up")
out.id = 0
override def toString = "Upstream"
setHandler(out, new OutHandler {
override def onPull() = {
// TODO handler needed but should it do anything?
}
override def toString = "Upstream.OutHandler"
})
}
object Downstream extends DownstreamBoundaryStageLogic[Int] {
override val in = Inlet[Int]("down")
in.id = 0
setHandler(in, new InHandler {
override def onPush() = {
// TODO handler needed but should it do anything?
}
override def toString = "Downstream.InHandler"
})
override def toString = "Downstream"
}
class AssemblyBuilder(stages: Seq[GraphStageWithMaterializedValue[_ <: Shape, _]]) {
var upstreams = Vector.empty[(UpstreamBoundaryStageLogic[_], Inlet[_])]
var downstreams = Vector.empty[(Outlet[_], DownstreamBoundaryStageLogic[_])]
var connections = Vector.empty[(Outlet[_], Inlet[_])]
private var upstreams = Vector.empty[UpstreamBoundaryStageLogic[_]]
private var downstreams = Vector.empty[DownstreamBoundaryStageLogic[_]]
private var connectedPorts = Vector.empty[(Outlet[_], Inlet[_])]
def connect[T](upstream: UpstreamBoundaryStageLogic[T], in: Inlet[T]): AssemblyBuilder = {
upstreams :+= upstream in
upstreams :+= upstream
connectedPorts :+= upstream.out in
this
}
def connect[T](out: Outlet[T], downstream: DownstreamBoundaryStageLogic[T]): AssemblyBuilder = {
downstreams :+= out downstream
downstreams :+= downstream
connectedPorts :+= out downstream.in
this
}
def connect[T](out: Outlet[T], in: Inlet[T]): AssemblyBuilder = {
connections :+= out in
connectedPorts :+= out in
this
}
def buildAssembly(): GraphAssembly = {
val ins = upstreams.map(_._2) ++ connections.map(_._2)
val outs = connections.map(_._1) ++ downstreams.map(_._1)
val inOwners = ins.map { in stages.indexWhere(_.shape.inlets.contains(in)) }
val outOwners = outs.map { out stages.indexWhere(_.shape.outlets.contains(out)) }
new GraphAssembly(
stages.toArray,
Array.fill(stages.size)(Attributes.none),
(ins ++ Vector.fill(downstreams.size)(null)).toArray,
(inOwners ++ Vector.fill(downstreams.size)(-1)).toArray,
(Vector.fill(upstreams.size)(null) ++ outs).toArray,
(Vector.fill(upstreams.size)(-1) ++ outOwners).toArray)
}
def init(): Unit = {
val assembly = buildAssembly()
val (logics, inOwners, outOwners) = createLogics(stages.toArray, upstreams.toArray, downstreams.toArray)
val conns = createConnections(logics, connectedPorts, inOwners, outOwners)
val (conns, logics) =
assembly.materialize(Attributes.none, assembly.stages.map(_.module), new java.util.HashMap, _ ())
_interpreter = new GraphInterpreter(assembly, NoMaterializer, logger, logics, conns,
(_, _, _) (), fuzzingMode = false, null)
for ((upstream, i) upstreams.zipWithIndex) {
_interpreter.attachUpstreamBoundary(conns(i), upstream._1)
}
for ((downstream, i) downstreams.zipWithIndex) {
_interpreter.attachDownstreamBoundary(conns(i + upstreams.size + connections.size), downstream._2)
}
_interpreter.init(null)
manualInit(logics.toArray, conns)
}
}
def manualInit(assembly: GraphAssembly): Unit = {
val (connections, logics) =
assembly.materialize(Attributes.none, assembly.stages.map(_.module), new java.util.HashMap, _ ())
_interpreter = new GraphInterpreter(assembly, NoMaterializer, logger, logics, connections,
(_, _, _) (), fuzzingMode = false, null)
def manualInit(logics: Array[GraphStageLogic], connections: Array[Connection]): Unit = {
_interpreter = new GraphInterpreter(
NoMaterializer,
logger,
logics,
connections,
onAsyncInput = (_, _, _) (),
fuzzingMode = false,
context = null)
_interpreter.init(null)
}
def builder(stages: GraphStageWithMaterializedValue[_ <: Shape, _]*): AssemblyBuilder = new AssemblyBuilder(stages)
def builder(stages: GraphStageWithMaterializedValue[_ <: Shape, _]*): AssemblyBuilder =
new AssemblyBuilder(stages.toVector)
}
abstract class TestSetup extends Builder {
@ -132,6 +297,7 @@ trait GraphInterpreterSpecKit extends StreamSpec {
setHandler(out, new OutHandler {
override def onPull(): Unit = lastEvent += RequestOne(UpstreamProbe.this)
override def onDownstreamFinish(): Unit = lastEvent += Cancel(UpstreamProbe.this)
override def toString = s"${UpstreamProbe.this.toString}.outHandler"
})
def onNext(elem: T, eventLimit: Int = Int.MaxValue): Unit = {
@ -161,6 +327,7 @@ trait GraphInterpreterSpecKit extends StreamSpec {
override def onPush(): Unit = lastEvent += OnNext(DownstreamProbe.this, grab(in))
override def onUpstreamFinish(): Unit = lastEvent += OnComplete(DownstreamProbe.this)
override def onUpstreamFailure(ex: Throwable): Unit = lastEvent += OnError(DownstreamProbe.this, ex)
override def toString = s"${DownstreamProbe.this.toString}.inHandler"
})
def requestOne(eventLimit: Int = Int.MaxValue): Unit = {
@ -185,6 +352,8 @@ trait GraphInterpreterSpecKit extends StreamSpec {
class EventPropagateStage extends GraphStage[FlowShape[Int, Int]] {
val in = Inlet[Int]("Propagate.in")
val out = Outlet[Int]("Propagate.out")
in.id = 0
out.id = 0
override val shape: FlowShape[Int, Int] = FlowShape(in, out)
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) with InHandler with OutHandler {
@ -196,6 +365,7 @@ trait GraphInterpreterSpecKit extends StreamSpec {
setHandlers(in, out, this)
}
override def toString = "EventPropagateStage"
}
// step() means different depending whether we have a stage between the two probes or not
@ -236,29 +406,22 @@ trait GraphInterpreterSpecKit extends StreamSpec {
})
}
private val assembly = if (!chasing) {
new GraphAssembly(
stages = Array.empty,
originalAttributes = Array.empty,
ins = Array(null),
inOwners = Array(-1),
outs = Array(null),
outOwners = Array(-1))
} else {
val propagateStage = new EventPropagateStage
new GraphAssembly(
stages = Array(propagateStage),
originalAttributes = Array(Attributes.none),
ins = Array(propagateStage.in, null),
inOwners = Array(0, -1),
outs = Array(null, propagateStage.out),
outOwners = Array(-1, 0))
}
val (logics, connections) =
if (!chasing) {
val logics = Array[GraphStageLogic](out, in)
setLogicIds(logics)
val connections = GraphInterpreterSpecKit.createLinearFlowConnections(logics)
(logics, connections)
} else {
val propagateStage = new EventPropagateStage
setPortIds(propagateStage)
val logics = Array[GraphStageLogic](out, propagateStage.createLogic(Attributes.none), in)
setLogicIds(logics)
val connections = GraphInterpreterSpecKit.createLinearFlowConnections(logics)
(logics, connections)
}
manualInit(assembly)
interpreter.attachDownstreamBoundary(interpreter.connections(if (chasing) 1 else 0), in)
interpreter.attachUpstreamBoundary(interpreter.connections(0), out)
interpreter.init(null)
manualInit(logics, connections)
}
abstract class FailingStageSetup(initFailOnNextEvent: Boolean = false) extends TestSetup {
@ -280,7 +443,7 @@ trait GraphInterpreterSpecKit extends StreamSpec {
// Must be lazy because I turned this stage "inside-out" therefore changing initialization order
// to make tests a bit more readable
lazy val stage: GraphStageLogic = new GraphStageLogic(stageshape) {
lazy val insideOutStage: GraphStageLogic = new GraphStageLogic(stageshape) {
private def mayFail(task: Unit): Unit = {
if (!_failOnNextEvent) task
else {
@ -293,24 +456,27 @@ trait GraphInterpreterSpecKit extends StreamSpec {
override def onPush(): Unit = mayFail(push(stageout, grab(stagein)))
override def onUpstreamFinish(): Unit = mayFail(completeStage())
override def onUpstreamFailure(ex: Throwable): Unit = mayFail(failStage(ex))
override def toString = "insideOutStage.stagein"
})
setHandler(stageout, new OutHandler {
override def onPull(): Unit = mayFail(pull(stagein))
override def onDownstreamFinish(): Unit = mayFail(completeStage())
override def toString = "insideOutStage.stageout"
})
override def preStart(): Unit = mayFail(lastEvent += PreStart(stage))
override def preStart(): Unit = mayFail(lastEvent += PreStart(insideOutStage))
override def postStop(): Unit =
if (!_failOnPostStop) lastEvent += PostStop(stage)
if (!_failOnPostStop) lastEvent += PostStop(insideOutStage)
else throw testException
override def toString = "stage"
override def toString = "insideOutStage"
}
private val sandwitchStage = new GraphStage[FlowShape[Int, Int]] {
override def shape = stageshape
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = stage
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = insideOutStage
override def toString = "sandwitchStage"
}
class UpstreamPortProbe[T] extends UpstreamProbe[T]("upstreamPort") {
@ -330,8 +496,7 @@ trait GraphInterpreterSpecKit extends StreamSpec {
.init()
}
abstract class OneBoundedSetupWithDecider[T](decider: Decider, _ops: GraphStageWithMaterializedValue[Shape, Any]*) extends Builder {
val ops = _ops.toArray
abstract class OneBoundedSetupWithDecider[T](decider: Decider, ops: GraphStageWithMaterializedValue[Shape, Any]*) extends Builder {
val upstream = new UpstreamOneBoundedProbe[T]
val downstream = new DownstreamOneBoundedPortProbe[T]
@ -349,35 +514,15 @@ trait GraphInterpreterSpecKit extends StreamSpec {
private def run() = interpreter.execute(Int.MaxValue)
private def initialize(): Unit = {
import GraphInterpreter.Boundary
var i = 0
val attributes = Array.fill[Attributes](ops.length)(ActorAttributes.supervisionStrategy(decider))
val ins = Array.ofDim[Inlet[_]](ops.length + 1)
val inOwners = Array.ofDim[Int](ops.length + 1)
val outs = Array.ofDim[Outlet[_]](ops.length + 1)
val outOwners = Array.ofDim[Int](ops.length + 1)
ins(ops.length) = null
inOwners(ops.length) = Boundary
outs(0) = null
outOwners(0) = Boundary
while (i < ops.length) {
val stage = ops(i).asInstanceOf[GraphStageWithMaterializedValue[FlowShape[_, _], _]]
ins(i) = stage.shape.in
inOwners(i) = i
outs(i + 1) = stage.shape.out
outOwners(i + 1) = i
i += 1
}
manualInit(new GraphAssembly(ops, attributes, ins, inOwners, outs, outOwners))
interpreter.attachUpstreamBoundary(0, upstream)
interpreter.attachDownstreamBoundary(ops.length, downstream)
interpreter.init(null)
val supervision = ActorAttributes.supervisionStrategy(decider)
val attributes = Array.fill[Attributes](ops.length)(supervision)
val (logics, _, _) = createLogics(
ops.toArray,
Array(upstream),
Array(downstream),
attributes)
val connections = createLinearFlowConnections(logics)
manualInit(logics, connections)
}
initialize()
@ -452,6 +597,7 @@ trait GraphInterpreterSpecKit extends StreamSpec {
}
abstract class OneBoundedSetup[T](_ops: GraphStageWithMaterializedValue[Shape, Any]*) extends OneBoundedSetupWithDecider[T](Supervision.stoppingDecider, _ops: _*)
abstract class OneBoundedSetup[T](_ops: GraphStageWithMaterializedValue[Shape, Any]*)
extends OneBoundedSetupWithDecider[T](Supervision.stoppingDecider, _ops: _*)
}
*/