2015-04-24 11:45:03 +03:00
|
|
|
/**
|
|
|
|
|
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
|
|
|
|
*/
|
|
|
|
|
package akka.stream.testkit
|
|
|
|
|
|
2015-09-17 12:57:07 +02:00
|
|
|
import akka.actor.{ ActorSystem, DeadLetterSuppression, NoSerializationVerificationNeeded }
|
2015-04-24 11:45:03 +03:00
|
|
|
import akka.stream._
|
2015-09-17 12:57:07 +02:00
|
|
|
import akka.stream.impl.StreamLayout.Module
|
2015-04-24 11:45:03 +03:00
|
|
|
import akka.stream.impl._
|
2015-04-24 11:07:26 +02:00
|
|
|
import akka.testkit.TestProbe
|
2015-04-24 11:45:03 +03:00
|
|
|
import org.reactivestreams.{ Publisher, Subscriber, Subscription }
|
2015-09-17 12:57:07 +02:00
|
|
|
import scala.annotation.tailrec
|
2015-04-24 11:45:03 +03:00
|
|
|
import scala.collection.immutable
|
|
|
|
|
import scala.concurrent.duration._
|
2015-09-17 12:57:07 +02:00
|
|
|
import scala.language.existentials
|
2015-10-31 14:46:10 +01:00
|
|
|
import java.io.StringWriter
|
|
|
|
|
import java.io.PrintWriter
|
2015-04-24 11:45:03 +03:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Provides factory methods for various Publishers.
|
|
|
|
|
*/
|
|
|
|
|
object TestPublisher {
|
|
|
|
|
|
|
|
|
|
import StreamTestKit._
|
|
|
|
|
|
2015-09-09 22:17:51 -04:00
|
|
|
trait PublisherEvent extends DeadLetterSuppression with NoSerializationVerificationNeeded
|
|
|
|
|
final case class Subscribe(subscription: Subscription) extends PublisherEvent
|
|
|
|
|
final case class CancelSubscription(subscription: Subscription) extends PublisherEvent
|
|
|
|
|
final case class RequestMore(subscription: Subscription, elements: Long) extends PublisherEvent
|
|
|
|
|
|
2015-04-24 11:45:03 +03:00
|
|
|
/**
|
|
|
|
|
* Publisher that signals complete to subscribers, after handing a void subscription.
|
|
|
|
|
*/
|
|
|
|
|
def empty[T](): Publisher[T] = EmptyPublisher[T]
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Publisher that subscribes the subscriber and completes after the first request.
|
|
|
|
|
*/
|
|
|
|
|
def lazyEmpty[T]: Publisher[T] = new Publisher[T] {
|
|
|
|
|
override def subscribe(subscriber: Subscriber[_ >: T]): Unit =
|
|
|
|
|
subscriber.onSubscribe(CompletedSubscription(subscriber))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Publisher that signals error to subscribers immediately, before handing out subscription.
|
|
|
|
|
*/
|
|
|
|
|
def error[T](cause: Throwable): Publisher[T] = ErrorPublisher(cause, "error").asInstanceOf[Publisher[T]]
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Publisher that subscribes the subscriber and signals error after the first request.
|
|
|
|
|
*/
|
|
|
|
|
def lazyError[T](cause: Throwable): Publisher[T] = new Publisher[T] {
|
|
|
|
|
override def subscribe(subscriber: Subscriber[_ >: T]): Unit =
|
|
|
|
|
subscriber.onSubscribe(FailedSubscription(subscriber, cause))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Probe that implements [[org.reactivestreams.Publisher]] interface.
|
|
|
|
|
*/
|
2015-06-18 22:24:24 +02:00
|
|
|
def manualProbe[T](autoOnSubscribe: Boolean = true)(implicit system: ActorSystem): ManualProbe[T] = new ManualProbe(autoOnSubscribe)
|
2015-04-24 11:45:03 +03:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Probe that implements [[org.reactivestreams.Publisher]] interface and tracks demand.
|
|
|
|
|
*/
|
|
|
|
|
def probe[T](initialPendingRequests: Long = 0)(implicit system: ActorSystem): Probe[T] = new Probe(initialPendingRequests)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Implementation of [[org.reactivestreams.Publisher]] that allows various assertions.
|
|
|
|
|
* This probe does not track demand. Therefore you need to expect demand before sending
|
|
|
|
|
* elements downstream.
|
|
|
|
|
*/
|
2015-06-18 22:24:24 +02:00
|
|
|
class ManualProbe[I] private[TestPublisher] (autoOnSubscribe: Boolean = true)(implicit system: ActorSystem) extends Publisher[I] {
|
2015-04-24 11:45:03 +03:00
|
|
|
|
|
|
|
|
type Self <: ManualProbe[I]
|
|
|
|
|
|
|
|
|
|
private val probe: TestProbe = TestProbe()
|
|
|
|
|
|
|
|
|
|
private val self = this.asInstanceOf[Self]
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Subscribes a given [[org.reactivestreams.Subscriber]] to this probe publisher.
|
|
|
|
|
*/
|
|
|
|
|
def subscribe(subscriber: Subscriber[_ >: I]): Unit = {
|
|
|
|
|
val subscription: PublisherProbeSubscription[I] = new PublisherProbeSubscription[I](subscriber, probe)
|
|
|
|
|
probe.ref ! Subscribe(subscription)
|
2015-06-18 22:24:24 +02:00
|
|
|
if (autoOnSubscribe) subscriber.onSubscribe(subscription)
|
2015-04-24 11:45:03 +03:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Expect a subscription.
|
|
|
|
|
*/
|
|
|
|
|
def expectSubscription(): PublisherProbeSubscription[I] =
|
|
|
|
|
probe.expectMsgType[Subscribe].subscription.asInstanceOf[PublisherProbeSubscription[I]]
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Expect demand from a given subscription.
|
|
|
|
|
*/
|
|
|
|
|
def expectRequest(subscription: Subscription, n: Int): Self = {
|
|
|
|
|
probe.expectMsg(RequestMore(subscription, n))
|
|
|
|
|
self
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Expect no messages.
|
|
|
|
|
*/
|
|
|
|
|
def expectNoMsg(): Self = {
|
|
|
|
|
probe.expectNoMsg()
|
|
|
|
|
self
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Expect no messages for a given duration.
|
|
|
|
|
*/
|
|
|
|
|
def expectNoMsg(max: FiniteDuration): Self = {
|
|
|
|
|
probe.expectNoMsg(max)
|
|
|
|
|
self
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Receive messages for a given duration or until one does not match a given partial function.
|
|
|
|
|
*/
|
|
|
|
|
def receiveWhile[T](max: Duration = Duration.Undefined, idle: Duration = Duration.Inf, messages: Int = Int.MaxValue)(f: PartialFunction[PublisherEvent, T]): immutable.Seq[T] =
|
|
|
|
|
probe.receiveWhile(max, idle, messages)(f.asInstanceOf[PartialFunction[AnyRef, T]])
|
|
|
|
|
|
2015-09-09 22:17:51 -04:00
|
|
|
def expectEventPF[T](f: PartialFunction[PublisherEvent, T]): T =
|
|
|
|
|
probe.expectMsgPF[T](probe.remaining)(f.asInstanceOf[PartialFunction[Any, T]])
|
|
|
|
|
|
2015-04-24 11:45:03 +03:00
|
|
|
def getPublisher: Publisher[I] = this
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Single subscription and demand tracking for [[TestPublisher.ManualProbe]].
|
|
|
|
|
*/
|
|
|
|
|
class Probe[T] private[TestPublisher] (initialPendingRequests: Long)(implicit system: ActorSystem) extends ManualProbe[T] {
|
|
|
|
|
|
|
|
|
|
type Self = Probe[T]
|
|
|
|
|
|
|
|
|
|
private var pendingRequests = initialPendingRequests
|
|
|
|
|
private lazy val subscription = expectSubscription()
|
|
|
|
|
|
2015-10-07 20:00:53 +02:00
|
|
|
/** Asserts that a subscription has been received or will be received */
|
|
|
|
|
def ensureSubscription(): Unit = subscription // initializes lazy val
|
|
|
|
|
|
2015-04-24 11:45:03 +03:00
|
|
|
/**
|
|
|
|
|
* Current pending requests.
|
|
|
|
|
*/
|
|
|
|
|
def pending: Long = pendingRequests
|
|
|
|
|
|
|
|
|
|
def sendNext(elem: T): Self = {
|
|
|
|
|
if (pendingRequests == 0) pendingRequests = subscription.expectRequest()
|
|
|
|
|
pendingRequests -= 1
|
|
|
|
|
subscription.sendNext(elem)
|
|
|
|
|
this
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def unsafeSendNext(elem: T): Self = {
|
|
|
|
|
subscription.sendNext(elem)
|
|
|
|
|
this
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def sendComplete(): Self = {
|
|
|
|
|
subscription.sendComplete()
|
|
|
|
|
this
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def sendError(cause: Exception): Self = {
|
|
|
|
|
subscription.sendError(cause)
|
|
|
|
|
this
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def expectRequest(): Long = subscription.expectRequest()
|
|
|
|
|
|
|
|
|
|
def expectCancellation(): Self = {
|
|
|
|
|
subscription.expectCancellation()
|
|
|
|
|
this
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
object TestSubscriber {
|
|
|
|
|
|
2015-09-09 22:17:51 -04:00
|
|
|
trait SubscriberEvent extends DeadLetterSuppression with NoSerializationVerificationNeeded
|
|
|
|
|
final case class OnSubscribe(subscription: Subscription) extends SubscriberEvent
|
|
|
|
|
final case class OnNext[I](element: I) extends SubscriberEvent
|
|
|
|
|
final case object OnComplete extends SubscriberEvent
|
2015-10-31 14:46:10 +01:00
|
|
|
final case class OnError(cause: Throwable) extends SubscriberEvent {
|
|
|
|
|
override def toString: String = {
|
|
|
|
|
val str = new StringWriter
|
|
|
|
|
val out = new PrintWriter(str)
|
|
|
|
|
out.print("OnError(")
|
|
|
|
|
cause.printStackTrace(out)
|
|
|
|
|
out.print(")")
|
|
|
|
|
str.toString
|
|
|
|
|
}
|
|
|
|
|
}
|
2015-04-24 11:45:03 +03:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Probe that implements [[org.reactivestreams.Subscriber]] interface.
|
|
|
|
|
*/
|
|
|
|
|
def manualProbe[T]()(implicit system: ActorSystem): ManualProbe[T] = new ManualProbe()
|
|
|
|
|
|
|
|
|
|
def probe[T]()(implicit system: ActorSystem): Probe[T] = new Probe()
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Implementation of [[org.reactivestreams.Subscriber]] that allows various assertions.
|
2015-09-17 13:10:09 +02:00
|
|
|
*
|
|
|
|
|
* All timeouts are dilated automatically, for more details about time dilation refer to [[akka.testkit.TestKit]].
|
2015-04-24 11:45:03 +03:00
|
|
|
*/
|
|
|
|
|
class ManualProbe[I] private[TestSubscriber] ()(implicit system: ActorSystem) extends Subscriber[I] {
|
2015-09-17 13:10:09 +02:00
|
|
|
import akka.testkit._
|
2015-04-24 11:45:03 +03:00
|
|
|
|
|
|
|
|
type Self <: ManualProbe[I]
|
|
|
|
|
|
|
|
|
|
private val probe = TestProbe()
|
|
|
|
|
|
2015-09-17 12:57:07 +02:00
|
|
|
@volatile private var _subscription: Subscription = _
|
|
|
|
|
|
2015-04-24 11:45:03 +03:00
|
|
|
private val self = this.asInstanceOf[Self]
|
|
|
|
|
|
|
|
|
|
/**
|
2015-09-17 12:57:07 +02:00
|
|
|
* Expect and return a [[Subscription]].
|
|
|
|
|
*/
|
|
|
|
|
def expectSubscription(): Subscription = {
|
|
|
|
|
_subscription = probe.expectMsgType[OnSubscribe].subscription
|
|
|
|
|
_subscription
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Expect and return [[SubscriberEvent]] (any of: `OnSubscribe`, `OnNext`, `OnError` or `OnComplete`).
|
2015-04-24 11:45:03 +03:00
|
|
|
*/
|
2015-09-17 12:57:07 +02:00
|
|
|
def expectEvent(): SubscriberEvent =
|
|
|
|
|
probe.expectMsgType[SubscriberEvent]
|
2015-04-24 11:45:03 +03:00
|
|
|
|
2015-09-17 13:10:09 +02:00
|
|
|
/**
|
|
|
|
|
* Expect and return [[SubscriberEvent]] (any of: `OnSubscribe`, `OnNext`, `OnError` or `OnComplete`).
|
|
|
|
|
*/
|
|
|
|
|
def expectEvent(max: FiniteDuration): SubscriberEvent =
|
2015-09-29 10:26:18 +02:00
|
|
|
probe.expectMsgType[SubscriberEvent](max)
|
2015-09-17 13:10:09 +02:00
|
|
|
|
2015-04-24 11:45:03 +03:00
|
|
|
/**
|
2015-09-17 12:57:07 +02:00
|
|
|
* Fluent DSL
|
|
|
|
|
*
|
|
|
|
|
* Expect [[SubscriberEvent]] (any of: `OnSubscribe`, `OnNext`, `OnError` or `OnComplete`).
|
2015-04-24 11:45:03 +03:00
|
|
|
*/
|
2015-09-17 13:10:09 +02:00
|
|
|
def expectEvent(event: SubscriberEvent): Self = {
|
2015-04-24 11:45:03 +03:00
|
|
|
probe.expectMsg(event)
|
|
|
|
|
self
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2015-09-17 12:57:07 +02:00
|
|
|
* Expect and return a stream element.
|
|
|
|
|
*/
|
|
|
|
|
def expectNext(): I = probe.expectMsgType[OnNext[I]].element
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Fluent DSL
|
|
|
|
|
*
|
|
|
|
|
* Expect a stream element.
|
2015-04-24 11:45:03 +03:00
|
|
|
*/
|
|
|
|
|
def expectNext(element: I): Self = {
|
|
|
|
|
probe.expectMsg(OnNext(element))
|
|
|
|
|
self
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2015-09-17 12:57:07 +02:00
|
|
|
* Fluent DSL
|
|
|
|
|
*
|
|
|
|
|
* Expect multiple stream elements.
|
2015-04-24 11:45:03 +03:00
|
|
|
*/
|
|
|
|
|
@annotation.varargs def expectNext(e1: I, e2: I, es: I*): Self =
|
|
|
|
|
expectNextN((e1 +: e2 +: es).map(identity)(collection.breakOut))
|
|
|
|
|
|
2015-09-17 12:57:07 +02:00
|
|
|
/**
|
|
|
|
|
* Fluent DSL
|
|
|
|
|
*
|
|
|
|
|
* Expect multiple stream elements in arbitrary order.
|
|
|
|
|
*/
|
2015-04-24 11:45:03 +03:00
|
|
|
@annotation.varargs def expectNextUnordered(e1: I, e2: I, es: I*): Self =
|
|
|
|
|
expectNextUnorderedN((e1 +: e2 +: es).map(identity)(collection.breakOut))
|
|
|
|
|
|
|
|
|
|
/**
|
2015-09-17 12:57:07 +02:00
|
|
|
* Expect and return the next `n` stream elements.
|
2015-04-24 11:45:03 +03:00
|
|
|
*/
|
2015-09-17 12:57:07 +02:00
|
|
|
def expectNextN(n: Long): immutable.Seq[I] = {
|
|
|
|
|
val b = immutable.Seq.newBuilder[I]
|
|
|
|
|
var i = 0
|
|
|
|
|
while (i < n) {
|
|
|
|
|
val next = probe.expectMsgType[OnNext[I]]
|
|
|
|
|
b += next.element
|
|
|
|
|
i += 1
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
b.result()
|
|
|
|
|
}
|
2015-04-24 11:45:03 +03:00
|
|
|
|
2015-09-17 12:57:07 +02:00
|
|
|
/**
|
|
|
|
|
* Fluent DSL
|
|
|
|
|
* Expect the given elements to be signalled in order.
|
|
|
|
|
*/
|
2015-04-24 11:45:03 +03:00
|
|
|
def expectNextN(all: immutable.Seq[I]): Self = {
|
|
|
|
|
all.foreach(e ⇒ probe.expectMsg(OnNext(e)))
|
|
|
|
|
self
|
|
|
|
|
}
|
|
|
|
|
|
2015-09-17 12:57:07 +02:00
|
|
|
/**
|
|
|
|
|
* Fluent DSL
|
|
|
|
|
* Expect the given elements to be signalled in any order.
|
|
|
|
|
*/
|
2015-04-24 11:45:03 +03:00
|
|
|
def expectNextUnorderedN(all: immutable.Seq[I]): Self = {
|
|
|
|
|
@annotation.tailrec def expectOneOf(all: immutable.Seq[I]): Unit = all match {
|
|
|
|
|
case Nil ⇒
|
|
|
|
|
case list ⇒
|
|
|
|
|
val next = expectNext()
|
|
|
|
|
assert(all.contains(next), s"expected one of $all, but received $next")
|
|
|
|
|
expectOneOf(all.diff(Seq(next)))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
expectOneOf(all)
|
|
|
|
|
self
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2015-09-17 12:57:07 +02:00
|
|
|
* Fluent DSL
|
|
|
|
|
*
|
2015-04-24 11:45:03 +03:00
|
|
|
* Expect completion.
|
|
|
|
|
*/
|
|
|
|
|
def expectComplete(): Self = {
|
|
|
|
|
probe.expectMsg(OnComplete)
|
|
|
|
|
self
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2015-09-17 12:57:07 +02:00
|
|
|
* Expect and return the signalled [[Throwable]].
|
|
|
|
|
*/
|
|
|
|
|
def expectError(): Throwable = probe.expectMsgType[OnError].cause
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Fluent DSL
|
|
|
|
|
*
|
2015-04-24 11:45:03 +03:00
|
|
|
* Expect given [[Throwable]].
|
|
|
|
|
*/
|
|
|
|
|
def expectError(cause: Throwable): Self = {
|
|
|
|
|
probe.expectMsg(OnError(cause))
|
|
|
|
|
self
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2015-09-17 12:57:07 +02:00
|
|
|
* Expect subscription to be followed immediatly by an error signal.
|
|
|
|
|
*
|
|
|
|
|
* By default `1` demand will be signalled in order to wake up a possibly lazy upstream.
|
|
|
|
|
*
|
|
|
|
|
* See also [[#expectSubscriptionAndError(Boolean)]] if no demand should be signalled.
|
2015-04-24 11:45:03 +03:00
|
|
|
*/
|
2015-09-17 12:57:07 +02:00
|
|
|
def expectSubscriptionAndError(): Throwable = {
|
|
|
|
|
expectSubscriptionAndError(true)
|
|
|
|
|
}
|
2015-04-24 11:45:03 +03:00
|
|
|
|
2015-09-17 12:57:07 +02:00
|
|
|
/**
|
|
|
|
|
* Expect subscription to be followed immediatly by an error signal.
|
|
|
|
|
*
|
|
|
|
|
* Depending on the `signalDemand` parameter demand may be signalled immediatly after obtaining the subscription
|
|
|
|
|
* in order to wake up a possibly lazy upstream. You can disable this by setting the `signalDemand` parameter to `false`.
|
|
|
|
|
*
|
|
|
|
|
* See also [[#expectSubscriptionAndError()]].
|
|
|
|
|
*/
|
|
|
|
|
def expectSubscriptionAndError(signalDemand: Boolean): Throwable = {
|
2015-04-24 11:45:03 +03:00
|
|
|
val sub = expectSubscription()
|
2015-09-17 12:57:07 +02:00
|
|
|
if (signalDemand) sub.request(1)
|
|
|
|
|
expectError()
|
2015-04-24 11:45:03 +03:00
|
|
|
}
|
|
|
|
|
|
2015-09-17 12:57:07 +02:00
|
|
|
/**
|
|
|
|
|
* Fluent DSL
|
|
|
|
|
*
|
|
|
|
|
* Expect subscription followed by immediate stream completion.
|
|
|
|
|
*
|
|
|
|
|
* By default `1` demand will be signalled in order to wake up a possibly lazy upstream.
|
|
|
|
|
*
|
|
|
|
|
* See also [[#expectSubscriptionAndComplete(Throwable, Boolean)]] if no demand should be signalled.
|
|
|
|
|
*/
|
|
|
|
|
def expectSubscriptionAndError(cause: Throwable): Self =
|
|
|
|
|
expectSubscriptionAndError(cause, true)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Fluent DSL
|
|
|
|
|
*
|
|
|
|
|
* Expect subscription followed by immediate stream completion.
|
|
|
|
|
* By default `1` demand will be signalled in order to wake up a possibly lazy upstream
|
|
|
|
|
*
|
|
|
|
|
* See also [[#expectSubscriptionAndError(Throwable)]].
|
|
|
|
|
*/
|
|
|
|
|
def expectSubscriptionAndError(cause: Throwable, signalDemand: Boolean): Self = {
|
2015-04-24 11:45:03 +03:00
|
|
|
val sub = expectSubscription()
|
2015-09-17 12:57:07 +02:00
|
|
|
if (signalDemand) sub.request(1)
|
|
|
|
|
expectError(cause)
|
|
|
|
|
self
|
2015-04-24 11:45:03 +03:00
|
|
|
}
|
|
|
|
|
|
2015-09-17 12:57:07 +02:00
|
|
|
/**
|
|
|
|
|
* Fluent DSL
|
|
|
|
|
*
|
|
|
|
|
* Expect subscription followed by immediate stream completion.
|
|
|
|
|
* By default `1` demand will be signalled in order to wake up a possibly lazy upstream
|
|
|
|
|
*
|
|
|
|
|
* See also [[#expectSubscriptionAndComplete(Boolean)]] if no demand should be signalled.
|
|
|
|
|
*/
|
|
|
|
|
def expectSubscriptionAndComplete(): Self =
|
|
|
|
|
expectSubscriptionAndComplete(true)
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Fluent DSL
|
|
|
|
|
*
|
|
|
|
|
* Expect subscription followed by immediate stream completion.
|
|
|
|
|
*
|
|
|
|
|
* Depending on the `signalDemand` parameter demand may be signalled immediatly after obtaining the subscription
|
|
|
|
|
* in order to wake up a possibly lazy upstream. You can disable this by setting the `signalDemand` parameter to `false`.
|
|
|
|
|
*
|
|
|
|
|
* See also [[#expectSubscriptionAndComplete]].
|
|
|
|
|
*/
|
|
|
|
|
def expectSubscriptionAndComplete(signalDemand: Boolean): Self = {
|
2015-04-24 11:45:03 +03:00
|
|
|
val sub = expectSubscription()
|
2015-09-17 12:57:07 +02:00
|
|
|
if (signalDemand) sub.request(1)
|
2015-04-24 11:45:03 +03:00
|
|
|
expectComplete()
|
|
|
|
|
self
|
|
|
|
|
}
|
|
|
|
|
|
2015-09-17 12:57:07 +02:00
|
|
|
/**
|
|
|
|
|
* Fluent DSL
|
|
|
|
|
*
|
|
|
|
|
* Expect given next element or error signal, returning whichever was signalled.
|
|
|
|
|
*/
|
|
|
|
|
def expectNextOrError(): Either[Throwable, I] = {
|
|
|
|
|
probe.fishForMessage(hint = s"OnNext(_) or error") {
|
|
|
|
|
case OnNext(element) ⇒ true
|
|
|
|
|
case OnError(cause) ⇒ true
|
|
|
|
|
} match {
|
|
|
|
|
case OnNext(n: I @unchecked) ⇒ Right(n)
|
|
|
|
|
case OnError(err) ⇒ Left(err)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Fluent DSL
|
|
|
|
|
* Expect given next element or error signal.
|
|
|
|
|
*/
|
2015-04-24 11:45:03 +03:00
|
|
|
def expectNextOrError(element: I, cause: Throwable): Either[Throwable, I] = {
|
|
|
|
|
probe.fishForMessage(hint = s"OnNext($element) or ${cause.getClass.getName}") {
|
2015-09-17 12:57:07 +02:00
|
|
|
case OnNext(`element`) ⇒ true
|
|
|
|
|
case OnError(`cause`) ⇒ true
|
2015-04-24 11:45:03 +03:00
|
|
|
} match {
|
|
|
|
|
case OnNext(n: I @unchecked) ⇒ Right(n)
|
|
|
|
|
case OnError(err) ⇒ Left(err)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-09-17 12:57:07 +02:00
|
|
|
/**
|
|
|
|
|
* Expect next element or stream completion - returning whichever was signalled.
|
|
|
|
|
*/
|
|
|
|
|
def expectNextOrComplete(): Either[OnComplete.type, I] = {
|
|
|
|
|
probe.fishForMessage(hint = s"OnNext(_) or OnComplete") {
|
2015-04-24 11:45:03 +03:00
|
|
|
case OnNext(n) ⇒ true
|
|
|
|
|
case OnComplete ⇒ true
|
2015-09-17 12:57:07 +02:00
|
|
|
} match {
|
|
|
|
|
case OnComplete ⇒ Left(OnComplete)
|
|
|
|
|
case OnNext(n: I @unchecked) ⇒ Right(n)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Fluent DSL
|
|
|
|
|
*
|
|
|
|
|
* Expect given next element or stream completion.
|
|
|
|
|
*/
|
|
|
|
|
def expectNextOrComplete(element: I): Self = {
|
|
|
|
|
probe.fishForMessage(hint = s"OnNext($element) or OnComplete") {
|
|
|
|
|
case OnNext(`element`) ⇒ true
|
|
|
|
|
case OnComplete ⇒ true
|
2015-04-24 11:45:03 +03:00
|
|
|
}
|
|
|
|
|
self
|
|
|
|
|
}
|
|
|
|
|
|
2015-09-17 12:57:07 +02:00
|
|
|
/**
|
|
|
|
|
* Fluent DSL
|
|
|
|
|
*
|
|
|
|
|
* Same as `expectNoMsg(remaining)`, but correctly treating the timeFactor.
|
|
|
|
|
*/
|
2015-04-24 11:45:03 +03:00
|
|
|
def expectNoMsg(): Self = {
|
|
|
|
|
probe.expectNoMsg()
|
|
|
|
|
self
|
|
|
|
|
}
|
|
|
|
|
|
2015-09-17 12:57:07 +02:00
|
|
|
/**
|
|
|
|
|
* Fluent DSL
|
|
|
|
|
*
|
|
|
|
|
* Assert that no message is received for the specified time.
|
|
|
|
|
*/
|
|
|
|
|
def expectNoMsg(remaining: FiniteDuration): Self = {
|
|
|
|
|
probe.expectNoMsg(remaining)
|
2015-04-24 11:45:03 +03:00
|
|
|
self
|
|
|
|
|
}
|
|
|
|
|
|
2015-09-09 22:17:51 -04:00
|
|
|
def expectNextPF[T](f: PartialFunction[Any, T]): T = {
|
|
|
|
|
expectEventPF {
|
|
|
|
|
case OnNext(n) ⇒
|
|
|
|
|
assert(f.isDefinedAt(n))
|
|
|
|
|
f(n)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def expectEventPF[T](f: PartialFunction[SubscriberEvent, T]): T =
|
|
|
|
|
probe.expectMsgPF[T](probe.remaining)(f.asInstanceOf[PartialFunction[Any, T]])
|
|
|
|
|
|
2015-04-24 11:45:03 +03:00
|
|
|
/**
|
|
|
|
|
* Receive messages for a given duration or until one does not match a given partial function.
|
|
|
|
|
*/
|
|
|
|
|
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]])
|
|
|
|
|
|
2015-09-17 12:57:07 +02:00
|
|
|
/**
|
|
|
|
|
* Drains a given number of messages
|
|
|
|
|
*/
|
2015-06-19 15:05:05 +02:00
|
|
|
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
|
|
|
|
|
|
2015-09-17 12:57:07 +02:00
|
|
|
/**
|
|
|
|
|
* Attempt to drain the stream into a strict collection (by requesting `Long.MaxValue` elements).
|
|
|
|
|
*
|
|
|
|
|
* '''Use with caution: Be warned that this may not be a good idea if the stream is infinite or its elements are very large!'''
|
|
|
|
|
*/
|
|
|
|
|
def toStrict(atMost: FiniteDuration): immutable.Seq[I] = {
|
|
|
|
|
val deadline = Deadline.now + atMost
|
|
|
|
|
val b = immutable.Seq.newBuilder[I]
|
|
|
|
|
|
|
|
|
|
@tailrec def drain(): immutable.Seq[I] =
|
2015-09-17 13:10:09 +02:00
|
|
|
self.expectEvent(deadline.timeLeft) match {
|
2015-09-17 12:57:07 +02:00
|
|
|
case OnError(ex) ⇒
|
2015-09-17 13:10:09 +02:00
|
|
|
// TODO once on JDK7+ this could be made an AssertionError, since it can carry ex in its cause param
|
|
|
|
|
throw new AssertionError(s"toStrict received OnError(${ex.getMessage}) while draining stream! Accumulated elements: ${b.result()}")
|
2015-09-17 12:57:07 +02:00
|
|
|
case OnComplete ⇒
|
|
|
|
|
b.result()
|
|
|
|
|
case OnNext(i: I @unchecked) ⇒
|
|
|
|
|
b += i
|
|
|
|
|
drain()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// if no subscription was obtained yet, we expect it
|
|
|
|
|
if (_subscription == null) self.expectSubscription()
|
|
|
|
|
_subscription.request(Long.MaxValue)
|
|
|
|
|
|
|
|
|
|
drain()
|
|
|
|
|
}
|
|
|
|
|
|
2015-04-24 11:45:03 +03:00
|
|
|
def within[T](max: FiniteDuration)(f: ⇒ T): T = probe.within(0.seconds, max)(f)
|
|
|
|
|
|
|
|
|
|
def onSubscribe(subscription: Subscription): Unit = probe.ref ! OnSubscribe(subscription)
|
|
|
|
|
def onNext(element: I): Unit = probe.ref ! OnNext(element)
|
|
|
|
|
def onComplete(): Unit = probe.ref ! OnComplete
|
|
|
|
|
def onError(cause: Throwable): Unit = probe.ref ! OnError(cause)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Single subscription tracking for [[ManualProbe]].
|
|
|
|
|
*/
|
|
|
|
|
class Probe[T] private[TestSubscriber] ()(implicit system: ActorSystem) extends ManualProbe[T] {
|
|
|
|
|
|
|
|
|
|
override type Self = Probe[T]
|
|
|
|
|
|
|
|
|
|
private lazy val subscription = expectSubscription()
|
|
|
|
|
|
2015-10-07 20:00:53 +02:00
|
|
|
/** Asserts that a subscription has been received or will be received */
|
|
|
|
|
def ensureSubscription(): Unit = subscription // initializes lazy val
|
|
|
|
|
|
2015-04-24 11:45:03 +03:00
|
|
|
def request(n: Long): Self = {
|
|
|
|
|
subscription.request(n)
|
|
|
|
|
this
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def requestNext(element: T): Self = {
|
|
|
|
|
subscription.request(1)
|
|
|
|
|
expectNext(element)
|
|
|
|
|
this
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def cancel(): Self = {
|
|
|
|
|
subscription.cancel()
|
|
|
|
|
this
|
|
|
|
|
}
|
2015-08-20 16:13:04 +02:00
|
|
|
|
|
|
|
|
def requestNext(): T = {
|
|
|
|
|
subscription.request(1)
|
|
|
|
|
expectNext()
|
|
|
|
|
}
|
2015-04-24 11:45:03 +03:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[testkit] object StreamTestKit {
|
2015-09-09 22:17:51 -04:00
|
|
|
import TestPublisher._
|
2015-04-24 11:45:03 +03:00
|
|
|
|
|
|
|
|
final case class CompletedSubscription[T](subscriber: Subscriber[T]) extends Subscription {
|
|
|
|
|
override def request(elements: Long): Unit = subscriber.onComplete()
|
|
|
|
|
override def cancel(): Unit = ()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
final case class FailedSubscription[T](subscriber: Subscriber[T], cause: Throwable) extends Subscription {
|
|
|
|
|
override def request(elements: Long): Unit = subscriber.onError(cause)
|
|
|
|
|
override def cancel(): Unit = ()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
final case class PublisherProbeSubscription[I](subscriber: Subscriber[_ >: I], publisherProbe: TestProbe) extends Subscription {
|
|
|
|
|
def request(elements: Long): Unit = publisherProbe.ref ! RequestMore(this, elements)
|
|
|
|
|
def cancel(): Unit = publisherProbe.ref ! CancelSubscription(this)
|
|
|
|
|
|
|
|
|
|
def expectRequest(n: Long): Unit = publisherProbe.expectMsg(RequestMore(this, n))
|
|
|
|
|
def expectRequest(): Long = publisherProbe.expectMsgPF() {
|
|
|
|
|
case RequestMore(sub, n) if sub eq this ⇒ n
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def expectCancellation(): Unit = publisherProbe.fishForMessage() {
|
|
|
|
|
case CancelSubscription(sub) if sub eq this ⇒ true
|
|
|
|
|
case RequestMore(sub, _) if sub eq this ⇒ false
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def sendNext(element: I): Unit = subscriber.onNext(element)
|
|
|
|
|
def sendComplete(): Unit = subscriber.onComplete()
|
|
|
|
|
def sendError(cause: Exception): Unit = subscriber.onError(cause)
|
2015-06-18 22:24:24 +02:00
|
|
|
|
|
|
|
|
def sendOnSubscribe(): Unit = subscriber.onSubscribe(this)
|
2015-04-24 11:45:03 +03:00
|
|
|
}
|
|
|
|
|
|
2015-06-23 17:32:55 +02:00
|
|
|
final class ProbeSource[T](val attributes: Attributes, shape: SourceShape[T])(implicit system: ActorSystem) extends SourceModule[T, TestPublisher.Probe[T]](shape) {
|
2015-04-24 11:45:03 +03:00
|
|
|
override def create(context: MaterializationContext) = {
|
|
|
|
|
val probe = TestPublisher.probe[T]()
|
|
|
|
|
(probe, probe)
|
|
|
|
|
}
|
|
|
|
|
override protected def newInstance(shape: SourceShape[T]): SourceModule[T, TestPublisher.Probe[T]] = new ProbeSource[T](attributes, shape)
|
2015-06-23 17:32:55 +02:00
|
|
|
override def withAttributes(attr: Attributes): Module = new ProbeSource[T](attr, amendShape(attr))
|
2015-04-24 11:45:03 +03:00
|
|
|
}
|
|
|
|
|
|
2015-06-23 17:32:55 +02:00
|
|
|
final class ProbeSink[T](val attributes: Attributes, shape: SinkShape[T])(implicit system: ActorSystem) extends SinkModule[T, TestSubscriber.Probe[T]](shape) {
|
2015-04-24 11:45:03 +03:00
|
|
|
override def create(context: MaterializationContext) = {
|
|
|
|
|
val probe = TestSubscriber.probe[T]()
|
|
|
|
|
(probe, probe)
|
|
|
|
|
}
|
|
|
|
|
override protected def newInstance(shape: SinkShape[T]): SinkModule[T, TestSubscriber.Probe[T]] = new ProbeSink[T](attributes, shape)
|
2015-06-23 17:32:55 +02:00
|
|
|
override def withAttributes(attr: Attributes): Module = new ProbeSink[T](attr, amendShape(attr))
|
2015-04-24 11:45:03 +03:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|