#19197 implement SubFusingActorMaterializer

and use it in FlattenMerge
This commit is contained in:
Roland Kuhn 2015-12-17 13:35:37 +01:00
parent a95a5b3af8
commit af99b1eae8
10 changed files with 290 additions and 156 deletions

View file

@ -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)
}
}
}
}

View file

@ -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)

View file

@ -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)
}

View file

@ -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))
})