fix FlowStageSpec (and some boyscouting)

This commit is contained in:
Roland Kuhn 2015-06-19 15:05:05 +02:00
parent d462cdd1b4
commit 097a10edfc
2 changed files with 63 additions and 78 deletions

View file

@ -315,6 +315,12 @@ object TestSubscriber {
def receiveWhile[T](max: Duration = Duration.Undefined, idle: Duration = Duration.Inf, messages: Int = Int.MaxValue)(f: PartialFunction[SubscriberEvent, T]): immutable.Seq[T] = def receiveWhile[T](max: Duration = Duration.Undefined, idle: Duration = Duration.Inf, messages: Int = Int.MaxValue)(f: PartialFunction[SubscriberEvent, T]): immutable.Seq[T] =
probe.receiveWhile(max, idle, messages)(f.asInstanceOf[PartialFunction[AnyRef, T]]) probe.receiveWhile(max, idle, messages)(f.asInstanceOf[PartialFunction[AnyRef, T]])
def receiveWithin(max: FiniteDuration, messages: Int = Int.MaxValue): immutable.Seq[I] =
probe.receiveWhile(max, max, messages) {
case OnNext(i) Some(i.asInstanceOf[I])
case _ None
}.flatten
def within[T](max: FiniteDuration)(f: T): T = probe.within(0.seconds, max)(f) def within[T](max: FiniteDuration)(f: T): T = probe.within(0.seconds, max)(f)
def onSubscribe(subscription: Subscription): Unit = probe.ref ! OnSubscribe(subscription) def onSubscribe(subscription: Subscription): Unit = probe.ref ! OnSubscribe(subscription)

View file

@ -15,9 +15,10 @@ import akka.stream.testkit._
import akka.stream.testkit.Utils._ import akka.stream.testkit.Utils._
import akka.testkit.{ EventFilter, TestProbe } import akka.testkit.{ EventFilter, TestProbe }
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.stream.testkit.scaladsl.TestSink
import akka.stream.testkit.scaladsl.TestSource
class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.receive=off\nakka.loglevel=INFO")) { class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.receive=off\nakka.loglevel=INFO")) {
@ -222,8 +223,7 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
} }
"allow early finish" in assertAllStagesStopped { "allow early finish" in assertAllStagesStopped {
val p = TestPublisher.manualProbe[Int]() val (p1, p2) = TestSource.probe[Int].
val p2 = Source(p).
transform(() new PushStage[Int, Int] { transform(() new PushStage[Int, Int] {
var s = "" var s = ""
override def onPush(element: Int, ctx: Context[Int]) = { override def onPush(element: Int, ctx: Context[Int]) = {
@ -233,18 +233,14 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
else else
ctx.push(element) ctx.push(element)
} }
}). })
runWith(Sink.publisher) .toMat(TestSink.probe[Int])(Keep.both).run
val proc = p.expectSubscription p2.request(10)
val c = TestSubscriber.manualProbe[Int]() p1.sendNext(1)
p2.subscribe(c) .sendNext(2)
val s = c.expectSubscription() p2.expectNext(1)
s.request(10) .expectComplete()
proc.sendNext(1) p1.expectCancellation()
proc.sendNext(2)
c.expectNext(1)
c.expectComplete()
proc.expectCancellation()
} }
"report error when exception is thrown" in assertAllStagesStopped { "report error when exception is thrown" in assertAllStagesStopped {
@ -261,16 +257,13 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
} }
} }
}). }).
runWith(Sink.publisher) runWith(TestSink.probe[Int])
val subscriber = TestSubscriber.manualProbe[Int]()
p2.subscribe(subscriber)
val subscription = subscriber.expectSubscription()
EventFilter[IllegalArgumentException]("two not allowed") intercept { EventFilter[IllegalArgumentException]("two not allowed") intercept {
subscription.request(100) p2.request(100)
subscriber.expectNext(1) .expectNext(1)
subscriber.expectNext(1) .expectNext(1)
subscriber.expectError().getMessage should be("two not allowed") .expectError().getMessage should be("two not allowed")
subscriber.expectNoMsg(200.millis) p2.expectNoMsg(200.millis)
} }
} }
@ -288,65 +281,56 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
} }
}). }).
filter(elem elem != 1). // it's undefined if element 1 got through before the error or not filter(elem elem != 1). // it's undefined if element 1 got through before the error or not
runWith(Sink.publisher) runWith(TestSink.probe[Int])
val subscriber = TestSubscriber.manualProbe[Int]()
p2.subscribe(subscriber)
val subscription = subscriber.expectSubscription()
EventFilter[IllegalArgumentException]("two not allowed") intercept { EventFilter[IllegalArgumentException]("two not allowed") intercept {
subscription.request(100) p2.request(100)
subscriber.expectNext(100) .expectNext(100)
subscriber.expectNext(101) .expectNext(101)
subscriber.expectComplete() .expectComplete()
subscriber.expectNoMsg(200.millis) .expectNoMsg(200.millis)
} }
} }
"support cancel as expected" in assertAllStagesStopped { "support cancel as expected" in assertAllStagesStopped {
val p = Source(List(1, 2, 3)).runWith(Sink.publisher) val p = Source(1 to 100).runWith(Sink.publisher)
val p2 = Source(p). val received = Source(p).
transform(() new StatefulStage[Int, Int] { transform(() new StatefulStage[Int, Int] {
override def initial = new State { override def initial = new State {
override def onPush(elem: Int, ctx: Context[Int]) = override def onPush(elem: Int, ctx: Context[Int]) =
emit(Iterator(elem, elem), ctx) emit(Iterator(elem, elem), ctx)
} }
}). })
runWith(Sink.publisher) .runWith(TestSink.probe[Int]())
val subscriber = TestSubscriber.manualProbe[Int]() .request(1000)
p2.subscribe(subscriber) .expectNext(1)
val subscription = subscriber.expectSubscription() .cancel()
subscription.request(2) .receiveWithin(1.second)
subscriber.expectNext(1) received.size should be < 200
subscription.cancel() received.foldLeft((true, 1)) {
subscriber.expectNext(1) case ((flag, last), next) (flag && (last == next || last == next - 1), next)
subscriber.expectNoMsg(500.millis) }._1 should be(true)
subscription.request(2)
subscriber.expectNoMsg(200.millis)
} }
"support producing elements from empty inputs" in assertAllStagesStopped { "support producing elements from empty inputs" in assertAllStagesStopped {
val p = Source(List.empty[Int]).runWith(Sink.publisher) val p = Source(List.empty[Int]).runWith(Sink.publisher)
val p2 = Source(p). Source(p).
transform(() new StatefulStage[Int, Int] { transform(() new StatefulStage[Int, Int] {
override def initial = new State { override def initial = new State {
override def onPush(elem: Int, ctx: Context[Int]) = ctx.pull() override def onPush(elem: Int, ctx: Context[Int]) = ctx.pull()
} }
override def onUpstreamFinish(ctx: Context[Int]) = override def onUpstreamFinish(ctx: Context[Int]) =
terminationEmit(Iterator(1, 2, 3), ctx) terminationEmit(Iterator(1, 2, 3), ctx)
}). })
runWith(Sink.publisher) .runWith(TestSink.probe[Int])
val subscriber = TestSubscriber.manualProbe[Int]() .request(4)
p2.subscribe(subscriber) .expectNext(1)
val subscription = subscriber.expectSubscription() .expectNext(2)
subscription.request(4) .expectNext(3)
subscriber.expectNext(1) .expectComplete()
subscriber.expectNext(2)
subscriber.expectNext(3)
subscriber.expectComplete()
} }
"support converting onComplete into onError" in { "support converting onComplete into onError" in {
val subscriber = TestSubscriber.manualProbe[Int]()
Source(List(5, 1, 2, 3)).transform(() new PushStage[Int, Int] { Source(List(5, 1, 2, 3)).transform(() new PushStage[Int, Int] {
var expectedNumberOfElements: Option[Int] = None var expectedNumberOfElements: Option[Int] = None
var count = 0 var count = 0
@ -365,15 +349,12 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
throw new RuntimeException(s"Expected $expected, got $count") with NoStackTrace throw new RuntimeException(s"Expected $expected, got $count") with NoStackTrace
case _ ctx.finish() case _ ctx.finish()
} }
}).to(Sink(subscriber)).run() }).runWith(TestSink.probe[Int])
.request(10)
val subscription = subscriber.expectSubscription() .expectNext(1)
subscription.request(10) .expectNext(2)
.expectNext(3)
subscriber.expectNext(1) .expectError().getMessage should be("Expected 5, got 3")
subscriber.expectNext(2)
subscriber.expectNext(3)
subscriber.expectError().getMessage should be("Expected 5, got 3")
} }
"be safe to reuse" in { "be safe to reuse" in {
@ -387,17 +368,15 @@ class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug
} }
}) })
val s1 = TestSubscriber.manualProbe[Int]() flow.runWith(TestSink.probe[Int])
flow.to(Sink(s1)).run() .request(3)
s1.expectSubscription().request(3) .expectNext(1, 2, 3)
s1.expectNext(1, 2, 3) .expectComplete()
s1.expectComplete()
val s2 = TestSubscriber.manualProbe[Int]() flow.runWith(TestSink.probe[Int])
flow.to(Sink(s2)).run() .request(3)
s2.expectSubscription().request(3) .expectNext(1, 2, 3)
s2.expectNext(1, 2, 3) .expectComplete()
s2.expectComplete()
} }
"handle early cancelation" in assertAllStagesStopped { "handle early cancelation" in assertAllStagesStopped {