pekko/akka-stream-tests/src/test/scala/akka/stream/impl/ActorInterpreterSpec.scala
Roland Kuhn d462cdd1b4 =str fix sub-upstream cancellation in concatAll
- ActorProcessor terminated eagerly when ConcatAll had just taken up a
  new input stream but not yet received onSubscribe for it

- The ActorProcessor eagerly shuts itself down upon onError and that
  cannot be changed without completely reworking the Pump, so I opted
  for just tracking the outstanding substreamSubscribers that have not
  yet seen OnSubscribe and making them cancel properly when that arrives
  (possibly later).
2015-06-19 16:26:48 +02:00

166 lines
5.4 KiB
Scala

/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.impl
import akka.stream.Supervision._
import akka.stream.testkit.AkkaSpec
import akka.stream._
import akka.stream.scaladsl._
import akka.stream.testkit._
import akka.stream.impl.fusing.ActorInterpreter
import akka.stream.stage.Stage
import akka.stream.stage.PushPullStage
import akka.stream.stage.Context
import akka.testkit.TestLatch
import scala.concurrent.Await
import scala.concurrent.duration._
class ActorInterpreterSpec extends AkkaSpec {
import FlowGraph.Implicits._
implicit val mat = ActorFlowMaterializer()
class Setup(ops: List[Stage[_, _]] = List(fusing.Map({ x: Any x }, stoppingDecider))) {
val up = TestPublisher.manualProbe[Int]()
val down = TestSubscriber.manualProbe[Int]
private val props = ActorInterpreter.props(mat.settings, ops, mat).withDispatcher("akka.test.stream-dispatcher")
val actor = system.actorOf(props)
val processor = ActorProcessorFactory[Int, Int](actor)
}
"An ActorInterpreter" must {
"pass along early cancellation" in new Setup {
processor.subscribe(down)
val sub = down.expectSubscription()
sub.cancel()
up.subscribe(processor)
val upsub = up.expectSubscription()
upsub.expectCancellation()
}
"heed cancellation signal while large demand is outstanding" in {
val latch = TestLatch()
val infinite = new PushPullStage[Int, Int] {
override def onPush(elem: Int, ctx: Context[Int]) = ???
override def onPull(ctx: Context[Int]) = {
Await.ready(latch, 5.seconds)
ctx.push(42)
}
}
val N = system.settings.config.getInt("akka.stream.materializer.output-burst-limit")
new Setup(infinite :: Nil) {
processor.subscribe(down)
val sub = down.expectSubscription()
up.subscribe(processor)
val upsub = up.expectSubscription()
sub.request(100000000)
sub.cancel()
watch(actor)
latch.countDown()
for (i 1 to N) withClue(s"iteration $i: ") {
try down.expectNext(42) catch { case e: Throwable fail(e) }
}
// now cancellation request is processed
down.expectNoMsg(500.millis)
upsub.expectCancellation()
expectTerminated(actor)
}
}
"heed upstream failure while large demand is outstanding" in {
val latch = TestLatch()
val infinite = new PushPullStage[Int, Int] {
override def onPush(elem: Int, ctx: Context[Int]) = ???
override def onPull(ctx: Context[Int]) = {
Await.ready(latch, 5.seconds)
ctx.push(42)
}
}
val N = system.settings.config.getInt("akka.stream.materializer.output-burst-limit")
new Setup(infinite :: Nil) {
processor.subscribe(down)
val sub = down.expectSubscription()
up.subscribe(processor)
val upsub = up.expectSubscription()
sub.request(100000000)
val ex = new Exception("FAIL!")
upsub.sendError(ex)
latch.countDown()
for (i 1 to N) withClue(s"iteration $i: ") {
try down.expectNext(42) catch { case e: Throwable fail(e) }
}
down.expectError(ex)
}
}
"hold back upstream completion while large demand is outstanding" in {
val latch = TestLatch()
val N = 3 * system.settings.config.getInt("akka.stream.materializer.output-burst-limit")
val infinite = new PushPullStage[Int, Int] {
private var remaining = N
override def onPush(elem: Int, ctx: Context[Int]) = ???
override def onPull(ctx: Context[Int]) = {
Await.ready(latch, 5.seconds)
remaining -= 1
if (remaining >= 0) ctx.push(42)
else ctx.finish()
}
override def onUpstreamFinish(ctx: Context[Int]) = {
if (remaining > 0) ctx.absorbTermination()
else ctx.finish()
}
}
new Setup(infinite :: Nil) {
processor.subscribe(down)
val sub = down.expectSubscription()
up.subscribe(processor)
val upsub = up.expectSubscription()
sub.request(100000000)
upsub.sendComplete()
latch.countDown()
for (i 1 to N) withClue(s"iteration $i: ") {
try down.expectNext(42) catch { case e: Throwable fail(e) }
}
down.expectComplete()
}
}
"satisfy large demand" in largeDemand(0)
"satisfy larger demand" in largeDemand(1)
def largeDemand(extra: Int): Unit = {
val N = 3 * system.settings.config.getInt("akka.stream.materializer.output-burst-limit")
val large = new PushPullStage[Int, Int] {
private var remaining = N
override def onPush(elem: Int, ctx: Context[Int]) = ???
override def onPull(ctx: Context[Int]) = {
remaining -= 1
if (remaining >= 0) ctx.push(42)
else ctx.finish()
}
}
new Setup(large :: Nil) {
processor.subscribe(down)
val sub = down.expectSubscription()
up.subscribe(processor)
val upsub = up.expectSubscription()
sub.request(100000000)
watch(actor)
for (i 1 to N) withClue(s"iteration $i: ") {
try down.expectNext(42) catch { case e: Throwable fail(e) }
}
down.expectComplete()
upsub.expectCancellation()
expectTerminated(actor)
}
}
}
}