=str: Optmizing GraphInterpreter
This commit is contained in:
parent
e7a14e53a9
commit
f4b614a186
19 changed files with 996 additions and 273 deletions
|
|
@ -3,11 +3,14 @@
|
|||
*/
|
||||
package akka.stream.impl.fusing
|
||||
|
||||
import akka.stream.{ NoMaterializer, Outlet, Inlet, Shape }
|
||||
import akka.stream.impl.fusing.GraphInterpreter.{ GraphAssembly, DownstreamBoundaryStageLogic, UpstreamBoundaryStageLogic }
|
||||
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.testkit.AkkaSpec
|
||||
import akka.stream.testkit.Utils.TE
|
||||
|
||||
trait GraphInterpreterSpecKit {
|
||||
trait GraphInterpreterSpecKit extends AkkaSpec {
|
||||
|
||||
sealed trait TestEvent {
|
||||
def source: GraphStageLogic
|
||||
|
|
@ -20,11 +23,17 @@ trait GraphInterpreterSpecKit {
|
|||
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
|
||||
private var _interpreter: GraphInterpreter = _
|
||||
protected def interpreter: GraphInterpreter = _interpreter
|
||||
|
||||
def stepAll(): Unit = interpreter.execute(eventLimit = Int.MaxValue)
|
||||
def step(): Unit = interpreter.execute(eventLimit = 1)
|
||||
|
||||
class AssemblyBuilder(stages: Seq[GraphStage[_ <: Shape]]) {
|
||||
var upstreams = Vector.empty[(UpstreamBoundaryStageLogic[_], Inlet[_])]
|
||||
var downstreams = Vector.empty[(Outlet[_], DownstreamBoundaryStageLogic[_])]
|
||||
|
|
@ -59,7 +68,7 @@ trait GraphInterpreterSpecKit {
|
|||
(Vector.fill(upstreams.size)(-1) ++ outOwners).toArray)
|
||||
|
||||
val (inHandlers, outHandlers, logics, _) = assembly.materialize()
|
||||
_interpreter = new GraphInterpreter(assembly, NoMaterializer, inHandlers, outHandlers, logics, (_, _, _) ⇒ ())
|
||||
_interpreter = new GraphInterpreter(assembly, NoMaterializer, Logging(system, classOf[TestSetup]), inHandlers, outHandlers, logics, (_, _, _) ⇒ ())
|
||||
|
||||
for ((upstream, i) ← upstreams.zipWithIndex) {
|
||||
_interpreter.attachUpstreamBoundary(i, upstream._1)
|
||||
|
|
@ -75,7 +84,7 @@ trait GraphInterpreterSpecKit {
|
|||
|
||||
def manualInit(assembly: GraphAssembly): Unit = {
|
||||
val (inHandlers, outHandlers, logics, _) = assembly.materialize()
|
||||
_interpreter = new GraphInterpreter(assembly, NoMaterializer, inHandlers, outHandlers, logics, (_, _, _) ⇒ ())
|
||||
_interpreter = new GraphInterpreter(assembly, NoMaterializer, Logging(system, classOf[TestSetup]), inHandlers, outHandlers, logics, (_, _, _) ⇒ ())
|
||||
}
|
||||
|
||||
def builder(stages: GraphStage[_ <: Shape]*): AssemblyBuilder = new AssemblyBuilder(stages.toSeq)
|
||||
|
|
@ -90,10 +99,11 @@ trait GraphInterpreterSpecKit {
|
|||
|
||||
class UpstreamProbe[T](override val toString: String) extends UpstreamBoundaryStageLogic[T] {
|
||||
val out = Outlet[T]("out")
|
||||
out.id = 0
|
||||
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = lastEvent += RequestOne(UpstreamProbe.this)
|
||||
override def onDownstreamFinish() = lastEvent += Cancel(UpstreamProbe.this)
|
||||
override def onDownstreamFinish(): Unit = lastEvent += Cancel(UpstreamProbe.this)
|
||||
})
|
||||
|
||||
def onNext(elem: T, eventLimit: Int = Int.MaxValue): Unit = {
|
||||
|
|
@ -105,11 +115,12 @@ trait GraphInterpreterSpecKit {
|
|||
|
||||
class DownstreamProbe[T](override val toString: String) extends DownstreamBoundaryStageLogic[T] {
|
||||
val in = Inlet[T]("in")
|
||||
in.id = 0
|
||||
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit = lastEvent += OnNext(DownstreamProbe.this, grab(in))
|
||||
override def onUpstreamFinish() = lastEvent += OnComplete(DownstreamProbe.this)
|
||||
override def onUpstreamFailure(ex: Throwable) = OnError(DownstreamProbe.this, ex)
|
||||
override def onUpstreamFinish(): Unit = lastEvent += OnComplete(DownstreamProbe.this)
|
||||
override def onUpstreamFailure(ex: Throwable): Unit = lastEvent += OnError(DownstreamProbe.this, ex)
|
||||
})
|
||||
|
||||
def requestOne(eventLimit: Int = Int.MaxValue): Unit = {
|
||||
|
|
@ -146,20 +157,20 @@ trait GraphInterpreterSpecKit {
|
|||
setHandler(in, new InHandler {
|
||||
|
||||
// Modified onPush that does not grab() automatically the element. This accesses some internals.
|
||||
override def onPush(): Unit =
|
||||
lastEvent +=
|
||||
OnNext(
|
||||
DownstreamPortProbe.this,
|
||||
interpreter.connectionStates(inToConn(in)))
|
||||
override def onPush(): Unit = {
|
||||
val internalEvent = interpreter.connectionSlots(inToConn(in.id))
|
||||
|
||||
internalEvent match {
|
||||
case Failed(_, elem) ⇒ lastEvent += OnNext(DownstreamPortProbe.this, elem)
|
||||
case elem ⇒ lastEvent += OnNext(DownstreamPortProbe.this, elem)
|
||||
}
|
||||
}
|
||||
|
||||
override def onUpstreamFinish() = lastEvent += OnComplete(DownstreamPortProbe.this)
|
||||
override def onUpstreamFailure(ex: Throwable) = OnError(DownstreamPortProbe.this, ex)
|
||||
override def onUpstreamFailure(ex: Throwable) = lastEvent += OnError(DownstreamPortProbe.this, ex)
|
||||
})
|
||||
}
|
||||
|
||||
def stepAll(): Unit = interpreter.execute(eventLimit = Int.MaxValue)
|
||||
def step(): Unit = interpreter.execute(eventLimit = 1)
|
||||
|
||||
private val assembly = GraphAssembly(
|
||||
stages = Array.empty,
|
||||
ins = Array(null),
|
||||
|
|
@ -173,4 +184,73 @@ trait GraphInterpreterSpecKit {
|
|||
interpreter.init()
|
||||
}
|
||||
|
||||
abstract class FailingStageSetup(initFailOnNextEvent: Boolean = false) extends TestSetup {
|
||||
|
||||
val upstream = new UpstreamPortProbe[Int]
|
||||
val downstream = new DownstreamPortProbe[Int]
|
||||
|
||||
private var _failOnNextEvent: Boolean = initFailOnNextEvent
|
||||
private var _failOnPostStop: Boolean = false
|
||||
|
||||
def failOnNextEvent(): Unit = _failOnNextEvent = true
|
||||
def failOnPostStop(): Unit = _failOnPostStop = true
|
||||
|
||||
def testException = TE("test")
|
||||
|
||||
private val stagein = Inlet[Int]("sandwitch.in")
|
||||
private val stageout = Outlet[Int]("sandwitch.out")
|
||||
private val stageshape = FlowShape(stagein, stageout)
|
||||
|
||||
// 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) {
|
||||
private def mayFail(task: ⇒ Unit): Unit = {
|
||||
if (!_failOnNextEvent) task
|
||||
else {
|
||||
_failOnNextEvent = false
|
||||
throw testException
|
||||
}
|
||||
}
|
||||
|
||||
setHandler(stagein, new InHandler {
|
||||
override def onPush(): Unit = mayFail(push(stageout, grab(stagein)))
|
||||
override def onUpstreamFinish(): Unit = mayFail(completeStage())
|
||||
override def onUpstreamFailure(ex: Throwable): Unit = mayFail(failStage(ex))
|
||||
})
|
||||
|
||||
setHandler(stageout, new OutHandler {
|
||||
override def onPull(): Unit = mayFail(pull(stagein))
|
||||
override def onDownstreamFinish(): Unit = mayFail(completeStage())
|
||||
})
|
||||
|
||||
override def preStart(): Unit = mayFail(lastEvent += PreStart(stage))
|
||||
override def postStop(): Unit =
|
||||
if (!_failOnPostStop) lastEvent += PostStop(stage)
|
||||
else throw testException
|
||||
|
||||
override def toString = "stage"
|
||||
}
|
||||
|
||||
private val sandwitchStage = new GraphStage[FlowShape[Int, Int]] {
|
||||
override def shape = stageshape
|
||||
override def createLogic: GraphStageLogic = stage
|
||||
}
|
||||
|
||||
class UpstreamPortProbe[T] extends UpstreamProbe[T]("upstreamPort") {
|
||||
def push(elem: T): Unit = push(out, elem)
|
||||
def complete(): Unit = complete(out)
|
||||
def fail(ex: Throwable): Unit = fail(out, ex)
|
||||
}
|
||||
|
||||
class DownstreamPortProbe[T] extends DownstreamProbe[T]("upstreamPort") {
|
||||
def pull(): Unit = pull(in)
|
||||
def cancel(): Unit = cancel(in)
|
||||
}
|
||||
|
||||
builder(sandwitchStage)
|
||||
.connect(upstream, stagein)
|
||||
.connect(stageout, downstream)
|
||||
.init()
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue