Low level interpreter tests working without GraphAssembly #22423
This commit is contained in:
parent
44260fe5d3
commit
c028b550f2
26 changed files with 354 additions and 575 deletions
|
|
@ -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: _*)
|
||||
}
|
||||
*/
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue