fix FlowStageSpec (and some boyscouting)
This commit is contained in:
parent
d462cdd1b4
commit
097a10edfc
2 changed files with 63 additions and 78 deletions
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue