!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
|
|
@ -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()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue