!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:
Roland Kuhn 2015-10-31 14:46:10 +01:00
parent 20f54435f1
commit 556012b7ee
107 changed files with 2456 additions and 3061 deletions

View file

@ -5,35 +5,32 @@ package akka.stream.impl.fusing
import akka.event.Logging
import akka.stream._
import akka.stream.impl.fusing.GraphInterpreter.{ Failed, GraphAssembly, DownstreamBoundaryStageLogic, UpstreamBoundaryStageLogic }
import akka.stream.stage.{ InHandler, OutHandler, GraphStage, GraphStageLogic }
import akka.stream.impl.fusing.GraphInterpreter.{ DownstreamBoundaryStageLogic, Failed, GraphAssembly, UpstreamBoundaryStageLogic }
import akka.stream.stage.AbstractStage.PushPullGraphStage
import akka.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler, _ }
import akka.stream.testkit.AkkaSpec
import akka.stream.testkit.Utils.TE
import akka.stream.impl.fusing.GraphInterpreter.GraphAssembly
trait GraphInterpreterSpecKit extends AkkaSpec {
sealed trait TestEvent {
def source: GraphStageLogic
}
case class OnComplete(source: GraphStageLogic) extends TestEvent
case class Cancel(source: GraphStageLogic) extends TestEvent
case class OnError(source: GraphStageLogic, cause: Throwable) extends TestEvent
case class OnNext(source: GraphStageLogic, elem: Any) extends TestEvent
case class RequestOne(source: GraphStageLogic) extends TestEvent
case class RequestAnother(source: GraphStageLogic) extends TestEvent
case class PreStart(source: GraphStageLogic) extends TestEvent
case class PostStop(source: GraphStageLogic) extends TestEvent
abstract class TestSetup {
protected var lastEvent: Set[TestEvent] = Set.empty
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
}
object Downstream extends DownstreamBoundaryStageLogic[Int] {
override val in = Inlet[Int]("down")
in.id = 0
}
class AssemblyBuilder(stages: Seq[GraphStage[_ <: Shape]]) {
var upstreams = Vector.empty[(UpstreamBoundaryStageLogic[_], Inlet[_])]
var downstreams = Vector.empty[(Outlet[_], DownstreamBoundaryStageLogic[_])]
@ -54,20 +51,25 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
this
}
def init(): Unit = {
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)) }
val assembly = new GraphAssembly(
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)
}
val (inHandlers, outHandlers, logics, _) = assembly.materialize()
def init(): Unit = {
val assembly = buildAssembly()
val (inHandlers, outHandlers, logics, _) = assembly.materialize(Attributes.none)
_interpreter = new GraphInterpreter(assembly, NoMaterializer, Logging(system, classOf[TestSetup]), inHandlers, outHandlers, logics, (_, _, _) ())
for ((upstream, i) upstreams.zipWithIndex) {
@ -83,11 +85,29 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
}
def manualInit(assembly: GraphAssembly): Unit = {
val (inHandlers, outHandlers, logics, _) = assembly.materialize()
val (inHandlers, outHandlers, logics, _) = assembly.materialize(Attributes.none)
_interpreter = new GraphInterpreter(assembly, NoMaterializer, Logging(system, classOf[TestSetup]), inHandlers, outHandlers, logics, (_, _, _) ())
}
def builder(stages: GraphStage[_ <: Shape]*): AssemblyBuilder = new AssemblyBuilder(stages.toSeq)
def builder(stages: GraphStage[_ <: Shape]*): AssemblyBuilder = new AssemblyBuilder(stages)
}
abstract class TestSetup extends Builder {
sealed trait TestEvent {
def source: GraphStageLogic
}
case class OnComplete(source: GraphStageLogic) extends TestEvent
case class Cancel(source: GraphStageLogic) extends TestEvent
case class OnError(source: GraphStageLogic, cause: Throwable) extends TestEvent
case class OnNext(source: GraphStageLogic, elem: Any) extends TestEvent
case class RequestOne(source: GraphStageLogic) extends TestEvent
case class RequestAnother(source: GraphStageLogic) extends TestEvent
case class PreStart(source: GraphStageLogic) extends TestEvent
case class PostStop(source: GraphStageLogic) extends TestEvent
protected var lastEvent: Set[TestEvent] = Set.empty
def lastEvents(): Set[TestEvent] = {
val result = lastEvent
@ -158,7 +178,7 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
// Modified onPush that does not grab() automatically the element. This accesses some internals.
override def onPush(): Unit = {
val internalEvent = interpreter.connectionSlots(inToConn(in.id))
val internalEvent = interpreter.connectionSlots(portToConn(in.id))
internalEvent match {
case Failed(_, elem) lastEvent += OnNext(DownstreamPortProbe.this, elem)
@ -173,6 +193,7 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
private val assembly = new GraphAssembly(
stages = Array.empty,
originalAttributes = Array.empty,
ins = Array(null),
inOwners = Array(-1),
outs = Array(null),
@ -233,7 +254,7 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
private val sandwitchStage = new GraphStage[FlowShape[Int, Int]] {
override def shape = stageshape
override def createLogic: GraphStageLogic = stage
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = stage
}
class UpstreamPortProbe[T] extends UpstreamProbe[T]("upstreamPort") {
@ -253,4 +274,133 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
.init()
}
abstract class OneBoundedSetup[T](ops: Array[GraphStageWithMaterializedValue[Shape, Any]]) extends Builder {
def this(ops: Iterable[Stage[_, _]]) = {
this(ops.map { op
new PushPullGraphStage[Any, Any, Any](
(_) op.asInstanceOf[Stage[Any, Any]],
Attributes.none)
}.toArray.asInstanceOf[Array[GraphStageWithMaterializedValue[Shape, Any]]])
}
val upstream = new UpstreamOneBoundedProbe[T]
val downstream = new DownstreamOneBoundedPortProbe[T]
var lastEvent = Set.empty[TestEvent]
sealed trait TestEvent
case object OnComplete extends TestEvent
case object Cancel extends TestEvent
case class OnError(cause: Throwable) extends TestEvent
case class OnNext(elem: Any) extends TestEvent
case object RequestOne extends TestEvent
case object RequestAnother extends TestEvent
private def run() = interpreter.execute(Int.MaxValue)
private def initialize(): Unit = {
import GraphInterpreter.Boundary
var i = 0
val attributes = Array.fill[Attributes](ops.length)(Attributes.none)
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[PushPullGraphStage[_, _, _]]
ins(i) = stage.shape.inlet
inOwners(i) = i
outs(i + 1) = stage.shape.outlet
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()
}
initialize()
run() // Detached stages need the prefetch
def lastEvents(): Set[TestEvent] = {
val events = lastEvent
lastEvent = Set.empty
events
}
class UpstreamOneBoundedProbe[T] extends UpstreamBoundaryStageLogic[T] {
val out = Outlet[T]("out")
out.id = 0
setHandler(out, new OutHandler {
override def onPull(): Unit = {
if (lastEvent.contains(RequestOne)) lastEvent += RequestAnother
else lastEvent += RequestOne
}
override def onDownstreamFinish(): Unit = lastEvent += Cancel
})
def onNext(elem: T): Unit = {
push(out, elem)
run()
}
def onComplete(): Unit = {
complete(out)
run()
}
def onNextAndComplete(elem: T): Unit = {
push(out, elem)
complete(out)
run()
}
def onError(ex: Throwable): Unit = {
fail(out, ex)
run()
}
}
class DownstreamOneBoundedPortProbe[T] extends DownstreamBoundaryStageLogic[T] {
val in = Inlet[T]("in")
in.id = 0
setHandler(in, new InHandler {
// Modified onPush that does not grab() automatically the element. This accesses some internals.
override def onPush(): Unit = {
lastEvent += OnNext(grab(in))
}
override def onUpstreamFinish() = lastEvent += OnComplete
override def onUpstreamFailure(ex: Throwable) = lastEvent += OnError(ex)
})
def requestOne(): Unit = {
pull(in)
run()
}
def cancel(): Unit = {
cancel(in)
run()
}
}
}
}