parent
a95a5b3af8
commit
af99b1eae8
10 changed files with 290 additions and 156 deletions
|
|
@ -11,11 +11,16 @@ import akka.stream.Attributes._
|
|||
import akka.stream.Fusing.FusedGraph
|
||||
import scala.annotation.tailrec
|
||||
import akka.stream.impl.StreamLayout.Module
|
||||
import org.scalatest.concurrent.ScalaFutures
|
||||
import scala.concurrent.duration._
|
||||
import akka.stream.impl.fusing.GraphInterpreter
|
||||
import akka.event.BusLogging
|
||||
|
||||
class FusingSpec extends AkkaSpec with ConversionCheckedTripleEquals {
|
||||
class FusingSpec extends AkkaSpec with ScalaFutures with ConversionCheckedTripleEquals {
|
||||
|
||||
final val Debug = false
|
||||
implicit val materializer = ActorMaterializer()
|
||||
implicit val patience = PatienceConfig(1.second)
|
||||
|
||||
def graph(async: Boolean) =
|
||||
Source.unfoldInf(1)(x ⇒ (x, x)).filter(_ % 2 == 1)
|
||||
|
|
@ -87,4 +92,38 @@ class FusingSpec extends AkkaSpec with ConversionCheckedTripleEquals {
|
|||
|
||||
}
|
||||
|
||||
"SubFusingActorMaterializer" must {
|
||||
|
||||
"work with asynchronous boundaries in the subflows" in {
|
||||
val async = Flow[Int].map(_ * 2).withAttributes(Attributes.asyncBoundary)
|
||||
Source(0 to 9)
|
||||
.map(_ * 10)
|
||||
.flatMapMerge(5, i ⇒ Source(i to (i + 9)).via(async))
|
||||
.grouped(1000)
|
||||
.runWith(Sink.head)
|
||||
.futureValue
|
||||
.sorted should ===(0 to 198 by 2)
|
||||
}
|
||||
|
||||
"use multiple actors when there are asynchronous boundaries in the subflows" in {
|
||||
def ref = {
|
||||
val bus = GraphInterpreter.currentInterpreter.log.asInstanceOf[BusLogging]
|
||||
bus.logSource
|
||||
}
|
||||
val async = Flow[Int].map(x ⇒ { testActor ! ref; x }).withAttributes(Attributes.asyncBoundary)
|
||||
Source(0 to 9)
|
||||
.map(x ⇒ { testActor ! ref; x })
|
||||
.flatMapMerge(5, i ⇒ Source.single(i).via(async))
|
||||
.grouped(1000)
|
||||
.runWith(Sink.head)
|
||||
.futureValue
|
||||
.sorted should ===(0 to 9)
|
||||
val refs = receiveN(20)
|
||||
withClue(s"refs=\n${refs.mkString("\n")}") {
|
||||
refs.toSet.size should ===(11)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -56,7 +56,7 @@ class GraphInterpreterSpec extends AkkaSpec with GraphInterpreterSpecKit {
|
|||
manualInit(assembly)
|
||||
interpreter.attachDownstreamBoundary(2, sink)
|
||||
interpreter.attachUpstreamBoundary(0, source)
|
||||
interpreter.init()
|
||||
interpreter.init(null)
|
||||
|
||||
lastEvents() should ===(Set.empty)
|
||||
|
||||
|
|
|
|||
|
|
@ -84,7 +84,7 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
|
|||
_interpreter.attachDownstreamBoundary(i + upstreams.size + connections.size, downstream._2)
|
||||
}
|
||||
|
||||
_interpreter.init()
|
||||
_interpreter.init(null)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -226,7 +226,7 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
|
|||
manualInit(assembly)
|
||||
interpreter.attachDownstreamBoundary(0, in)
|
||||
interpreter.attachUpstreamBoundary(0, out)
|
||||
interpreter.init()
|
||||
interpreter.init(null)
|
||||
}
|
||||
|
||||
abstract class FailingStageSetup(initFailOnNextEvent: Boolean = false) extends TestSetup {
|
||||
|
|
@ -351,7 +351,7 @@ trait GraphInterpreterSpecKit extends AkkaSpec {
|
|||
interpreter.attachUpstreamBoundary(0, upstream)
|
||||
interpreter.attachDownstreamBoundary(ops.length, downstream)
|
||||
|
||||
interpreter.init()
|
||||
interpreter.init(null)
|
||||
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -22,6 +22,7 @@ import scala.collection.immutable
|
|||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.control.NoStackTrace
|
||||
import akka.stream.impl.fusing.GraphInterpreterShell
|
||||
|
||||
object FlowSpec {
|
||||
class Fruit
|
||||
|
|
@ -41,22 +42,14 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
|||
val identity: Flow[Any, Any, Unit] ⇒ Flow[Any, Any, Unit] = in ⇒ in.map(e ⇒ e)
|
||||
val identity2: Flow[Any, Any, Unit] ⇒ Flow[Any, Any, Unit] = in ⇒ identity(in)
|
||||
|
||||
class BrokenActorInterpreter(
|
||||
_assembly: GraphAssembly,
|
||||
_inHandlers: Array[InHandler],
|
||||
_outHandlers: Array[OutHandler],
|
||||
_logics: Array[GraphStageLogic],
|
||||
_shape: Shape,
|
||||
_settings: ActorMaterializerSettings,
|
||||
_materializer: Materializer,
|
||||
brokenMessage: Any)
|
||||
extends ActorGraphInterpreter(_assembly, _inHandlers, _outHandlers, _logics, _shape, _settings, _materializer) {
|
||||
class BrokenActorInterpreter(_shell: GraphInterpreterShell, brokenMessage: Any)
|
||||
extends ActorGraphInterpreter(_shell) {
|
||||
|
||||
import akka.stream.actor.ActorSubscriberMessage._
|
||||
|
||||
override protected[akka] def aroundReceive(receive: Receive, msg: Any) = {
|
||||
msg match {
|
||||
case ActorGraphInterpreter.OnNext(0, m) if m == brokenMessage ⇒
|
||||
case ActorGraphInterpreter.OnNext(_, 0, m) if m == brokenMessage ⇒
|
||||
throw new NullPointerException(s"I'm so broken [$m]")
|
||||
case _ ⇒ super.aroundReceive(receive, msg)
|
||||
}
|
||||
|
|
@ -77,14 +70,17 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
|||
val (inHandlers, outHandlers, logics) =
|
||||
assembly.materialize(Attributes.none, assembly.stages.map(_.module), new java.util.HashMap, _ ⇒ ())
|
||||
|
||||
val props = Props(new BrokenActorInterpreter(assembly, inHandlers, outHandlers, logics, stage.shape, settings, materializer, "a3"))
|
||||
val shell = new GraphInterpreterShell(assembly, inHandlers, outHandlers, logics, stage.shape, settings,
|
||||
materializer.asInstanceOf[ActorMaterializerImpl])
|
||||
|
||||
val props = Props(new BrokenActorInterpreter(shell, "a3"))
|
||||
.withDispatcher("akka.test.stream-dispatcher").withDeploy(Deploy.local)
|
||||
val impl = system.actorOf(props, "borken-stage-actor")
|
||||
|
||||
val subscriber = new ActorGraphInterpreter.BoundarySubscriber(impl, 0)
|
||||
val publisher = new ActorPublisher[Any](impl) { override val wakeUpMsg = ActorGraphInterpreter.SubscribePending(0) }
|
||||
val subscriber = new ActorGraphInterpreter.BoundarySubscriber(impl, shell, 0)
|
||||
val publisher = new ActorPublisher[Any](impl) { override val wakeUpMsg = ActorGraphInterpreter.SubscribePending(shell, 0) }
|
||||
|
||||
impl ! ActorGraphInterpreter.ExposedPublisher(0, publisher)
|
||||
impl ! ActorGraphInterpreter.ExposedPublisher(shell, 0, publisher)
|
||||
|
||||
Flow.fromSinkAndSource(Sink(subscriber), Source(publisher))
|
||||
})
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue